diff --git a/.gitignore b/.gitignore index e5079d190a4..e7e91890044 100644 --- a/.gitignore +++ b/.gitignore @@ -64,10 +64,7 @@ src/version.h src/config.h java/openmldb-native/src/main/resources java/openmldb-native/src/main/java -java/openmldb-common/src/main/java/com/_4paradigm/openmldb/proto/ -java/openmldb-import/src/main/java/com/_4paradigm/openmldb/proto/ -java/openmldb-nearline-tablet/src/main/java/com/_4paradigm/openmldb/proto/ -java/openmldb-taskmanager/src/main/java/com/_4paradigm/openmldb/proto/ +java/openmldb-*/src/main/java/com/_4paradigm/openmldb/proto/ java/hybridse-native/src/main/java java/hybridse-proto/src **/.mvn/wrapper/maven-wrapper.jar diff --git a/docs/en/developer/code_structure.md b/docs/en/developer/code_structure.md index d5a9cfb275f..cb06e26dad6 100644 --- a/docs/en/developer/code_structure.md +++ b/docs/en/developer/code_structure.md @@ -34,7 +34,8 @@ src/ ├── catalog ├── client // the difinition and implementation of ns/tablet/taskmanager client interfaces ├── cmd // CLI and OpenMLDB binary generation -├── codec // decode and encode +├── codec // decode and encode +├── datacollector // online -> offline sync tool ├── log // the formats, reading and writing of binlog and snapshot ├── nameserver ├── proto // definition of protobuf @@ -63,6 +64,7 @@ java/ ├── openmldb-jmh // used for performance and stability testing ├── openmldb-native // codes generated automatically by swig ├── openmldb-spark-connector // the implementation of spark connector used for reading from and writing into OpenMLDB +├── openmldb-synctool // online -> offline sync tool └── openmldb-taskmanager // offline tasks management module ``` diff --git a/docs/zh/deploy/index.rst b/docs/zh/deploy/index.rst index a86f5ee69e9..5c067bac54f 100644 --- a/docs/zh/deploy/index.rst +++ b/docs/zh/deploy/index.rst @@ -10,3 +10,4 @@ compile integrate_hadoop offline_integrate_kubernetes + online_offline_sync diff --git a/docs/zh/deploy/online_offline_sync.md b/docs/zh/deploy/online_offline_sync.md new file mode 100644 index 00000000000..fd4f30420c0 --- /dev/null +++ b/docs/zh/deploy/online_offline_sync.md @@ -0,0 +1,87 @@ +# 在离线数据同步 + +在离线数据同步,指将在线数据同步到离线地址,离线地址指大容量持久化存储地址,用户可以自行指定,不一定是OpenMLDB表中的离线数据地址,仅支持写入到hdfs集群。 + +开启在离线同步功能,需要部署两种组件,DataCollector和SyncTool。一期仅支持单个SyncTool,DataCollector需要在**每台部署TabletServer的机器**上**至少**部署一台。举例说明,一台机器上可以存在多个TabletServer,同步任务将使用该机器上的一个DataCollector,如果你添加了更多的DataCollector,它们不会工作,直到运行的DataCollector下线,将由下一个DataCollector代替以继续工作。 + +虽然SyncTool仅支持单体运行,但它再启动时可恢复工作进度,无需额外操作。 + +## 部署方式 + +由于SyncTool有状态,如果先启动它,可能会在无DataCollector的情况下尝试分配同步任务。所以,请保证先启动所有DataCollector,再启动SyncTool。 + +部署包请从github release或镜像网站中下载,版本>0.7.3,并解压。不可使用旧版本的TabletServer进行同步。 + +### DataCollector + +#### 配置(重点) + +更新`/conf/data_collector.flags`配置文件。配置中请填写正确的zk地址和路径,以及配置无端口冲突`endpoint`(endpoint与TabletServer保持一致,如果TabletServer使用本机的公网IP,DataCollector endpoint使用127.0.0.1地址,无法自动转换)。 + +需要注意的是,请慎重选择`collector_datadir`。我们将在同步中对TabletServer的磁盘数据进行硬链接,所以`collector_datadir`需要与TabletServer的数据地址`hdd_root_path`/`ssd_root_path`在同一磁盘上,否则报错`Invalid cross-device link`。 + +`max_pack_size`默认1M,如果同步任务过多,容易出现`[E1011]The server is overcrowded`,请再适当调小此配置。也可适当调整`socket_max_unwritten_bytes`,增大写缓存容忍度。 + +#### 启动 + +``` +./bin/start.sh start data_collector +``` +#### 状态确认 + +启动后使用以下命令,可以获得实时的DataCollector RPC状态页面。如果失败,查询日志。 +``` +curl http:///status +``` +当前我们无法查询 `DataCollector ` 列表,将来会在相关工具中提供支持。 + +### SyncTool + +#### 配置 +- 请更新`/conf/synctool.properties`配置,在start时它将覆盖`/synctool/conf/synctool.properties`。 +- 当前只支持直写到HDFS,可通过properties文件配置`hadoop.conf.dir`或环境变量`HADOOP_CONF_DIR`来配置HDFS连接,请保证SyncTool的OS启动用户拥有HDFS路径(路径由每个同步任务创建时指定)的写权限。 + +#### 启动 +``` +./bin/start.sh start synctool +``` + +SyncTool目前只支持单进程运行,如果启动多个,它们互相独立,需要用户自行判断是否有重复的同步任务。SyncTool实时保存进度,如果下线,可原地启动恢复任务进度。 + +SyncTool负责同步任务的管理和数据收集、写入离线地址。首先,说明一下任务关系,SyncTool收到用户的“表同步任务”,将会被分割为多个“分片同步任务”(后续简称为子任务)进行创建和管理。 + +任务管理中,如果DataCollector掉线或出错,将会让DataCollector重新创建任务。如果重新赋予任务时,找不到合适的DataCollector,将会标记任务失败。如果不这样,SyncTool将会一直尝试赋予新任务,同步任务进度停滞,错误也不明显,所以为了及时发现问题,这种情况将标记子任务为失败。 + +由于创建表同步任务时不支持从某个点开始,所以当前情况下,不适合删除任务再创建,如果目的地一样,会有较多重复数据。可以考虑更换同步目的地,或者重启SyncTool(SyncTool recover时不会检查子任务是否曾经failed,会当作init状态开始任务)。 + +#### SyncTool Helper + +创建、删除与查询同步任务,使用`/tools/synctool_helper.py`。 + +```bash +# create +python tools/synctool_helper.py create -t db.table -m 1 -ts 233 -d /tmp/hdfs-dest [ -s ] +# delete +python tools/synctool_helper.py delete -t db.table [ -s ] +# task status +python tools/synctool_helper.py status [ -s ] +# sync tool status for dev +python tools/synctool_helper.py tool-status [ -f ] +``` + +Mode配置填写0/1/2,分别对应全量同步FULL,按时间增量同步INCREMENTAL_BY_TIMESTAMP,完全增量同步FULL_AND_CONTINUOUS三种模式。如果是Mode 1,使用`-ts`配置起始时间,小于该时间的数据将不会被同步。 + +Mode 0 没有严格的终止时间点,当每个子任务同步完当前的数据后,就会结束,结束会删除整个表任务,如果使用helper查询status没有该表的同步任务,则视为该表同步任务完成。Mode 1/2 都不会停止,将永远运行。 + +status结果说明: + +执行命令status将会打印每个partition task的状态,如果你只关注整体情况,可以只查看`table scope`之后的内容,它展示了表级别的同步任务状态,如果有某表存在`FAILED`子任务,也会提示。 + +对于每个子任务而言,注意其'status'字段,如果它是刚启动,还未收到DataCollector的第一次数据同步,将会是INIT状态。收到第一次数据同步后,将变为RUNNING状态。(我们尤其关注DataCollector和SyncTool的初始状态,所以,特别设置INIT状态。)如果同步任务是随着SyncTool重启而恢复,将直接进入RUNNING状态。任务在过程中可能出现REASSIGNING状态,这是中间状态,不代表任务已经不可用。在Mode 0,可能出现SUCCESS状态,表示任务已完成。当一张表的所有子任务都完成时,SyncTool将自动清理掉该表的任务,使用helper将查询不到该表的任务。 + +只有FAILED表示该子任务失败,不会重试,也不会删除该任务。确认失败原因且修复后,可以删除再重建同步任务。如果不想要丢失已导入的进度,可以重启SyncTool,让SyncTool恢复任务继续同步(但可能出现更多的重复数据)。 + +## 功能边界 + +DataCollector中对表的进度(snapshot进度)标记方式没有唯一性,如果子task中途shutdown,用当前进度创建任务,可能有一段重复数据。 +SyncTool HDFS先写入再持久化进度,如果此时SyncTool shutdown,由于进度没更新,将会重复写入一段数据。由于此功能必然有重复隐患,此处暂时不做额外工作。 diff --git a/docs/zh/developer/code_structure.md b/docs/zh/developer/code_structure.md index 080605cc2bb..5fc88ace2c9 100644 --- a/docs/zh/developer/code_structure.md +++ b/docs/zh/developer/code_structure.md @@ -35,6 +35,7 @@ src/ ├── client // ns/tablet/taskmanager client的接口定义和实现 ├── cmd // CLI以及openmldb二进制生成相关 ├── codec // 编解码相关 +├── datacollector // 在离线同步工具 ├── log // binlog和snapshot格式以及读写 ├── nameserver // nameserver相关 ├── proto // protobuf相关定义 @@ -63,6 +64,7 @@ java/ ├── openmldb-jmh // 用作性能和稳定性测试相关 ├── openmldb-native // swig自动生成的代码 ├── openmldb-spark-connector // spark的connector实现,用来读写OpenMLDB +├── openmldb-synctool // 在离线同步工具 └── openmldb-taskmanager // 离线任务管理模块 ``` diff --git a/java/openmldb-jdbc/src/main/java/com/_4paradigm/openmldb/jdbc/Statement.java b/java/openmldb-jdbc/src/main/java/com/_4paradigm/openmldb/jdbc/Statement.java index 78e58533e23..79aeca51555 100644 --- a/java/openmldb-jdbc/src/main/java/com/_4paradigm/openmldb/jdbc/Statement.java +++ b/java/openmldb-jdbc/src/main/java/com/_4paradigm/openmldb/jdbc/Statement.java @@ -39,6 +39,7 @@ public ResultSet getResultSet() throws SQLException { return new SQLResultSet(resultSet); } + // TODO(hw): why return sqlresultset? @Override public SQLResultSet executeQuery(String sql) throws SQLException { checkClosed(); diff --git a/java/openmldb-jdbc/src/main/java/com/_4paradigm/openmldb/sdk/SqlExecutor.java b/java/openmldb-jdbc/src/main/java/com/_4paradigm/openmldb/sdk/SqlExecutor.java index ae47bc25699..c89e53379bd 100644 --- a/java/openmldb-jdbc/src/main/java/com/_4paradigm/openmldb/sdk/SqlExecutor.java +++ b/java/openmldb-jdbc/src/main/java/com/_4paradigm/openmldb/sdk/SqlExecutor.java @@ -77,7 +77,7 @@ PreparedStatement getBatchRequestPreparedStmt(String db, String sql, ProcedureInfo showProcedure(String dbName, String proName) throws SQLException; - NS.TableInfo getTableInfo(String db, String table) throws SQLException; + NS.TableInfo getTableInfo(String db, String table); List getTableNames(String db); diff --git a/java/openmldb-synctool/pom.xml b/java/openmldb-synctool/pom.xml new file mode 100644 index 00000000000..54df04c3402 --- /dev/null +++ b/java/openmldb-synctool/pom.xml @@ -0,0 +1,218 @@ + + + 4.0.0 + + openmldb-parent + com.4paradigm.openmldb + 0.7.0-SNAPSHOT + + openmldb-synctool + openmldb-synctool + jar + + UTF-8 + 8 + 8 + 3.2.4 + 1.14.6 + 2.12 + + + + + junit + junit + 4.13.1 + test + + + + com.4paradigm.openmldb + openmldb-common + ${project.parent.version} + + + com.4paradigm.openmldb + openmldb-jdbc + ${project.parent.version} + + + + com.google.protobuf + protobuf-java + 3.16.3 + + + + com.baidu + brpc-java + 3.0.2 + + + + + org.apache.curator + curator-client + 4.2.0 + + + + + org.slf4j + slf4j-api + 1.7.25 + + + + + org.slf4j + slf4j-log4j12 + 1.7.25 + + + slf4j-api + org.slf4j + + + + + + + + com.google.guava + guava + 30.1.1-jre + + + + net.jcip + jcip-annotations + 1.0 + + + + + + org.apache.flink + flink-java + ${flink.version} + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + + + org.apache.flink + flink-clients_${scala.binary.version} + ${flink.version} + + + org.apache.flink + flink-runtime-web_${scala.binary.version} + ${flink.version} + + + + + org.apache.flink + flink-connector-files + ${flink.version} + + + + + + org.apache.hadoop + hadoop-client + ${hdfs.version} + + + + + + org.apache.hadoop + hadoop-minicluster + ${hdfs.version} + test + + + + org.apache.commons + commons-csv + 1.9.0 + + + + + + + + src/main/resources + + **/*.properties + **/*.xml + **/*.conf + + true + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 2.3.2 + + 1.8 + 1.8 + + + + org.codehaus.mojo + appassembler-maven-plugin + 1.3 + + + make-assembly + package + + assemble + + + + + conf + src/main/resources + true + true + flat + lib + ${project.build.directory}/${project.name}-binary + + .sh + + + unix + + + + com._4paradigm.openmldb.synctool.SyncTool + synctool + + + + + + + + + + openmldb + openmldb sdk repo + https://repo.maven.apache.org/maven2 + + + diff --git a/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/CsvFilePathFilter.java b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/CsvFilePathFilter.java new file mode 100644 index 00000000000..6eb902804d3 --- /dev/null +++ b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/CsvFilePathFilter.java @@ -0,0 +1,31 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com._4paradigm.openmldb.synctool; + +import org.apache.flink.api.common.io.FilePathFilter; +import org.apache.flink.core.fs.Path; + +class CsvFilePathFilter extends FilePathFilter { + @Override + public boolean filterPath(Path filePath) { + return filePath == null + || filePath.getName().startsWith(".") + || filePath.getName().startsWith("_") + || filePath.getName().contains(HADOOP_COPYING) + || !filePath.getName().endsWith(".csv"); + } +} diff --git a/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/DataCollectorService.java b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/DataCollectorService.java new file mode 100644 index 00000000000..8b6ca6621be --- /dev/null +++ b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/DataCollectorService.java @@ -0,0 +1,26 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com._4paradigm.openmldb.synctool; + +import com._4paradigm.openmldb.proto.DataSync; +import com.baidu.brpc.protocol.BrpcMeta; + +public interface DataCollectorService { + // c++ serviceName doesn't contain the package name. + @BrpcMeta(serviceName = "DataCollector", methodName = "AddSyncTask") + DataSync.GeneralResponse addSyncTask(DataSync.AddSyncTaskRequest request); +} diff --git a/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/DataParser.java b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/DataParser.java new file mode 100644 index 00000000000..fb81305e02c --- /dev/null +++ b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/DataParser.java @@ -0,0 +1,91 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com._4paradigm.openmldb.synctool; + +import com._4paradigm.openmldb.proto.Common.ColumnDesc; + +import java.io.BufferedWriter; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.List; + +import com._4paradigm.openmldb.common.codec.RowView; + +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.StringEscapeUtils; +import com.google.common.base.Preconditions; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class DataParser { + private final ByteBuf data; + private final long count; + private final RowView rowView; + + public DataParser(ByteBuf data, final long count, List schema) throws Exception { + this.data = data; + this.count = count; + this.rowView = new RowView(schema); + } + + // spark-readable format, escape string, null is null + public static String rowToString(RowView rowView) throws Exception { + StringBuilder sb = new StringBuilder(); + List schema = rowView.getSchema(); + for (int i = 0; i < schema.size(); i++) { + if (i != 0) { + sb.append(","); + } + ColumnDesc col = schema.get(i); + Object value = rowView.getValue(i, col.getDataType()); + String appendStr = value == null ? "null" : value.toString(); + if (value instanceof String) { + // escape for string + appendStr = String.format("\"%s\"", StringEscapeUtils.escapeJson((String) value)); + } + // bool, date and timestamp string can be read by spark + // bool: true, date: 2023-04-11, timestamp: 2001-09-09T09:46:39.999+08:00 + sb.append(appendStr); + } + return sb.toString(); + } + + public void writeAll(BufferedWriter bufferedWriter) { + try { + long readCount = 0; + while (readCount < count) { + int start = data.readerIndex(); + data.readerIndex(start + 2); // skip 2 bytes + int rowSize = data.readIntLE(); + log.debug("read row size: {}", rowSize); + // netty is big endian, but we use little endian + ByteBuffer row = data.nioBuffer(start, rowSize).order(ByteOrder.LITTLE_ENDIAN); + Preconditions.checkState(rowView.reset(row, rowSize), "reset row view failed"); + data.readerIndex(start + rowSize); + bufferedWriter.write(rowToString(rowView)); + bufferedWriter.write("\n"); + readCount++; + } + Preconditions.checkState(readCount == count, + String.format("read count not match, read: %d, expect: %d", readCount, count)); + } catch (IOException e) { + throw new RuntimeException("data write failed", e); + } catch (Exception e) { + throw new RuntimeException("data parse failed", e); + } + } +} diff --git a/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/FlinkTunnel.java b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/FlinkTunnel.java new file mode 100644 index 00000000000..2fba2d11736 --- /dev/null +++ b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/FlinkTunnel.java @@ -0,0 +1,263 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com._4paradigm.openmldb.synctool; + +import org.apache.flink.api.common.serialization.Encoder; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.filesystem.BucketAssigner; +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.SimpleVersionedStringSerializer; +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.OnCheckpointRollingPolicy; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.environment.CheckpointConfig; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; +import org.apache.flink.api.java.io.TextInputFormat; +import org.apache.flink.streaming.api.functions.source.FileProcessingMode; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; + +import java.io.PrintStream; +import java.net.HttpURLConnection; +import java.net.URI; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Collection; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import lombok.extern.slf4j.Slf4j; +import lombok.*; + +@Slf4j +public class FlinkTunnel { + + private static FlinkTunnel instance; + + public static synchronized FlinkTunnel getInstance() { + if (instance == null) { + instance = new FlinkTunnel(); + try { + SyncToolConfig.parse(); + instance.init(SyncToolConfig.getProp()); + } catch (Exception e) { + e.printStackTrace(); + instance = null; + } + } + return instance; + } + + @Getter + private MiniCluster miniCluster; + private URI miniClusterAddr; + private StreamExecutionEnvironment env; + // + private Map sourceMap = new ConcurrentHashMap<>(); + + private FlinkTunnel() { + } + + private void init(Properties prop) throws Exception { + Configuration configuration = Configuration.fromMap(Maps.fromProperties(prop)); + // ExecutionOptions + configuration.setString("execution.runtime-mode", "STREAMING"); + // HighAvailabilityOptions ref zookeeper ha + configuration.setString("high-availability", "zookeeper"); + configuration.setString("high-availability.zookeeper.quorum", SyncToolConfig.ZK_CLUSTER); + + // **Finished** :流模式(`STREAMING`)下的成功的 Checkpoint 或者批模式(`BATCH`)下输入结束,文件的 + // Pending 状态转换为 Finished 状态 + MiniClusterConfiguration miniClusterConfig = new MiniClusterConfiguration.Builder() + .setConfiguration(configuration) + .setNumTaskManagers(1) + .setNumSlotsPerTaskManager(SyncToolConfig.FLINK_SLOTS) // can't run job num > slot? + .build(); + + miniCluster = new MiniCluster(miniClusterConfig); + miniCluster.start(); + + miniClusterAddr = miniCluster.getRestAddress().get(); + log.info("start a mini cluster, addr: {}", miniClusterAddr); + + env = StreamExecutionEnvironment.createRemoteEnvironment( + miniClusterAddr.getHost(), + miniClusterAddr.getPort(), + configuration); + // env = StreamExecutionEnvironment.getExecutionEnvironment(configuration); + env.setParallelism(1); + env.enableCheckpointing(5000); // ms + CheckpointConfig checkpointConfig = env.getCheckpointConfig(); + // save the checkpoint to recover from SyncTool failures + checkpointConfig.setExternalizedCheckpointCleanup( + CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); + // one for one sync tool? + checkpointConfig.setCheckpointStorage("file:///tmp/flink-checkpoints/"); + // setStateBackend? + + // recover jobs + Collection oldJobs = miniCluster.listJobs().get(); + // old jobs start time will be updated + // if running job no source bytes received, it will be canceled? + // get Dataflow Plan by http or client + + // TODO how about watch a dir for a table? no append to old file, but create new + // files + // test about it + // so job is unique for a table + for (JobStatusMessage job : oldJobs) { + if (!job.getJobState().isGloballyTerminalState()) { + log.info("old job: {}", job); + String name = job.getJobName(); + int tid = Integer.parseInt(name.split("-")[2]); + java.nio.file.Path cachePath = Paths.get(SyncToolImpl.genCacheDir(tid)); + if (Files.notExists(cachePath)) { + log.warn("cache dir not exists(weird), create it: {}", cachePath); + Files.createDirectories(cachePath); + } + sourceMap.put(tid, job.getJobId()); + } + } + log.info("recovered jobs: {}", sourceMap); + } + + public synchronized boolean recoverTunnel(int tid) { + // already recovered when init, so just check here + return sourceMap.containsKey(tid); + } + + // synchronized to avoid create twice in the same time + // init phase is quite slow, no need to do lock free + public synchronized boolean createTunnel(int tid, String sourcePath, String sinkPath) { + if (sourceMap.containsKey(tid)) { + log.warn("tunnel for tid: {} already exists", tid); + return false; + } + // for flink, we should add the prefix "file://" + String fsURI = "file://" + sourcePath; + log.info("create source for tid: {}, fsURI: {}", tid, fsURI); + TextInputFormat format = new TextInputFormat(new org.apache.flink.core.fs.Path(fsURI)); + format.setFilesFilter(new CsvFilePathFilter()); + + long INTERVAL = 100; + // String for one line, or List for one line? + // or byte for one line? + // set uid for source, so we can recover? + DataStream source = env.readFile(format, fsURI, FileProcessingMode.PROCESS_CONTINUOUSLY, INTERVAL) + .setParallelism(1).name("source" + tid); + + // file suffix + // parquet is better? parquet should use bulk format + // TODO a test way, link to hdfs later + // if multi sink to one table path, better to use tid-pid as suffix(only one + // task running for one tid-pid), so redesign the bucket assigner? + final FileSink sink = FileSink + .forRowFormat(new Path(sinkPath), + (Encoder) (element, stream) -> { + PrintStream out = new PrintStream(stream); + // TODO(hw): escape? + out.println(element); + }) + // all to 0/, don't split by hour + .withBucketAssigner(new KeyBucketAssigner()) + .withRollingPolicy(OnCheckpointRollingPolicy.build()) + .build(); + + // Streaming streamingFileSink + source.sinkTo(sink); + + JobClient client; + try { + client = env.executeAsync(genJobName(tid)); + } catch (Exception e) { + return false; + } + sourceMap.put(tid, client.getJobID()); + log.info("create job for tid: {} success, job {}", tid, client.getJobID()); + return true; + } + + public JobStatus getJobStatus(int tid) { + JobID id = sourceMap.get(tid); + if (id == null) { + log.error("tid: {} not exist", tid); + return null; + } + try { + // rpc may failed + return miniCluster.getJobStatus(id).get(); + } catch (InterruptedException | ExecutionException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + return null; + } + + public void close() { + + } + + public String genJobName(int tid) { + return "stream-tid-" + tid; + } + + public synchronized void closeTunnel(int tid) { + JobID id = Preconditions.checkNotNull(sourceMap.get(tid)); + // TODO too soon, the job may not be finished + // test get job status by rest api + try { + URL url = miniClusterAddr.resolve("/jobs/" + id).toURL(); + log.info("check job: {}", url); + HttpURLConnection con = (HttpURLConnection) url.openConnection(); + con.setRequestMethod("GET"); + log.info(con.getResponseMessage()); + } catch (Exception e) { + } + // targetDirectory: null if the default savepointdirectory should be used + // terminate, not suspend + // miniCluster.stopWithSavepoint(id, null, true); // can't? + miniCluster.cancelJob(id); // checkpoint is retain on cancellation, but it's better to wait a while? + + // task record count? == source count or wahtever? + } + + /** Use first field for buckets. */ + public static final class KeyBucketAssigner + implements BucketAssigner { + private static final long serialVersionUID = 987325769970523326L; + + @Override + public String getBucketId(final String element, final Context context) { + return "0"; + } + + @Override + public SimpleVersionedSerializer getSerializer() { + return SimpleVersionedStringSerializer.INSTANCE; + } + } +} diff --git a/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/HDFSTunnel.java b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/HDFSTunnel.java new file mode 100644 index 00000000000..1c3f98f62d6 --- /dev/null +++ b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/HDFSTunnel.java @@ -0,0 +1,119 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com._4paradigm.openmldb.synctool; + +import java.io.BufferedWriter; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ConcurrentHashMap; + +import com._4paradigm.openmldb.proto.Common.ColumnDesc; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import com.google.common.base.Preconditions; +import io.netty.buffer.ByteBuf; + +class HDFSTunnel { + + private static HDFSTunnel instance; + + public static synchronized HDFSTunnel getInstance() { + if (instance == null) { + instance = new HDFSTunnel(); + try { + SyncToolConfig.parse(); + instance.init(SyncToolConfig.getProp()); + } catch (Exception e) { + e.printStackTrace(); + instance = null; + } + } + return instance; + } + + private FileSystem fileSystem; + // + private Map sourceMap = new ConcurrentHashMap<>(); + + private void init(Properties prop) throws Exception { + String confDir = SyncToolConfig.HADOOP_CONF_DIR; + Configuration configuration = new Configuration(); + configuration.addResource(new Path(confDir + "/core-site.xml")); + configuration.addResource(new Path(confDir + "/hdfs-site.xml")); + // else will throw org.apache.hadoop.fs.UnsupportedFileSystemException: No + // FileSystem for scheme "hdfs" + if (configuration.get("fs.hdfs.impl") == null) { + configuration.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem"); + } + fileSystem = FileSystem.get(configuration); + } + + public boolean recoverTunnel(int tid, String sinkPath) { + return createTunnel(tid, null, sinkPath); + } + + // for hdfs, sourcePath is useless, write to hdfs sinkPath + public synchronized boolean createTunnel(int tid, String sourcePath, String sinkPath) { + Preconditions.checkState(!sourceMap.containsKey(tid), "tunnel already exists for tid " + tid); + // create the directory + Path path = new Path(sinkPath); + try { + if (!fileSystem.exists(path)) { + fileSystem.mkdirs(path); + } + } catch (IOException e) { + e.printStackTrace(); + return false; + } + // end with / + if (!sinkPath.endsWith("/")) { + sinkPath += "/"; + } + sourceMap.put(tid, sinkPath); + return true; + } + + public void closeTunnel(int tid) { + sourceMap.remove(tid); + } + + // not thread safe + public void writeData(int tid, ByteBuf data, long count, List columnDescList) { + try { + DataParser parser = new DataParser(data, count, columnDescList); + String sinkPath = sourceMap.get(tid); + String fileName = SyncToolImpl.uniqueFileName(); + Path hdfsWritePath = new Path(sinkPath + fileName); + // Progressable? + FSDataOutputStream fsDataOutputStream = fileSystem.create(hdfsWritePath); + BufferedWriter bufferedWriter = new BufferedWriter( + new OutputStreamWriter(fsDataOutputStream, StandardCharsets.UTF_8)); + parser.writeAll(bufferedWriter); + bufferedWriter.close(); + } catch (Exception e) { + throw new RuntimeException("write to hdfs failed", e); + } + } + +} diff --git a/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/IgnorableException.java b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/IgnorableException.java new file mode 100644 index 00000000000..991bbdd1c1f --- /dev/null +++ b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/IgnorableException.java @@ -0,0 +1,7 @@ +package com._4paradigm.openmldb.synctool; + +public class IgnorableException extends Exception { + public IgnorableException(String message) { + super(message); + } +} diff --git a/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncTask.java b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncTask.java new file mode 100644 index 00000000000..2372fef599b --- /dev/null +++ b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncTask.java @@ -0,0 +1,208 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com._4paradigm.openmldb.synctool; + +import com._4paradigm.openmldb.proto.DataSync; + +import com.google.common.base.Preconditions; +import com.google.protobuf.TextFormat; + +import javax.annotation.concurrent.ThreadSafe; + +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import javax.annotation.concurrent.GuardedBy; +import lombok.extern.slf4j.Slf4j; +import lombok.*; + +// for every tid-pid sync task +// TODO backup in local filesystem, and we can recover from it +@ThreadSafe +@Slf4j +public class SyncTask { + @GuardedBy("this") + private DataSync.AddSyncTaskRequest progress; + @GuardedBy("this") + private Long count = 0L; + + @Getter + private AtomicLong lastUpdateTime; + + public enum Status { + INIT, RUNNING, SUCCESS, FAILED, REASSIGNING + } + + @GuardedBy("this") + private Status status; + + Object lock = new Object(); + @GuardedBy("lock") + private String dataCollector; + @Getter + private final String progressPath; + // @Getter private final String dbName; + // @Getter private final String tableName; + + public SyncTask(DataSync.AddSyncTaskRequest request, String dataCollector) { + this.progress = request; + this.dataCollector = dataCollector; + // the path to save progress in local filesystem + // /.progress + this.progressPath = SyncToolConfig.SYNC_TASK_PROGRESS_PATH + "/" + request.getTid() + "/" + + request.getPid() + ".progress"; + // we give the new task more patience, check thread won't check INIT task + this.status = Status.INIT; + this.lastUpdateTime = new AtomicLong(System.currentTimeMillis()); + } + + // when you create a new sync task, you should call this method + public void persist() throws IOException { + Path path = Paths.get(progressPath); + Preconditions.checkState( + Files.notExists(path), "progress file already exists, why create again? " + progressPath); + Files.createDirectories(path.getParent()); + Files.createFile(path); + saveProgressInFile(); + } + + public static SyncTask recover(String filePath) throws IOException { + // TODO how about dataCollector? + DataSync.AddSyncTaskRequest bak = DataSync.AddSyncTaskRequest.parseDelimitedFrom(new FileInputStream(filePath)); + return new SyncTask(bak, ""); + } + + private void saveProgressInFile() throws IOException { + FileOutputStream output = new FileOutputStream(progressPath); + progress.writeDelimitedTo(output); + output.close(); + } + + // TODO(hw): it's better to get a copy of progress + public synchronized DataSync.AddSyncTaskRequest getProgress() { + return progress; + } + + public synchronized void preCheck(DataSync.SendDataRequest request) throws IgnorableException { + Preconditions.checkState(status == Status.INIT || status == Status.RUNNING, + "task is not init or running, ignore this request. st: " + status); + Preconditions.checkArgument( + request.getTid() == progress.getTid() && request.getPid() == progress.getPid(), + "tid and pid not match"); + // token not match means we have a new task, it's normal when sync tool restart + // a sync task + if (!request.getToken().equals(progress.getToken())) { + throw new IgnorableException("token not match, ignore this request"); + } + Preconditions.checkArgument(request.getStartPoint().equals(progress.getSyncPoint()), + "start point " + request.getStartPoint() + " != current point " + progress.getSyncPoint()); + if (status == Status.INIT) { + log.info("task {} is init, received the first data, make it running", + TextFormat.shortDebugString(progress)); + status = Status.RUNNING; + } + } + + public synchronized void updateProgress(DataSync.SendDataRequest request) + throws IgnorableException { + preCheck(request); + // even count == 0, point can be changed, so we use sync point to check if we + // need to update + DataSync.SyncPoint oldPoint = progress.getSyncPoint(); + if (request.getNextPoint() != progress.getSyncPoint()) { + progress = progress.toBuilder().setSyncPoint(request.getNextPoint()).build(); + count += request.getCount(); + try { + saveProgressInFile(); + } catch (IOException e) { + // can't handle this exception + // after preCheck, RUNNING->FAILED + status = Status.FAILED; + throw new RuntimeException( + "progress persist failed, but we already cache the data. mark this task as unalive, check it manually. failed progress " + + progress, + e); + } + } + // if no data, we still need to update lastUpdateTime + lastUpdateTime.set(System.currentTimeMillis()); + log.info("update progress {}: {} -> {}, update time {}", TextFormat.shortDebugString(progress), + TextFormat.shortDebugString(oldPoint), TextFormat.shortDebugString(request.getNextPoint()), + lastUpdateTime.get()); + } + + // message Task is only for show + public synchronized DataSync.Task buildTaskPB() { + return DataSync.Task.newBuilder() + .setProgress(progress) + .setDataCollector(getDataCollector()) + .setCount(count) + .setStatus(status.name()) + .build(); + } + + public synchronized void setToken(String token) { + progress = progress.toBuilder().setToken(token).build(); + } + + public String extraInfo() { + // compact string, you can split it by ";" and get kv by "=" + // and you can know tid and pid in progressPath + synchronized (lock) { + return "lastUpdateTime=" + lastUpdateTime.get() + ";progressPath=" + progressPath; + } + } + + public String getDataCollector() { + synchronized (lock) { + return dataCollector; + } + } + + public void setDataCollector(String dataCollector) { + synchronized (lock) { + this.dataCollector = dataCollector; + } + } + + public synchronized void setTabletServer(String tabletServer) { + progress = progress.toBuilder().setTabletEndpoint(tabletServer).build(); + } + + public synchronized Status getStatus() { + return status; + } + + public synchronized void setStatus(Status status) { + this.status = status; + } + + public synchronized void close() throws IOException { + status = Status.SUCCESS; + // rename the file to finished + java.nio.file.Path newFile = Paths.get(progressPath + ".finished." + System.currentTimeMillis()); + Files.move(Paths.get(progressPath), newFile); + } +} diff --git a/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncTool.java b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncTool.java new file mode 100644 index 00000000000..f2ffa3f9b3c --- /dev/null +++ b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncTool.java @@ -0,0 +1,81 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com._4paradigm.openmldb.synctool; + +import com._4paradigm.openmldb.sdk.SqlException; +import com.baidu.brpc.server.RpcServer; +import com.baidu.brpc.server.RpcServerOptions; +import lombok.extern.slf4j.Slf4j; +import lombok.Getter; + +@Slf4j +public class SyncTool { + private RpcServer rpcServer; + @Getter + private SyncToolImpl syncToolService; + + public void start() throws SqlException, InterruptedException { + start(true); + } + + public void start(boolean blocking) throws SqlException, InterruptedException { + RpcServerOptions options = new RpcServerOptions(); + options.setReceiveBufferSize(64 * 1024 * 1024); + options.setSendBufferSize(64 * 1024 * 1024); + options.setIoThreadNum(SyncToolConfig.IO_THREAD); + options.setWorkThreadNum(SyncToolConfig.WORKER_THREAD); + rpcServer = new RpcServer(SyncToolConfig.HOST, SyncToolConfig.PORT, options); + syncToolService = new SyncToolImpl(String.format("%s:%s", + SyncToolConfig.HOST, SyncToolConfig.PORT)); + // recover before register service + syncToolService.init(); + rpcServer.registerService(syncToolService); + rpcServer.start(); + // with worker thread number? + log.info("Start SyncTool on {}:{}", SyncToolConfig.HOST, SyncToolConfig.PORT); + if (blocking) { + // make server keep running + synchronized (SyncTool.class) { + try { + SyncTool.class.wait(); + } catch (Throwable e) { + log.error("Get exception when waiting, message: " + e.getMessage()); + throw new RuntimeException(e); + } + } + } + + } + + public void stop() { + if (rpcServer != null) { + rpcServer.shutdown(); + } + } + + public static void main(String[] args) { + SyncToolConfig.parse(); + SyncTool syncTool = new SyncTool(); + try { + log.info("Start"); + System.out.println("Start SyncTool"); + syncTool.start(); + } catch (Exception e) { + e.printStackTrace(); + } + } +} diff --git a/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncToolConfig.java b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncToolConfig.java new file mode 100644 index 00000000000..26680f85c17 --- /dev/null +++ b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncToolConfig.java @@ -0,0 +1,115 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com._4paradigm.openmldb.synctool; + +import java.io.IOException; +import java.util.Properties; + +import lombok.extern.slf4j.Slf4j; +import lombok.*; +import com.google.common.base.Preconditions; + +/** + * The global configuration of SyncTool. + * + * Need to call SyncToolConfig.parser() before accessing configuration. + */ +@Slf4j +public class SyncToolConfig { + public static String HOST; + public static int PORT; + public static int WORKER_THREAD; + public static int IO_THREAD; + // public static int CHANNEL_KEEP_ALIVE_TIME; + public static String ZK_CLUSTER; + public static String ZK_ROOT_PATH; + public static String SYNC_TASK_PROGRESS_PATH; + + public static String HADOOP_CONF_DIR; + + public static String DATA_CACHE_PATH; + public static int FLINK_SLOTS; + + public static int TASK_CHECK_PERIOD; // seconds + // sync tool->data collector add sync task timeout + public static int REQUEST_TIMEOUT; // seconds + + private static boolean isParsed = false; + @Getter + private static Properties prop = new Properties(); + + public synchronized static void parse() { + if (!isParsed) { + doParse(); + isParsed = true; + } + } + + private static void doParse() { + try { + prop.load(SyncToolConfig.class.getClassLoader().getResourceAsStream("synctool.properties")); + } catch (IOException e) { + throw new RuntimeException("Fail to load synctool.properties", e); + } + parseFromProperties(prop); + } + + private static void parseFromProperties(Properties prop) { + HOST = prop.getProperty("server.host", ""); + PORT = Integer.parseInt(prop.getProperty("server.port", "")); + if (PORT < 1 || PORT > 65535) { + throw new RuntimeException("server.port invalid port, should be in range of 1 through 65535"); + } + WORKER_THREAD = Integer.parseInt(prop.getProperty("server.worker_threads", "16")); + IO_THREAD = Integer.parseInt(prop.getProperty("server.io_threads", "4")); + + ZK_CLUSTER = prop.getProperty("zookeeper.cluster", ""); + if (ZK_CLUSTER.isEmpty()) { + throw new RuntimeException("zookeeper.cluster should not be empty"); + } + + ZK_ROOT_PATH = prop.getProperty("zookeeper.root_path", ""); + if (ZK_ROOT_PATH.isEmpty()) { + throw new RuntimeException("zookeeper.root_path should not be empty"); + } + + HADOOP_CONF_DIR = prop.getProperty("hadoop.conf.dir", ""); + if (HADOOP_CONF_DIR.isEmpty()) { + log.info("no hadoop.conf.dir, use env"); + HADOOP_CONF_DIR = System.getenv("HADOOP_CONF_DIR"); + Preconditions.checkArgument(!HADOOP_CONF_DIR.isEmpty(), + "hadoop.conf.dir should not be empty, set in system env or synctool.properties"); + } + + // TODO(hw): just for flink tunnel, not used now + DATA_CACHE_PATH = prop.getProperty("data.cache_path", ""); + // Preconditions.checkArgument(!DATA_CACHE_PATH.isEmpty(), "data.cache_path + // should not be empty"); + + SYNC_TASK_PROGRESS_PATH = prop.getProperty("sync_task.progress_path", ""); + Preconditions.checkArgument( + !SYNC_TASK_PROGRESS_PATH.isEmpty(), "sync_task.progress_path should not be empty"); + + FLINK_SLOTS = Integer.parseInt(prop.getProperty("flink.slots", "32")); + + TASK_CHECK_PERIOD = Integer.parseInt(prop.getProperty("sync_task.check_period", "10")); + + REQUEST_TIMEOUT = Integer.parseInt(prop.getProperty("request.timeout", "5")); + + log.info("SyncToolConfig: {}", prop); + } +} diff --git a/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncToolImpl.java b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncToolImpl.java new file mode 100644 index 00000000000..f63ff2ae406 --- /dev/null +++ b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncToolImpl.java @@ -0,0 +1,627 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com._4paradigm.openmldb.synctool; + +import com._4paradigm.openmldb.proto.Common; +import com._4paradigm.openmldb.proto.DataSync; +import com._4paradigm.openmldb.proto.NS; +import com._4paradigm.openmldb.proto.Common.ColumnDesc; +import com._4paradigm.openmldb.sdk.SdkOption; +import com._4paradigm.openmldb.sdk.SqlException; +import com._4paradigm.openmldb.sdk.SqlExecutor; +import com._4paradigm.openmldb.sdk.impl.SqlClusterExecutor; +import com._4paradigm.openmldb.common.zk.ZKClient; +import com._4paradigm.openmldb.common.zk.ZKConfig; +import com.baidu.brpc.RpcContext; +import com.baidu.brpc.client.BrpcProxy; +import com.baidu.brpc.client.RpcClient; +import com.baidu.brpc.client.RpcClientOptions; + +import java.io.BufferedWriter; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.math.NumberUtils; +import com.google.common.base.Preconditions; +import org.apache.commons.io.FileUtils; +import com.google.protobuf.TextFormat; + +import io.netty.buffer.ByteBuf; +import lombok.extern.slf4j.Slf4j; + +import static com.google.common.collect.MoreCollectors.onlyElement; + +@Slf4j +public class SyncToolImpl implements SyncToolInterface { + private final String endpoint; + private volatile ZKClient zkClient; + private SqlExecutor router; + private String zkCollectorPath; + private ScheduledExecutorService executor; + + // holds all sync tasks >, big lock + // guard by itself + private Map> syncTasks = Collections.synchronizedMap(new HashMap<>()); + // >, we need to know db and table name for each tid to get + // table info(to find data collector, to get the latest schema) + private Map> tid2dbtable = Collections.synchronizedMap(new HashMap<>()); + + public SyncToolImpl(String endpoint) throws SqlException, InterruptedException { + this.endpoint = endpoint; + this.zkClient = new ZKClient(ZKConfig.builder() + .cluster(SyncToolConfig.ZK_CLUSTER) + .namespace(SyncToolConfig.ZK_ROOT_PATH) + .build()); + Preconditions.checkState(zkClient.connect(), "zk connect failed"); + SdkOption option = new SdkOption(); + option.setZkCluster(SyncToolConfig.ZK_CLUSTER); + option.setZkPath(SyncToolConfig.ZK_ROOT_PATH); + this.router = new SqlClusterExecutor(option); + this.zkCollectorPath = SyncToolConfig.ZK_ROOT_PATH + "/sync_tool/collector"; + + // a background thread to check sync task status and reassign tasks + executor = Executors.newScheduledThreadPool(1); + } + + public void init() { + // setup tunnel env first + // hdfs tunnel hasn't cache, just write to hdfs + Preconditions.checkNotNull(HDFSTunnel.getInstance()); + // sleep? + // recover sync tasks from file + recover(); + + // start task check bg thread, don't start before recover(may set task to + // unalive too fast), so delay 5s + int initialDelay = 5; + int period = SyncToolConfig.TASK_CHECK_PERIOD; + TimeUnit unit = TimeUnit.SECONDS; + // for check thread + long periodMs = unit.toMillis(period); + SimpleDateFormat formatter = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss"); + Runnable taskCheck = () -> { + log.debug("check sync task status"); + // for each task + List reassignTasks = new ArrayList<>(); + synchronized (syncTasks) { + for (Map.Entry> entry : syncTasks.entrySet()) { + int tid = entry.getKey(); + Map tasksForTid = entry.getValue(); + // check if all in SUCCESS status + if (tasksForTid.values().stream().allMatch( + t -> t.getStatus() == SyncTask.Status.SUCCESS)) { + log.info("tid {} sync task all SUCCESS, remove it", tid); + cleanEnv(tid); + syncTasks.remove(tid); + } + for (Map.Entry entry2 : tasksForTid.entrySet()) { + int pid = entry2.getKey(); + SyncTask task = entry2.getValue(); + // check sync task update time, ignore unalive tasks + synchronized (task) { + if (task.getStatus() == SyncTask.Status.RUNNING + && task.getLastUpdateTime().get() + periodMs * 2 < System.currentTimeMillis()) { + // task timeout, reassign + log.info("task timeout, reassign, {}, last update time {}", task.extraInfo(), + formatter.format(new Date(task.getLastUpdateTime().get()))); + // mark task as reassigning to avoid send data to it + task.setStatus(SyncTask.Status.REASSIGNING); + reassignTasks.add(task); + } + } + // TODO if flink tunnel, check count with flink sink metric? to know if mode 0 + // task is finished + } + } + } + // reassign tasks outside the lock + for (SyncTask task : reassignTasks) { + // choose another data collector if the current one is down or partition leader + // changed, just use the current living one + synchronized (task) { + try { + Pair pair = findDataCollectorInHost(task.getProgress().getTid(), + task.getProgress().getPid()); + task.setTabletServer(pair.getLeft()); + task.setDataCollector(pair.getRight()); + // new token + task.setToken(genToken()); + addTaskInDataCollector(task); + // a longer timeout + task.getLastUpdateTime().set(period * 2 + System.currentTimeMillis()); + // enable SendData + task.setStatus(SyncTask.Status.RUNNING); + } catch (Exception e) { + log.error("reassign task in data collector failed, task: {}, no longer handle it", + task.extraInfo(), e); + task.setStatus(SyncTask.Status.FAILED); + } + } + } + }; + + executor.scheduleAtFixedRate(taskCheck, initialDelay, period, unit); + } + + // not thread safe, call before serving. If recover failed, don't start serving + private void recover() { + Path progressPathRoot = Paths.get(SyncToolConfig.SYNC_TASK_PROGRESS_PATH); + if (Files.notExists(progressPathRoot)) { + log.info("no sync task progress file, skip recover"); + return; + } + // read all sync tasks' progress from file, mark it as alive, bg thread will + // reassgin it if timeout + // tids = new HashMap<>(); + try (Stream stream = Files.walk(progressPathRoot)) { + List bakFiles = stream + .filter(Files::isRegularFile) + // skip bak files and bak tid dir + .filter(file -> file.getFileName().toString().endsWith(".progress") + && NumberUtils.isParsable(file.getParent().getFileName().toString())) + .collect(Collectors.toList()); + log.info("try to recover {} sync task progress", bakFiles.size()); + // repartition it + Map> tid2Files = new HashMap<>(); + for (Path file : bakFiles) { + // /.progress + int tid = Integer.parseInt(file.getParent().getFileName().toString()); + // int pid = Integer.parseInt(file.getFileName().toString().split("\\.")[0]); + List files = tid2Files.get(tid); + if (files == null) { + files = new ArrayList<>(); + tid2Files.put(tid, files); + } + files.add(file); + } + log.info("find {} table sync task, detail {}", tid2Files.size(), tid2Files); + String sinkPath = null; + for (Map.Entry> entry : tid2Files.entrySet()) { + int tid = entry.getKey(); + Preconditions.checkState( + !syncTasks.containsKey(tid), "tid already in syncTasks, sth wrong, tid: " + tid); + syncTasks.put(tid, new HashMap<>()); + // recover all pid task + log.info("recover sync task for tid: {}, pid size {}", tid, entry.getValue().size()); + for (Path file : entry.getValue()) { + log.info("recover sync task progress, file: {}", file); + SyncTask task = SyncTask.recover(file.toAbsolutePath().toString()); + log.info("extracted task: {}", task.buildTaskPB()); + DataSync.AddSyncTaskRequest progress = task.getProgress(); + Preconditions.checkState(tid == progress.getTid(), "tid not match, file: %s", file); + // TODO(hw): check pid consecutive? + int pid = progress.getPid(); + // add task to syncTasks + Map tasksForTid = Preconditions.checkNotNull(syncTasks.get(tid)); + // we don't assign task to data collector, just add it to syncTasks, let bg + // thread to reassgin it, so we should set running here + task.setStatus(SyncTask.Status.RUNNING); + tasksForTid.put(pid, task); + if (sinkPath == null) { + sinkPath = progress.getDest(); + } else { + Preconditions.checkState(sinkPath.equals(progress.getDest()), + "sink path not match, file: %s, sinkPath: %s", file, sinkPath); + } + } + + Preconditions.checkState(!tids.containsKey(tid), + "tid already in tids, sth wrong, tid: " + tid + ", sinkPath: " + sinkPath); + tids.put(tid, Preconditions.checkNotNull(sinkPath)); + // recover tunnel env + try { + Preconditions.checkState(HDFSTunnel.getInstance().recoverTunnel(tid, sinkPath)); + } catch (Exception e) { + log.error("recover tunnel env failed, tid: {}, remove the env", tid, e); + } + } + } catch (IOException e) { + throw new RuntimeException( + "read sync task progress dir failed, dir: " + SyncToolConfig.SYNC_TASK_PROGRESS_PATH, e); + } + + // get db and table name for each tid + try { + Statement stmt = router.getStatement(); + Preconditions.checkState(stmt.execute("show table status")); + ResultSet rs = stmt.getResultSet(); + // can't use name, use idx to get + while (rs.next()) { + // Table_id + int tid = Integer.parseInt(rs.getString(1)); + if (!tids.containsKey(tid)) { + continue; + } + // Table_name 2, Database_name 3 + String db = rs.getString(3); + String table = rs.getString(2); + tid2dbtable.put(tid, Pair.of(db, table)); + } + // if the disappeared table will be recovered later, how to handle it? So leave + // the progress files, don't delete them + Preconditions.checkState(tids.size() == tid2dbtable.size(), + "some tables are disappeared, tids: %s, tid2dbtable: %s", tids, tid2dbtable); + } catch (SQLException e) { + throw new RuntimeException("recover tid2dbtable failed", e); + } + log.info("recover {} sync tasks, syncTasks: {}, tid2dbtable: {}", syncTasks.size(), syncTasks, + tid2dbtable); + } + + @Override + public DataSync.GeneralResponse CreateSyncTask(DataSync.CreateSyncTaskRequest request) { + log.info("Receive CreateSyncTask request: {}", request); + // create sync task env + DataSync.GeneralResponse.Builder respBuilder = DataSync.GeneralResponse.newBuilder().setCode(0); + // get partition info of table getTableInfo + NS.TableInfo tableInfo; + int tid = -1; + try { + tableInfo = router.getTableInfo(request.getDb(), request.getName()); + log.debug("tableInfo: {}", tableInfo); + // check storage mode + Preconditions.checkArgument(tableInfo.getStorageMode() == Common.StorageMode.kSSD + || tableInfo.getStorageMode() == Common.StorageMode.kHDD, + "only SSD and HDD are supported"); + tid = tableInfo.getTid(); + + // use creatingTasks to setup for one table + synchronized (syncTasks) { + if (syncTasks.containsKey(tid)) { + // just return, do not do clean + respBuilder.setCode(-1).setMsg("sync task already exists(creating or created)"); + return respBuilder.build(); + } + syncTasks.put(tid, new HashMap<>()); + log.info( + "creating sync tasks for table {}.{}({})", request.getDb(), request.getName(), tid); + } + + // do not lock the whole part cuz creating tasks on data collector may update + // the task progress + for (NS.TablePartition partitionInfo : tableInfo.getTablePartitionList()) { + log.debug("partitionInfo: {}", partitionInfo); + Pair pair = findDataCollectorInHost(partitionInfo); + String tabletServer = pair.getLeft(); + String dataCollector = pair.getRight(); + // add sync task + int pid = partitionInfo.getPid(); + SyncTask task = createSyncTask(tid, pid, request, tabletServer, dataCollector); + syncTasks.get(tid).put(pid, task); + // add sync task to data collector, anyone failed, reject all + addTaskInDataCollector(task); + } + tid2dbtable.put(tid, Pair.of(request.getDb(), request.getName())); + + Preconditions.checkState(HDFSTunnel.getInstance().createTunnel(tid, "", request.getDest()), + "create hdfs tunnel failed"); + log.info("add sync tasks for table {}.{}({}[{}]), sync tasks: {}", request.getDb(), + request.getName(), tid, syncTasks.get(tid).size(), syncTasks); + } catch (Exception e) { + e.printStackTrace(); + // just cleanup in sync tool, if DataCollector send data for not-exist task, + // reject it + cleanEnv(tid); + respBuilder.setCode(-1).setMsg(e.toString()); + } + return respBuilder.build(); + } + + /* cache methods used by flink */ + public static String genCacheDir(int tid) { + return SyncToolConfig.DATA_CACHE_PATH + "/" + tid; + } + + private void createCacheDir(int tid) throws IOException { + java.nio.file.Path cachePath = Paths.get(genCacheDir(tid)); + Preconditions.checkState( + Files.notExists(cachePath), "cache dir already exists, why? " + cachePath); + Files.createDirectories(cachePath); + Preconditions.checkState(Files.exists(cachePath) && Files.isDirectory(cachePath), + "not exists or not a dir: " + cachePath); + } + + private void deleteCacheDir(int tid) throws IOException { + java.nio.file.Path cachePath = Paths.get(genCacheDir(tid)); + Preconditions.checkState(Files.exists(cachePath), "cache dir not exists, why? " + cachePath); + FileUtils.deleteDirectory(cachePath.toFile()); + Preconditions.checkState(Files.notExists(cachePath), "delete cache dir failed, " + cachePath); + } + + private void cleanEnv(int tid) { + synchronized (syncTasks) { + syncTasks.remove(tid); + Path tidProcess = Paths.get(SyncToolConfig.SYNC_TASK_PROGRESS_PATH + '/' + tid); + if (Files.exists(tidProcess)) { + try { + Files.move(tidProcess, + Paths.get(SyncToolConfig.SYNC_TASK_PROGRESS_PATH + '/' + tid + ".deleted." + + System.currentTimeMillis())); + } catch (IOException e) { + e.printStackTrace(); + log.warn("move sync task progress dir failed, dir: {}, continue.", tidProcess); + } + } + // hdfs no cache + HDFSTunnel.getInstance().closeTunnel(tid); + } + } + + private Pair findDataCollectorInHost(int tid, int pid) throws Exception { + Pair pair = tid2dbtable.get(tid); + Preconditions.checkState(pair != null, "can't find tid in tid2dbtable, tid: " + tid); + NS.TableInfo tableInfo = router.getTableInfo(pair.getLeft(), pair.getRight()); + NS.TablePartition partitionInfo = tableInfo.getTablePartition(pid); + Preconditions.checkState(partitionInfo.getPid() == pid, + "partition id not match, partition list doesn't in order? tid: " + tid + ", pid: " + pid); + log.debug("partitionInfo: {}", partitionInfo); + return findDataCollectorInHost(partitionInfo); + } + + private Pair findDataCollectorInHost(NS.TablePartition partitionInfo) { + NS.PartitionMeta leaderPartition = partitionInfo.getPartitionMetaList() + .stream() + .filter(partitionMeta -> { + return partitionMeta.getIsLeader(); + }) + .collect(onlyElement()); + log.debug("leaderPartition: {}", leaderPartition); + // choose data collector in the same host to add sync task + String tabletServer = leaderPartition.getEndpoint(); + String dataCollector = findDataCollectorInHost(tabletServer); + return Pair.of(tabletServer, dataCollector); + } + + // find data collector in the same host + private String findDataCollectorInHost(String tabletServer) { + String host = tabletServer.split(":")[0]; + Preconditions.checkState(!host.isEmpty(), "host is empty"); // or check host pattern? + // full path in zk, get alive data collectors + List dataCollectors; + try { + Preconditions.checkState(zkClient.checkExists(zkCollectorPath)); + dataCollectors = zkClient.getChildren(zkCollectorPath); + } catch (Exception e) { + throw new RuntimeException("can't find data collectors in zk", e); + } + for (String dataCollector : dataCollectors) { + String dataCollectorHost = dataCollector.split(":")[0]; + if (dataCollectorHost.equals(host)) { + return dataCollector; + } + } + throw new RuntimeException("can't find living data collector in host: " + host); + } + + // gen unique token + private synchronized String genToken() { + return UUID.randomUUID().toString(); + } + + private SyncTask createSyncTask(int tid, int pid, DataSync.CreateSyncTaskRequest request, + String tabletServer, String dataCollector) { + Preconditions.checkState(request.hasMode(), "mode is required"); + DataSync.AddSyncTaskRequest.Builder addSyncTaskRequestBuilder = DataSync.AddSyncTaskRequest.newBuilder() + .setTid(tid).setPid(pid).setMode(request.getMode()); + if (request.hasStartTs()) { + addSyncTaskRequestBuilder.setStartTs(request.getStartTs()); + } + // new sync point start from snapshot + addSyncTaskRequestBuilder.getSyncPointBuilder().setType(DataSync.SyncType.kSNAPSHOT); + addSyncTaskRequestBuilder.setTabletEndpoint(tabletServer); + addSyncTaskRequestBuilder.setDesEndpoint(this.endpoint); // set me + addSyncTaskRequestBuilder.setToken(genToken()); + addSyncTaskRequestBuilder.setDest(request.getDest()); // for persist + SyncTask task = new SyncTask(addSyncTaskRequestBuilder.build(), dataCollector); + try { + task.persist(); + } catch (IOException e) { + throw new RuntimeException("persist new sync task failed", e); + } + return task; + } + + // not thread safe + private void addTaskInDataCollector(SyncTask task) { + RpcClientOptions clientOption = new RpcClientOptions(); + // clientOption.setWriteTimeoutMillis(rpcWriteTimeout); // concurrent rpc may + // let write slowly + + // rpc may take time, cuz data collector needs to create sync env + clientOption.setReadTimeoutMillis((int) TimeUnit.SECONDS.toMillis(SyncToolConfig.REQUEST_TIMEOUT)); + // clientOption.setMinIdleConnections(10); + // clientOption.setCompressType(Options.CompressType.COMPRESS_TYPE_NONE); + clientOption.setGlobalThreadPoolSharing(true); + // disable retry for simplicity, timeout retry is dangerous + clientOption.setMaxTryTimes(1); + // Must list:// + String serviceUrl = "list://" + task.getDataCollector(); + RpcClient rpcClient = new RpcClient(serviceUrl, clientOption); + DataCollectorService dataCollectorService = BrpcProxy.getProxy(rpcClient, DataCollectorService.class); + DataSync.GeneralResponse resp = dataCollectorService.addSyncTask(task.getProgress()); + if (resp.getCode() != 0) { + throw new RuntimeException( + "add sync task failed, code: " + resp.getCode() + ", msg: " + resp.getMsg()); + } + } + + @Override + public DataSync.GeneralResponse DeleteSyncTask(DataSync.DeleteSyncTaskRequest request) { + log.info("Receive DeleteSyncTask request: {}", request); + DataSync.GeneralResponse.Builder responseBuilder = DataSync.GeneralResponse.newBuilder().setCode(0); + // only delete sync task in me, not in data collector + try { + // db.name -> tid + NS.TableInfo tableInfo = Preconditions + .checkNotNull(router.getTableInfo(request.getDb(), request.getName())); + int tid = tableInfo.getTid(); + cleanEnv(tid); + } catch (NullPointerException e) { + responseBuilder.setCode(-1).setMsg(e.getMessage()); + } catch (Exception e) { + responseBuilder.setCode(-1).setMsg("delete sync task failed"); + } + + return responseBuilder.build(); + } + + @Override + public DataSync.SendDataResponse SendData(DataSync.SendDataRequest request) { + log.info("Receive SendData request: {}", TextFormat.shortDebugString(request)); + DataSync.SendDataResponse.Builder respBuilder = DataSync.SendDataResponse.newBuilder(); + respBuilder.getResponseBuilder().setCode(0); + try { + SyncTask task; + synchronized (syncTasks) { + if (!syncTasks.containsKey(request.getTid())) { + // we may receive many invalid send data request, just log it, no need to print + // stack + throw new IgnorableException("can't find sync task, tid: " + request.getTid()); + } + Map tasksForTid = syncTasks.get(request.getTid()); + if (!tasksForTid.containsKey(request.getPid())) { + throw new IgnorableException( + "can't find sync task, tid: " + request.getTid() + ", pid: " + request.getPid()); + } + task = tasksForTid.get(request.getPid()); + } + synchronized (task) { + // precheck, but still may fail cuz syncTasks table has created before tunnel + // created, we can't avoid this(although we delay the first sync on data + // collector, but it's not a guarantee) + task.preCheck(request); + // store data, store to file or else. data collector don't need to read them + // again, so update the sync point + if (request.getCount() != 0) { + ByteBuf data = RpcContext.getContext().getRequestBinaryAttachment(); + Preconditions.checkState(data != null, "attachment data is null"); + log.info("store data, tid: {}, pid: {}, count: {}, size: {}", request.getTid(), + request.getPid(), request.getCount(), data.readableBytes()); + Pair pair = tid2dbtable.get(request.getTid()); + // get the newest schema, to avoid schema change + // not getTableSchema, because it's not good to RowView + NS.TableInfo tableInfo = router.getTableInfo(pair.getLeft(), pair.getRight()); + + // write to hdfs + HDFSTunnel.getInstance().writeData( + tableInfo.getTid(), data, request.getCount(), tableInfo.getColumnDescList()); + // for flink, write to local file, flink monitor the file and read it + // saveToCache(genCacheDir(tableInfo.getTid()), data, request.getCount(), + // tableInfo.getColumnDescList()); + } + // TODO(hw): what if sync tool shutdown here? already store data, but progress + // is not updated + + // If store failed, don't update sync point, and no need to do other things. + // Outdated tasks will be find by taskCheck. + task.updateProgress(request); + } + + if (request.hasFinished() && request.getFinished()) { + // if data collector send finished, it's removed the task in itself already + log.info("sync task {}-{} finished, bak and remove it", request.getTid(), request.getPid()); + task.close(); + } + } catch (Exception e) { + if (!(e instanceof IgnorableException)) { + e.printStackTrace(); + } + respBuilder.getResponseBuilder().setCode(-1).setMsg(e.toString()); + respBuilder.setDeleteTask(true); + log.warn("Response error, let data collector delete the sync task, cuz {}", e.toString()); + } + return respBuilder.build(); + } + + public void saveToCache( + String cacheDir, ByteBuf data, final long count, List schema) { + Preconditions.checkArgument(!schema.isEmpty(), "schema is empty"); + Preconditions.checkArgument(!cacheDir.isEmpty(), ""); + + // unpack data here, or in the row format? + // ref ResultSetBase, write in java, don't make swig interface too complex + try { + // save to temp file, then rename to new file + java.nio.file.Path tempFile = Files.createTempFile("cache-", ".tmp"); + try (BufferedWriter writer = new BufferedWriter(new FileWriter(tempFile.toFile()))) { + DataParser parser = new DataParser(data, count, schema); + parser.writeAll(writer); + } + java.nio.file.Path newFile = Paths.get(cacheDir + "/" + uniqueFileName()); + Files.move(tempFile, newFile); + log.info("save to cache dir: {}, count: {}", newFile, count); + } catch (Exception e) { + throw new RuntimeException("save to cache dir failed", e); + } + } + + // only gen csv + public synchronized static String uniqueFileName() { + return UUID.randomUUID().toString() + ".csv"; + } + + @Override + public DataSync.TaskStatusResponse TaskStatus(DataSync.TaskStatusRequest request) { + log.info("Receive TaskStatus request: {}", request); + DataSync.TaskStatusResponse.Builder respBuilder = DataSync.TaskStatusResponse.newBuilder(); + // default code 0 + respBuilder.getResponseBuilder().setCode(0); + + synchronized (syncTasks) { + if (request.hasClearAll() && request.getClearAll()) { + log.warn("clear all sync tasks and env"); + synchronized (syncTasks) { + syncTasks.keySet().forEach(tid -> cleanEnv(tid)); + syncTasks.clear(); + } + } else { + // get all tasks + for (Map.Entry> entry : syncTasks.entrySet()) { + Map tasksForTid = entry.getValue(); + for (Map.Entry taskEntry : tasksForTid.entrySet()) { + SyncTask task = taskEntry.getValue(); + respBuilder.addTask(task.buildTaskPB()); + respBuilder.addReadableInfo(task.extraInfo()); + } + } + } + } + + return respBuilder.build(); + } +} diff --git a/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncToolInterface.java b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncToolInterface.java new file mode 100644 index 00000000000..5ebd42e69b7 --- /dev/null +++ b/java/openmldb-synctool/src/main/java/com/_4paradigm/openmldb/synctool/SyncToolInterface.java @@ -0,0 +1,35 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com._4paradigm.openmldb.synctool; + +import com._4paradigm.openmldb.proto.DataSync; +import com.baidu.brpc.protocol.BrpcMeta; + +public interface SyncToolInterface { + @BrpcMeta(serviceName = "openmldb.datasync.SyncTool", methodName = "CreateSyncTask") + DataSync.GeneralResponse CreateSyncTask(DataSync.CreateSyncTaskRequest request); + + @BrpcMeta(serviceName = "openmldb.datasync.SyncTool", methodName = "DeleteSyncTask") + DataSync.GeneralResponse DeleteSyncTask(DataSync.DeleteSyncTaskRequest request); + + @BrpcMeta(serviceName = "openmldb.datasync.SyncTool", methodName = "TaskStatus") + DataSync.TaskStatusResponse TaskStatus(DataSync.TaskStatusRequest request); + // TODO(hw): is pause/resume needed? + + @BrpcMeta(serviceName = "openmldb.datasync.SyncTool", methodName = "SendData") + DataSync.SendDataResponse SendData(DataSync.SendDataRequest request); +} diff --git a/java/openmldb-synctool/src/main/resources/core-site.xml b/java/openmldb-synctool/src/main/resources/core-site.xml new file mode 100644 index 00000000000..3943673e9df --- /dev/null +++ b/java/openmldb-synctool/src/main/resources/core-site.xml @@ -0,0 +1,24 @@ + + + + + + + + + fs.defaultFS + hdfs://0.0.0.0:9000 + + diff --git a/java/openmldb-synctool/src/main/resources/log4j.properties b/java/openmldb-synctool/src/main/resources/log4j.properties new file mode 100644 index 00000000000..f53e6ba7762 --- /dev/null +++ b/java/openmldb-synctool/src/main/resources/log4j.properties @@ -0,0 +1,11 @@ +log4j.rootLogger=INFO, console, file +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d %p [%c] - %m%n + +log4j.appender.file=org.apache.log4j.DailyRollingFileAppender +log4j.appender.file.append=true +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d %p [%c] - %m%n +log4j.appender.file.file=./logs/synctool.log +log4j.appender.file.DatePattern='.'yyyy-MM-dd \ No newline at end of file diff --git a/java/openmldb-synctool/src/main/resources/synctool.properties b/java/openmldb-synctool/src/main/resources/synctool.properties new file mode 100644 index 00000000000..313c7de1f73 --- /dev/null +++ b/java/openmldb-synctool/src/main/resources/synctool.properties @@ -0,0 +1,14 @@ +server.host=0.0.0.0 +server.port=8848 + +zookeeper.cluster=0.0.0.0:2181 +zookeeper.root_path=/openmldb + +sync_task.progress_path=/tmp/sync_task_progress + +# hdfs tunnel settings +# hdfs just write to hdfs, no cache, if failed, let data collector send again +# plz set env HADOOP_CONF_DIR to read hdfs-site.xml&core-site.xml in dir +# https://hadoop.apache.org/docs/r3.1.1/api/org/apache/hadoop/conf/Configuration.html +# the same as this file +hadoop.conf.dir=/etc/hadoop/conf diff --git a/java/openmldb-synctool/src/test/resources/log4j.xml b/java/openmldb-synctool/src/test/resources/log4j.xml new file mode 100644 index 00000000000..ef49ce3e325 --- /dev/null +++ b/java/openmldb-synctool/src/test/resources/log4j.xml @@ -0,0 +1,29 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/java/openmldb-synctool/src/test/resources/synctool.properties b/java/openmldb-synctool/src/test/resources/synctool.properties new file mode 100644 index 00000000000..cd20d4b3f83 --- /dev/null +++ b/java/openmldb-synctool/src/test/resources/synctool.properties @@ -0,0 +1,18 @@ +server.host=0.0.0.0 +server.port=8848 + +zookeeper.cluster=0.0.0.0:2181 +zookeeper.root_path=/openmldb + +# flink tunnel settings +rest.bind-address=0.0.0.0 +rest.port=8850-8860 +# storageDir is saving JobManager's metadata +high-availability.storageDir=/tmp/flink/ha +# usage? bind with one sync tool? +high-availability.cluster-id=sync-test + +# for each table, we save data in a separate directory +# flink will watch it(as a source) and sync data to sink +data.cache_path=/tmp/data_cache +sync_task.progress_path=/tmp/sync_task_progress diff --git a/java/pom.xml b/java/pom.xml index 1ce4141530d..47d724724ed 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -15,6 +15,7 @@ openmldb-jdbc openmldb-native openmldb-common + openmldb-synctool openmldb-spark-connector openmldb-batch diff --git a/release/bin/start.sh b/release/bin/start.sh index 3852049a1ec..4f1c1260b29 100755 --- a/release/bin/start.sh +++ b/release/bin/start.sh @@ -16,7 +16,7 @@ set -e -export COMPONENTS="tablet tablet2 nameserver apiserver taskmanager standalone_tablet standalone_nameserver standalone_apiserver" +export COMPONENTS="tablet tablet2 nameserver apiserver taskmanager standalone_tablet standalone_nameserver standalone_apiserver data_collector synctool" if [ $# -lt 2 ]; then echo "Usage: start.sh start/stop/restart [mon]" @@ -65,10 +65,13 @@ fi OPENMLDB_PID_FILE="./bin/$COMPONENT.pid" mkdir -p "$(dirname "$OPENMLDB_PID_FILE")" -if [ "$COMPONENT" != "taskmanager" ]; then - LOG_DIR=$(grep log_dir ./conf/"$COMPONENT".flags | awk -F '=' '{print $2}') -else +if [ "$COMPONENT" == "taskmanager" ]; then LOG_DIR=$(grep job.log.path ./conf/"$COMPONENT".properties | awk -F '=' '{print $2}') +elif [ "$COMPONENT" == "synctool" ] || [ "$COMPONENT" == "data_collector" ]; then + # use bootstrap dir + LOG_DIR="${ROOTDIR}/logs" +else + LOG_DIR=$(grep log_dir ./conf/"$COMPONENT".flags | awk -F '=' '{print $2}') fi [ -n "$LOG_DIR" ] || { echo "Invalid log dir"; exit 1; } @@ -85,9 +88,48 @@ case $OP in fi fi - if [ "$COMPONENT" != "taskmanager" ]; then + # java style + if [ "$COMPONENT" == "taskmanager" ] || [ "$COMPONENT" == "synctool" ]; then + # unique setup + if [ "$COMPONENT" == "taskmanager" ]; then + echo "SPARK_HOME: ${SPARK_HOME}" + fi + if [ -f "./conf/${COMPONENT}.properties" ]; then + echo "Rewrite properties by ./conf/${COMPONENT}.properties" + cp ./conf/"${COMPONENT}".properties ./"${COMPONENT}"/conf/"${COMPONENT}".properties + fi + pushd ./"$COMPONENT"/bin/ > /dev/null + mkdir -p "$LOG_DIR" # no matter the path is abs or relat + if [ "$DAEMON_MODE" = "true" ]; then + "${ROOTDIR}"/"${MON_BINARY}" "./${COMPONENT}.sh" -d -s 10 -l "$LOG_DIR"/"$COMPONENT"_mon.log -m "${ROOTDIR}/${OPENMLDB_PID_FILE}" -p "${ROOTDIR}/${OPENMLDB_PID_FILE}.child" + sleep 3 + MON_PID=$(tr -d '\0' < "${ROOTDIR}/${OPENMLDB_PID_FILE}") + PID=$(tr -d '\0' < "${ROOTDIR}/${OPENMLDB_PID_FILE}.child") + echo "mon pid is $MON_PID, process pid is $PID, check $PID status" + else + sh ./"${COMPONENT}".sh > "$LOG_DIR"/"$COMPONENT".log 2>&1 & + PID=$! + echo "process pid is $PID" + fi + + popd > /dev/null + sleep 10 + if kill -0 "$PID" > /dev/null 2>&1; then + if [ "$DAEMON_MODE" != "true" ]; then + echo "$PID" > "$OPENMLDB_PID_FILE" + fi + echo "Start ${COMPONENT} success" + exit 0 + fi + echo -e "${RED}Start ${COMPONENT} failed! Please check log in ${LOG_DIR}/${COMPONENT}[_mon].log and ./${COMPONENT}/bin/logs/${COMPONENT}.log ${RES}" + else + # cxx style + BIN_NAME="openmldb" + if [ "$COMPONENT" = "data_collector" ]; then + BIN_NAME="data_collector" + fi if [ "$DAEMON_MODE" = "true" ]; then # nohup? test it - START_CMD="./bin/openmldb --flagfile=./conf/${COMPONENT}.flags --enable_status_service=true" + START_CMD="./bin/${BIN_NAME} --flagfile=./conf/${COMPONENT}.flags --enable_status_service=true" # save the mon process pid, but check the component pid ${MON_BINARY} "${START_CMD}" -d -s 10 -l "$LOG_DIR"/"$COMPONENT"_mon.log -m "${OPENMLDB_PID_FILE}" -p "${OPENMLDB_PID_FILE}.child" # sleep for pid files @@ -97,7 +139,7 @@ case $OP in echo "mon pid is $MON_PID, process pid is $PID, check $PID status" else # DO NOT put the whole command in variable - ./bin/openmldb --flagfile=./conf/"${COMPONENT}".flags --enable_status_service=true >> "$LOG_DIR"/"$COMPONENT".log 2>&1 & + ./bin/${BIN_NAME} --flagfile=./conf/"${COMPONENT}".flags --enable_status_service=true >> "$LOG_DIR"/"$COMPONENT".log 2>&1 & PID=$! echo "process pid is $PID" fi @@ -133,35 +175,6 @@ case $OP in fi fi echo -e "${RED}Start ${COMPONENT} failed! Please check log in ${LOG_DIR}/${COMPONENT}[_mon].log and ${LOG_DIR}/${COMPONENT}.INFO ${RES}" - else - echo "SPARK_HOME: ${SPARK_HOME}" - if [ -f "./conf/taskmanager.properties" ]; then - cp ./conf/taskmanager.properties ./taskmanager/conf/taskmanager.properties - fi - pushd ./taskmanager/bin/ > /dev/null - mkdir -p logs - if [ "$DAEMON_MODE" = "true" ]; then - "${ROOTDIR}"/"${MON_BINARY}" "./taskmanager.sh" -d -s 10 -l "$LOG_DIR"/"$COMPONENT"_mon.log -m "${ROOTDIR}/${OPENMLDB_PID_FILE}" -p "${ROOTDIR}/${OPENMLDB_PID_FILE}.child" - sleep 3 - MON_PID=$(tr -d '\0' < "${ROOTDIR}/${OPENMLDB_PID_FILE}") - PID=$(tr -d '\0' < "${ROOTDIR}/${OPENMLDB_PID_FILE}.child") - echo "mon pid is $MON_PID, process pid is $PID, check $PID status" - else - sh ./taskmanager.sh > "$LOG_DIR"/"$COMPONENT".log 2>&1 & - PID=$! - echo "process pid is $PID" - fi - - popd > /dev/null - sleep 10 - if kill -0 $PID > /dev/null 2>&1; then - if [ "$DAEMON_MODE" != "true" ]; then - echo $PID > "$OPENMLDB_PID_FILE" - fi - echo "Start ${COMPONENT} success" - exit 0 - fi - echo -e "${RED}Start ${COMPONENT} failed! Please check log in ${LOG_DIR}/${COMPONENT}[_mon].log and ./taskmanager/bin/logs/taskmanager.log ${RES}" fi ;; stop) diff --git a/release/conf/data_collector.flags b/release/conf/data_collector.flags new file mode 100644 index 00000000000..056c1e0cda9 --- /dev/null +++ b/release/conf/data_collector.flags @@ -0,0 +1,19 @@ +# no need to set role and use original glog flags +--endpoint=127.0.0.1:8888 +--enable_status_service=true + +--zk_cluster=127.0.0.1:2181 +--zk_root_path=/openmldb +--collector_datadir=/tmp/data_collector + +# log conf +--openmldb_log_dir=./logs +--log_level=info + +# if get reached timeout in data collector, set it bigger, default is 20000(20s) +# --request_timeout_ms= + +# if it's big or tasks are too many, it will be overcrowded, set it smaller, default is 1048576(byte,1M) +# --max_pack_size= +# if overcrowded, set it bigger, default is 67108864(byte,64M) +# --socket_max_unwritten_bytes= diff --git a/release/conf/synctool.properties b/release/conf/synctool.properties new file mode 100644 index 00000000000..77e2e6ab2cc --- /dev/null +++ b/release/conf/synctool.properties @@ -0,0 +1,13 @@ +server.host=0.0.0.0 +server.port=8848 + +zookeeper.cluster=0.0.0.0:2181 +zookeeper.root_path=/openmldb + +sync_task.progress_path=/tmp/sync_task_progress + +# hdfs tunnel settings +# hdfs just write to hdfs, no cache, if failed, let data collector send again +# plz set env HADOOP_CONF_DIR or this option to read hdfs-site.xml&core-site.xml in dir +# https://hadoop.apache.org/docs/r3.1.1/api/org/apache/hadoop/conf/Configuration.html +hadoop.conf.dir=/etc/hadoop/conf diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d94c4cf20ab..ea6a4a32393 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -31,6 +31,7 @@ function(compile_proto proto_name project_dir) --java_out=${project_dir}/java/openmldb-import/src/main/java --java_out=${project_dir}/java/openmldb-taskmanager/src/main/java --java_out=${project_dir}/java/openmldb-common/src/main/java + --java_out=${project_dir}/java/openmldb-synctool/src/main/java ${project_dir}/src/proto/${proto_name}.proto DEPENDS ${project_dir}/src/proto/${proto_name}.proto) list(APPEND PROTO_FILES ${CMAKE_BINARY_DIR}/src/proto/${proto_name}.pb.cc) @@ -50,7 +51,7 @@ function(compile_lib LIB_NAME DIR DEPEND_FILE_LIST) add_library(${LIB_NAME} STATIC ${FILE_LIST} $) endfunction(compile_lib) -function(compile_test DIR) +function(compile_test_with_extra DIR DEPEND_FILE_LIST) set(TEST_LIBS openmldb_test_base apiserver nameserver tablet query_response_time openmldb_sdk openmldb_catalog @@ -68,7 +69,10 @@ function(compile_test DIR) file(RELATIVE_PATH RELATIVE_TEST_PATH ${CMAKE_CURRENT_SOURCE_DIR} ${SRC_FILE}) get_filename_component(TEST_TARGET_DIR ${RELATIVE_TEST_PATH} DIRECTORY) get_filename_component(TEST_TARGET_NAME ${RELATIVE_TEST_PATH} NAME_WE) - add_executable(${TEST_TARGET_NAME} ${SRC_FILE} $) + if(DEPEND_FILE_LIST) + message(STATUS "add extra depend file ${DEPEND_FILE_LIST}") + endif() + add_executable(${TEST_TARGET_NAME} ${SRC_FILE} $ ${DEPEND_FILE_LIST}) if (${TEST_TARGET_NAME} STREQUAL "single_tablet_test") add_dependencies(${TEST_TARGET_NAME} test_udf) endif() @@ -87,6 +91,11 @@ function(compile_test DIR) endif() endforeach() set(test_list ${test_list} PARENT_SCOPE) +endfunction(compile_test_with_extra) +function(compile_test DIR) + # modify the var test_list + compile_test_with_extra(${DIR} "") + set(test_list ${test_list} PARENT_SCOPE) endfunction(compile_test) compile_proto(type ${PROJECT_SOURCE_DIR}) @@ -96,6 +105,7 @@ compile_proto(name_server ${PROJECT_SOURCE_DIR}) compile_proto(sql_procedure ${PROJECT_SOURCE_DIR}) compile_proto(api_server ${PROJECT_SOURCE_DIR}) compile_proto(taskmanager ${PROJECT_SOURCE_DIR}) +compile_proto(data_sync ${PROJECT_SOURCE_DIR}) add_library(openmldb_proto STATIC ${PROTO_FILES}) add_library(openmldb_flags STATIC flags.cc) @@ -139,6 +149,8 @@ if(TESTING_ENABLE) compile_test(schema) compile_test(log) compile_test(apiserver) + # abs path + compile_test_with_extra(datacollector ${CMAKE_CURRENT_SOURCE_DIR}/datacollector/data_collector.cc) add_library(test_udf SHARED examples/test_udf.cc) endif() @@ -158,10 +170,14 @@ if(CMAKE_CXX_COMPILER_ID MATCHES "GNU" AND CMAKE_CXX_COMPILER_VERSION VERSION_LE endif() target_link_libraries(data_exporter ${EXPORTER_LIBS}) add_executable(openmldb cmd/openmldb.cc base/linenoise.cc) -target_link_libraries(openmldb ${BIN_LIBS}) +# tablet needs std::filesystem(stdc++fs) +target_link_libraries(openmldb ${EXPORTER_LIBS}) + +add_executable(data_collector datacollector/data_collector_server.cc datacollector/data_collector.cc) +target_link_libraries(data_collector ${EXPORTER_LIBS}) install( - TARGETS openmldb + TARGETS openmldb data_collector RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} ) diff --git a/src/base/file_util.h b/src/base/file_util.h index d3a21e2a827..3c80529555f 100644 --- a/src/base/file_util.h +++ b/src/base/file_util.h @@ -20,9 +20,9 @@ #include #include #include -#include #include #include +#include #include #include #include @@ -324,6 +324,7 @@ __attribute__((unused)) static bool CopyFile(const std::string& src_file, const inline static int HardLinkDir(const std::string& src, const std::string& dest) { if (!IsExists(src)) { + LOG(WARNING) << "src not exists " << src; return -2; } @@ -331,12 +332,20 @@ inline static int HardLinkDir(const std::string& src, const std::string& dest) { RemoveDirRecursive(dest); } - MkdirRecur(dest); + if (!MkdirRecur(dest)) { + LOG(WARNING) << "mkdir failed " << dest; + return -3; + } std::vector files; - GetSubFiles(src, files); + if (GetSubFiles(src, files) != 0) { + LOG(WARNING) << "get sub files failed " << src; + return -4; + } + for (const auto& file : files) { int ret = link((src + "/" + file).c_str(), (dest + "/" + file).c_str()); if (ret) { + LOG(WARNING) << "link failed " << src << "/" << file << " " << dest << "/" << file; return ret; } } diff --git a/src/base/response_util.h b/src/base/response_util.h new file mode 100644 index 00000000000..ea7e126211b --- /dev/null +++ b/src/base/response_util.h @@ -0,0 +1,29 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#ifndef SRC_BASE_RESPONSE_UTIL_H_ +#define SRC_BASE_RESPONSE_UTIL_H_ + +/// @brief +#define SET_RESP_AND_WARN(s, code, msg_str) \ + do { \ + auto* _s = (s); \ + _s->set_code((code)); \ + _s->set_msg((msg_str)); \ + LOG(WARNING) << "Set response: " << _s->msg(); \ + } while (0) + +#endif // SRC_BASE_RESPONSE_UTIL_H_ diff --git a/src/datacollector/data_collector.cc b/src/datacollector/data_collector.cc new file mode 100644 index 00000000000..8cf02a3ab2b --- /dev/null +++ b/src/datacollector/data_collector.cc @@ -0,0 +1,908 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "datacollector/data_collector.h" + +#include +#include +#include +#include +#include + +#include "absl/strings/str_join.h" +#include "boost/algorithm/string/predicate.hpp" +#include "gflags/gflags.h" +#include "google/protobuf/util/message_differencer.h" + +#include "base/response_util.h" +#include "base/status.h" +#include "replica/replicate_node.h" + +DECLARE_string(zk_cluster); +DECLARE_string(zk_root_path); +DECLARE_int32(thread_pool_size); +DECLARE_int32(zk_session_timeout); +DECLARE_int32(zk_keep_alive_check_interval); +DEFINE_int32(task_thread_pool_size, 32, "thread pool size for exec sync task"); +DEFINE_string(collector_datadir, "/tmp/openmldb/datacollector", + "data collector dir for meta data and snapshot hard link"); +DEFINE_uint64(max_pack_size, 1 * 1024 * 1024, + "need < 64M(brpc max_body_size), max size of one pack, != 64M cuz of meta overhead"); +DEFINE_uint64(sync_task_long_interval_ms, 5000, + "interval of one sync task, if binlog meet end, use this one, should be smaller than sync tool " + "sync_task.check_period. Note that it should be much less than binlog_delete_interval, The frequency we " + "fetch binlog is the same as the frequency of sync task execution"); +DEFINE_uint64(sync_task_short_interval_ms, 1000, + "interval of one sync task, if has next data to send, use this one, should be smaller than sync tool " + "sync_task.check_period"); + +namespace fs = std::filesystem; +namespace openmldb::datacollector { +// not thread safe +std::string LogPartsToString(replica::LogParts* log_parts) { + std::stringstream ss; + ss << "["; + auto it = log_parts->NewIterator(); + it->SeekToFirst(); + while (it->Valid()) { + ss << "(" << it->GetKey() << ", " << it->GetValue() << "),"; + it->Next(); + } + ss << "] (size: " << log_parts->GetSize() << ")"; + return ss.str(); +} + +std::shared_ptr genLogReplicatorFromParent(const std::string& binlog_parent) { + LOG(INFO) << "use binlog parent path: " << binlog_parent; + std::map real_ep_map; + // set follower to avoid redundant init of leader, tid-pid is not used, use GetLogPath() to check + auto replicator = std::make_shared(0, 0, binlog_parent, real_ep_map, + replica::ReplicatorRole::kFollowerNode); + bool ok = replicator->Init(); + if (!ok) { + LOG(ERROR) << "init log replicator failed"; + return {}; + } + // debug info + LOG(INFO) << "log parts: " << LogPartsToString(replicator->GetLogPart()); + return replicator; +} + +std::shared_ptr genLogReplicatorFromBinlog(const std::string& binlog_path) { + // binlog_path is '/binlog/', replicator needs '' + auto no_end_slash = binlog_path.back() == '/' ? binlog_path.substr(0, binlog_path.size() - 1) : binlog_path; + auto binlog_parent = fs::path(no_end_slash).parent_path(); + return genLogReplicatorFromParent(binlog_parent); +} + +// LogPart is static, so we generate the LogReader in any time, it can read the latest binlog +std::shared_ptr getLogReader(std::shared_ptr replicator, uint64_t start_offset, + bool* no_binlog_file) { + // not own LogParts, and never meet compressed binlog + auto reader = std::make_shared(replicator->GetLogPart(), replicator->GetLogPath(), false); + if (reader->GetMinOffset() == UINT64_MAX) { + LOG(INFO) << "no binlog file currently"; + *no_binlog_file = true; + return {}; + } + if (!reader->SetOffset(start_offset)) { + LOG(ERROR) << "set offset failed, check the binlog files"; + return {}; + } + LOG(INFO) << "set offset to " << start_offset << " success"; + return reader; +} + +// Manifest location: /MANIFEST +// You can use `has_name()` to check if the manifest is valid(otherwise, it does not exist or is invalid) +api::Manifest GetManifest(const std::string& snapshot_path) { + api::Manifest manifest; + std::string manifest_file = snapshot_path + "MANIFEST"; + if (int ret = (storage::Snapshot::GetLocalManifest(manifest_file, manifest)); ret != 0) { + LOG(WARNING) << "get manifest failed, ret " << ret; + return {}; + } + return manifest; +} + +bool HardLinkSnapshot(const std::string& snapshot_path, const std::string& dest, uint64_t* snapshot_offset) { + // HardLinkDir is not recursive, hard link the manifest file for debug + auto ret = base::HardLinkDir(snapshot_path, dest); + if (ret) { + LOG(WARNING) << " hard link manifest failed, ret " << ret << ", err: " << errno << " " << strerror(errno); + return false; + } + // to hard link the snapshot dir, read dir name from MANIFEST + auto manifest = GetManifest(snapshot_path); + if (!manifest.has_name()) { + LOG(WARNING) << "no manifest name failed"; + return false; + } + // /data + ret = base::HardLinkDir(snapshot_path + manifest.name(), dest + "data"); + if (ret) { + LOG(WARNING) << " hard link snapshot data failed, ret " << ret << ", err: " << errno << " " << strerror(errno); + return false; + } + if (snapshot_offset) { + *snapshot_offset = manifest.offset(); + } + return true; +} + +bool SaveTaskInfoInDisk(const datasync::AddSyncTaskRequest* info, const std::string& path) { + std::ofstream output(path + "task.progress", std::ios::out | std::ios::binary); + // Write the message to the file + auto ret = info->SerializeToOstream(&output); + // Close the file + output.close(); + return ret; +} + +bool LoadTaskInfoFromDisk(const std::string& path, datasync::AddSyncTaskRequest* info) { + auto file = path + "task.progress"; + std::ifstream input(file, std::ios::in | std::ios::binary); + if (!input.is_open()) { + LOG(WARNING) << "open task progress file [" << file << "] failed, " << std::strerror(errno); + return false; + } + // Read the message from the file + auto ret = info->ParseFromIstream(&input); + // Close the file + input.close(); + return ret; +} + +DataCollectorImpl::DataCollectorImpl() + : zk_client_(nullptr), keep_alive_pool_(1), task_pool_(FLAGS_task_thread_pool_size) {} + +DataCollectorImpl::~DataCollectorImpl() { + keep_alive_pool_.Stop(true); + task_pool_.Stop(true); +} + +bool DataCollectorImpl::Init(const std::string& endpoint) { + return Init(FLAGS_zk_cluster, FLAGS_zk_root_path, endpoint); +} +bool DataCollectorImpl::Init(const std::string& zk_cluster, const std::string& zk_path, const std::string& endpoint) { + zk_client_ = std::make_shared(zk_cluster, FLAGS_zk_session_timeout, endpoint, zk_path, + zk_path + kDataCollectorRegisterPath); + if (!zk_client_->Init()) { + LOG(WARNING) << "fail to init zk client"; + return false; + } + LOG(INFO) << "init zk client success"; + // recover tasks from FLAGS_collector_datadir + Recover(); + return true; +} + +bool DataCollectorImpl::RegisterZK() { + if (zk_client_ == nullptr) { + LOG(WARNING) << "zk client is null, please init first"; + return false; + } + // no use_name logic + // no need to startup_ + if (!zk_client_->Register(false)) { + LOG(WARNING) << "fail to register zk"; + return false; + } + + keep_alive_pool_.DelayTask(FLAGS_zk_keep_alive_check_interval, std::bind(&DataCollectorImpl::CheckZkClient, this)); + return true; +} + +void DataCollectorImpl::CheckZkClient() { + if (zk_client_ == nullptr) { + LOG(WARNING) << "zk client is null, weird"; + return; + } + if (!zk_client_->IsConnected()) { + LOG(WARNING) << "reconnect zk"; + if (zk_client_->Reconnect() && zk_client_->Register()) { + LOG(WARNING) << "reconnect zk ok"; + } + } else if (!zk_client_->IsRegisted()) { + LOG(WARNING) << "registe zk"; + if (zk_client_->Register()) { + LOG(WARNING) << "registe zk ok"; + } + } + keep_alive_pool_.DelayTask(FLAGS_zk_keep_alive_check_interval, std::bind(&DataCollectorImpl::CheckZkClient, this)); +} + +std::string DataCollectorImpl::GetWorkDir(const std::string& name) { + return FLAGS_collector_datadir + "/" + name + "/"; +} + +void DataCollectorImpl::CreateTaskEnv(const datasync::AddSyncTaskRequest* request, + datasync::GeneralResponse* response) { + auto tid = request->tid(); + auto pid = request->pid(); + auto name = EncodeId(tid, pid); + // create env for task + { + std::lock_guard lock(process_map_mutex_); + // two situation: + // 1. task is not existing, add it + // 2. task exists, maybe the token is new, update it + if (!AddTaskInfoUnlocked(request)) { + SET_RESP_AND_WARN(response, -1, "why send a same task?"); + return; + } + auto tablet_endpoint = request->tablet_endpoint(); + if (tablet_client_map_.find(tablet_endpoint) == tablet_client_map_.end()) { + tablet_client_map_[tablet_endpoint] = + std::make_shared(request->tablet_endpoint(), ""); + if (tablet_client_map_[tablet_endpoint]->Init() != 0) { + SET_RESP_AND_WARN(response, -1, "init tablet client failed"); + return; + } + } + auto tablet_client = tablet_client_map_[tablet_endpoint]; + api::TableStatus table_status; + if (!tablet_client->GetTableStatus(tid, pid, table_status)) { + SET_RESP_AND_WARN(response, -1, "get table status from tablet server failed, maybe table doesn't exist"); + return; + } + if (!ValidateTableStatus(table_status)) { + SET_RESP_AND_WARN(response, -1, "table status is not valid: " + table_status.ShortDebugString()); + return; + } + + // datasync::AddSyncTaskRequest adjusted_request = *request; + if (request->sync_point().type() == datasync::SyncType::kSNAPSHOT) { + // if no snapshot, we should let sync tool know the change + if (auto manifest = GetManifest(table_status.snapshot_path()); + !manifest.has_name() || manifest.count() == 0) { + LOG(INFO) << "no snapshot or snapshot is empty count in db, switch to binlog in the first sync once"; + } else { + // has snapshot, create snapshot env + api::TableMeta table_meta; + tablet_client->GetTableSchema(tid, pid, table_meta); + // create or update mock table for read snapshot + if (!CreateSnapshotEnvUnlocked(name, table_meta, table_status.snapshot_path())) { + SET_RESP_AND_WARN(response, -1, "create snapshot env failed"); + return; + } + } + } + + // create binlog replicator, in any case, we need to read binlog(no matter binlog files exist or not now) + // we should hardlink again when add new binlog file in db + // NOTE: don't need to rollback snapshot env? maybe use while break + + // link to work dir. We'll create a replicator, no need to update it + if (!FetchBinlogUnlocked(name, table_status.binlog_path(), nullptr)) { + SET_RESP_AND_WARN(response, -1, "hardlink binlog failed"); + return; + } + // binlog path in db may changed later, we cache this in data collector(sync tasks request it frequently) + { + std::lock_guard lock(cache_mutex_); + binlog_path_map_[name] = table_status.binlog_path(); + } + // read binlog from hardlink dir + auto replicator = genLogReplicatorFromParent(GetWorkDir(name)); + if (replicator == nullptr) { + SET_RESP_AND_WARN(response, -1, "create binlog env failed"); + return; + } + replicator_map_[name] = replicator; + // save task info in disk + SaveTaskInfoInDisk(request, GetWorkDir(name)); + } + + // add task to task pool, delay a long time to avoid sync tool not ready(adding tasks are parallel), and it's ok to + // do a failed SendData when sync tool not ready, data collector will retry later + // TODO(hw): add a random delay to avoid all tasks send data at the same time? rpc can't hold so much data. + task_pool_.DelayTask(FLAGS_sync_task_long_interval_ms, std::bind(&DataCollectorImpl::SyncOnce, this, tid, pid)); +} + +void DataCollectorImpl::AddSyncTask(::google::protobuf::RpcController* controller, + const ::openmldb::datasync::AddSyncTaskRequest* request, + ::openmldb::datasync::GeneralResponse* response, + ::google::protobuf::Closure* done) { + brpc::ClosureGuard done_guard(done); + LOG(INFO) << "get AddSyncTask request: " << request->ShortDebugString(); + + CreateTaskEnv(request, response); +} + +bool DataCollectorImpl::ValidateTableStatus(const api::TableStatus& table_status) { + if (table_status.storage_mode() == common::StorageMode::kMemory) { + LOG(WARNING) << "storage mode is memory, not disk"; + return false; + } + // it may be empty in tablet server of old version + if (table_status.snapshot_path().empty()) { + LOG(WARNING) << "snapshot path is empty"; + return false; + } + if (table_status.binlog_path().empty()) { + LOG(WARNING) << "binlog path is empty"; + return false; + } + if (!fs::path(table_status.snapshot_path()).is_absolute()) { + LOG(WARNING) << "snapshot path is not absolute"; + return false; + } + if (!fs::path(table_status.binlog_path()).is_absolute()) { + LOG(WARNING) << "binlog path is not absolute"; + return false; + } + return true; +} + +bool DataCollectorImpl::CreateSnapshotEnvUnlocked(const std::string& name, const api::TableMeta& meta, + const std::string& snapshot_path) { + auto snapshot_hardlink_path = GetWorkDir(name) + "snapshot/"; + auto it = snapshot_map_.find(name); + if (it != snapshot_map_.end()) { + LOG(INFO) << "snapshot env already exists, name: " << name; + // diff the snapshot in db and hardlink + auto manifest_in_db = GetManifest(snapshot_path); + auto manifest_in_hardlink = GetManifest(snapshot_hardlink_path); + if (!manifest_in_db.has_name() || !manifest_in_hardlink.has_name()) { + // still create snapshot env + LOG(WARNING) << "manifest in db or hardlink is empty, but we've created snapshot env before. Still " + "recreate env. db: " + << manifest_in_db.ShortDebugString() << ", hl: " << manifest_in_hardlink.ShortDebugString(); + } else if (google::protobuf::util::MessageDifferencer::Equals(manifest_in_db, manifest_in_hardlink)) { + LOG(INFO) << "manifest is same, no need to update in snapshot env"; + return true; + } + + LOG(INFO) << "remove old snapshot env"; + // old dir can be removed by HardLinkSnapshot + snapshot_map_.erase(it); + } + // create new snapshot env + + // AddSyncTask is called by SyncTool, we don't need to check the old snapshot + // Just hard link the newest snapshot + uint64_t offset = 0; + if (!HardLinkSnapshot(snapshot_path, snapshot_hardlink_path, &offset)) { + LOG(WARNING) << "hard link snapshot failed, probably the collector_datadir is bad"; + return false; + } + snapshot_map_[name] = std::make_shared(meta, snapshot_hardlink_path); + snapshot_map_[name]->Init(); + // set snapshot offset, so we can know the next offset when switch to binlog + snapshot_map_[name]->SetOffset(offset); + return true; +} + +// not thread safe, don't call it in multi thread +bool DataCollectorImpl::FetchBinlogUnlocked(const std::string& name, const std::string& binlog_path, bool* updated) { + // binlog_path is <>/binlog/, just files in it + fs::path binlog_dir(binlog_path); + // even empty, binlog dir should exist, if not, sth is wrong + if (!fs::exists(binlog_dir)) { + LOG(ERROR) << "binlog dir not exist, path: " << binlog_path; + return false; + } + + auto binlog_hardlink_path = GetWorkDir(name) + "binlog/"; + if (!fs::exists(binlog_hardlink_path)) { + std::error_code ec; + fs::create_directories(binlog_hardlink_path, ec); + if (ec) { + LOG(ERROR) << "create binlog hardlink dir failed, path: " << binlog_hardlink_path + << ", err: " << ec.message(); + return false; + } + } + + for (auto& p : fs::directory_iterator(binlog_dir)) { + if (fs::is_regular_file(p)) { + auto file_name = p.path().filename().string(); + if (!boost::ends_with(file_name, ".log")) { + continue; + } + auto src = binlog_path + file_name; + auto dst = binlog_hardlink_path + file_name; + if (fs::exists(dst)) { + // hardlinked before, skip. TODO(hw): check inode? + continue; + } + std::error_code ec; + fs::create_hard_link(src, dst, ec); + // link failed, may miss some binlog, stop the task + if (ec) { + LOG(WARNING) << "create hard link failed, src: " << src << ", dst: " << dst + << ", err: " << ec.message(); + return false; + } + if (updated) { + *updated = true; + } + } + } + return true; +} + +// for one tid-pid, SyncOnce is serial(supported by add next task logic), so we don't need to lock for one tid-pid +void DataCollectorImpl::SyncOnce(uint32_t tid, uint32_t pid) { + // read the progress + auto task = GetTaskInfo(tid, pid); + if (!task.has_tid()) { + LOG(WARNING) << "task not exist(deleted), tid: " << tid << ", pid: " << pid << ", reject task"; + return; + } + LOG(INFO) << "sync once task: " << task.ShortDebugString(); + auto name = EncodeId(tid, pid); + auto start_point = task.sync_point(); + auto mode = task.mode(); + + // TODO(hw): fetch binlog in a work thread + // we fetch binlog even when we read snapshot, cuz we don't want lost any binlog file, should fetch in time + std::string binlog_path_in_db; + { + std::lock_guard lock(cache_mutex_); + binlog_path_in_db = binlog_path_map_[name]; + } + bool updated = false; + if (FetchBinlogUnlocked(name, binlog_path_in_db, &updated) == false) { + LOG(WARNING) << "fail to fetch binlog, may lost data, reject task: " << task.ShortDebugString(); + return; + } + // to avoid too much used binlog + auto replicator = GetReplicator(name); + if (replicator == nullptr) { + LOG(WARNING) << "fail to get replicator, reject task: " << task.ShortDebugString(); + return; + } + // only update log part when fetch new, and no need to delete old log files when there is no new log file + if (updated) { + replicator->Recover(); + LOG(INFO) << "log parts: " << LogPartsToString(replicator->GetLogPart()); + if (start_point.type() == datasync::SyncType::kBINLOG && + replicator->GetSnapshotLastOffset() < start_point.offset()) { + replicator->SetSnapshotLogPartIndex(start_point.offset()); + // delete old log file if SetSnapshotLogPartIndex + // delete will update log parts, it's ok + replicator->DeleteBinlog(); + } + } + + // fullfil the request: io_buf, count, next_point, finished(for mode 0) + butil::IOBuf io_buf; + uint64_t count = 0; + decltype(start_point) next_point; + bool meet_binlog_end = false; + // we can do switch in it + if (!PackData(task, &io_buf, &count, &next_point, &meet_binlog_end)) { + LOG(WARNING) << "fail to pack data, reject task: " << task.ShortDebugString(); + return; + } + + VLOG(1) << "pack cnt:" << count << "-size:" << io_buf.size() << ", start " << start_point.ShortDebugString() + << ", next: " << next_point.ShortDebugString(); + + bool delay_longer = meet_binlog_end; + bool is_finished = false; + // even count is 0(all data in snapshot, binlog is empty) + if (meet_binlog_end && mode == datasync::SyncMode::kFull) { + is_finished = true; + LOG(INFO) << "Mode FULL and meet binlog end, set finished flag"; + } + bool need_update = true; + // validations + if (count == 0) { + if (io_buf.size() != 0) { + LOG(ERROR) << "no data but io_buf not empty, reject task" << task.ShortDebugString(); + return; + } + // it's ok to have no data but next_point is not equal to start_point, cuz we may have ts filter in mode 1 + // or many delete entries in binlog + // check next_point >= start_point? + + // validate about status, if failed, reject task + if (next_point.type() == datasync::SyncType::kSNAPSHOT) { + LOG(ERROR) << "why snapshot can't get any data and can't switch to binlog? reject task: " + << task.ShortDebugString(); + return; + } + + // validate meet_binlog_end if start_point is binlog(don't check by next_point, cuz it won't update + // meet_binlog_end when snapshot->binlog, just check start_point) + if (start_point.type() == datasync::SyncType::kBINLOG && !meet_binlog_end) { + LOG(ERROR) << "why binlog can't get any data and can't meet binlog end? reject task: " + << task.ShortDebugString(); + return; + } + + if (google::protobuf::util::MessageDifferencer::Equals(start_point, next_point)) { + LOG(INFO) << "sync point not changed, skip update in file: " << task.ShortDebugString(); + need_update = false; + } + } + // send sync rpc(not async, we should retry in the next turn if failed) + decltype(task) updated_task; + // DO NOT use io_buf after this line, because io_buf may be moved + datasync::SendDataResponse response; + LOG(INFO) << "send data, task: " << task.ShortDebugString() << ", count: " << count + << ", is_finished: " << is_finished << ", next_point: " << next_point.ShortDebugString() + << ", io_buf size: " << io_buf.size(); + auto ok = SendDataUnlock(&task, io_buf, count, next_point, is_finished, &response, &updated_task); + // if failed, retry later + if (!ok) { + LOG(WARNING) << "send data failed(rpc), retry later, task: " << task.ShortDebugString(); + delay_longer = true; + } else { + bool destroy_env = false; + // got sync tool response to delete outdated task + if (response.has_delete_task() && response.delete_task()) { + destroy_env = true; + } + if (response.response().code() != 0) { + // retry or delete below + LOG(WARNING) << "send data failed(sync), task: " << task.ShortDebugString() + << ", response: " << response.ShortDebugString(); + } + + // if sync_point field will changed, even is_finished, store the last change + if (response.response().code() == 0 && need_update) { + // update progress if send success + { + std::lock_guard lock(process_map_mutex_); + // update task info in memory, replace the old one(set replace to avoid failed when the same token) + AddTaskInfoUnlocked(&updated_task, true); + // save task info in disk + SaveTaskInfoInDisk(&updated_task, GetWorkDir(name)); + } + } + // (count == 0 && is_finished == true) is possible, so we do independent check for is_finished + if (destroy_env || is_finished) { + if (destroy_env) { + LOG(INFO) << "task fault and need to be deleted, tid: " << tid << ", pid: " << pid; + } else if (is_finished) { + LOG(INFO) << "task finished, tid: " << tid << ", pid: " << pid; + } + CleanTaskEnv(name); + return; + } + } + + // Add next task.For each task, the sync progress is serial, so we add the next task to the task pool after the + // current task is done. + if (delay_longer) { + // if meet binlog end, we can delay the next sync longer TODO(hw): param + task_pool_.DelayTask(FLAGS_sync_task_long_interval_ms, std::bind(&DataCollectorImpl::SyncOnce, this, tid, pid)); + } else { + task_pool_.DelayTask(FLAGS_sync_task_short_interval_ms, + std::bind(&DataCollectorImpl::SyncOnce, this, tid, pid)); + } +} + +bool DataCollectorImpl::PackData(const datasync::AddSyncTaskRequest& task, butil::IOBuf* io_buf, uint64_t* count, + datasync::SyncPoint* next_point, bool* meet_binlog_end) { + auto name = EncodeId(task.tid(), task.pid()); + auto start_point = task.sync_point(); + auto mode = task.mode(); + auto start_ts = task.start_ts(); + + auto pack_with_ts = [io_buf, count, &start_ts](uint64_t ts, const base::Slice& slice) { + if (ts < start_ts) { + return true; + } + if (io_buf->size() + slice.size() > FLAGS_max_pack_size) { + return false; + } + auto ret = io_buf->append(slice.data(), slice.size()); + if (ret != 0) { + LOG(WARNING) << "fail to append slice to io_buf"; + return false; + } + *count += 1; + return true; + }; + // ts is not used in mode 0/2 + auto pack = [io_buf, count](uint64_t ts, const base::Slice& slice) { + if (io_buf->size() + slice.size() > FLAGS_max_pack_size) { + return false; + } + auto ret = io_buf->append(slice.data(), slice.size()); + if (ret != 0) { + LOG(WARNING) << "fail to append slice to io_buf"; + return false; + } + *count += 1; + return true; + }; + + if (start_point.type() == datasync::SyncType::kSNAPSHOT) { + // read snapshot + std::shared_ptr iter; + uint64_t snapshot_offset = 0; // snapshot end offset + { + std::lock_guard lock(process_map_mutex_); + auto it = snapshot_map_.find(name); + if (it == snapshot_map_.end()) { + LOG(INFO) << "snapshot env does not exist, switch to binlog"; + next_point->set_type(datasync::SyncType::kBINLOG); + // start read from 1(include) + next_point->set_offset(1); + return true; + } + // 0 is the base index + iter.reset(it->second->NewTraverseIterator(0)); + snapshot_offset = it->second->GetOffset(); + } + if (iter == nullptr) { + LOG(WARNING) << "fail to get snapshot iterator of index 0"; + return false; + } + if (!start_point.has_pk()) { + LOG(INFO) << "read snapshot from begin"; + iter->SeekToFirst(); + } else { + LOG(INFO) << "read snapshot from pk: " << start_point.pk() << " ts: " << start_point.ts(); + iter->Seek(start_point.pk(), start_point.ts()); + } + + // pack Slices, include start_point, exclude next_point + bool finished = false; + if (mode == datasync::SyncMode::kIncrementalByTimestamp) { + // to parse the ts, we need the table meta + finished = PackSNAPSHOT(iter, pack_with_ts, next_point); + } else { + finished = PackSNAPSHOT(iter, pack, next_point); + } + // if snapshot finished, we should set the next offset to snapshot_offset(manifest's offset) + 1 + if (finished) { + next_point->set_type(datasync::SyncType::kBINLOG); + next_point->set_offset(snapshot_offset + 1); + } + } else { + std::shared_ptr reader; + // we read the binlog path in db, it can be shared + // DO NOT write in the binlog path + // it's ok to check no_binlog before reader + { + std::lock_guard lock(process_map_mutex_); + if (replicator_map_.find(name) == replicator_map_.end()) { + LOG(WARNING) << "replicator not exist, task " << task.ShortDebugString(); + return false; + } + auto replicator = replicator_map_[name]; + // if no binlog file(partition is empty), we can check it + bool no_binlog = false; + reader = getLogReader(replicator, start_point.offset(), &no_binlog); + if (no_binlog) { + LOG(INFO) << "no binlog file, task " << task.ShortDebugString(); + // buf stay the same + *count = 0; + *next_point = start_point; + *meet_binlog_end = true; + return true; + } + } + if (reader == nullptr) { + LOG(WARNING) << "fail to get binlog reader, task " << task.ShortDebugString(); + return false; + } + bool ok = false; + if (mode == datasync::SyncMode::kIncrementalByTimestamp) { + ok = PackBINLOG(reader, start_point.offset(), pack_with_ts, next_point, meet_binlog_end); + } else { + ok = PackBINLOG(reader, start_point.offset(), pack, next_point, meet_binlog_end); + } + if (!ok) { + LOG(WARNING) << "fail to pack binlog, task " << task.ShortDebugString(); + return false; + } + } + return true; +} + +// snapshot iterator is simple, no where to know if error, so the return value is used to indicate if finished +// return true: read snapshot finished, false: still has snapshot to read +// If finished, don't forget to switch to binlog and set offset later +template +bool DataCollectorImpl::PackSNAPSHOT(std::shared_ptr it, Func pack_func, + datasync::SyncPoint* next_point) { + while (it->Valid()) { + if (!pack_func(it->GetKey(), it->GetValue())) { + break; + } + it->Next(); + } + + // still has next to read + if (it->Valid()) { + next_point->set_type(datasync::SyncType::kSNAPSHOT); + next_point->set_pk(it->GetPK()); + next_point->set_ts(it->GetKey()); + } + return !it->Valid(); +} + +// TODO(hw): what if SchedDelBinlog is called in tablet server when we are reading binlog? +// return the next offset to read? +template +int PackRecords(std::shared_ptr reader, uint64_t start_offset, Func&& pack_func, uint64_t* next_offset, + bool* meet_binlog_end) { + api::LogEntry entry; + std::string buffer; + base::Slice record; + log::Status status; + int cur_log_index = reader->GetLogIndex(); + *meet_binlog_end = false; + *next_offset = start_offset; + // TODO(hw): refactor this + while (true) { + status = reader->ReadNextRecord(&record, &buffer); + if (status.ok()) { + if (!entry.ParseFromString(record.ToString())) { + LOG(ERROR) << "parse log entry failed, skip it. " << base::DebugString(record.ToString()); + continue; + } + // TODO(hw): just skip delete record? + if (entry.has_method_type() && entry.method_type() == api::MethodType::kDelete) { + continue; + } + // TODO(hw): need check index in order? + if (entry.log_index() >= start_offset) { + if (!pack_func(entry.ts(), entry.value())) { + // stop pack, current entry is exclude, so read in next. + // and meet_binlog_end is false + *next_offset = entry.log_index(); + return 0; + } + *next_offset = entry.log_index() + 1; + } + } else if (status.IsWaitRecord()) { + // handle unfinished binlog file + // ref Binlog::RecoverFromBinlog, but don't use it directly, cuz we can't modify the binlog in db + // it isn't an error status, just wait for next record + int end_log_index = reader->GetEndLogIndex(); + int cur_log_index = reader->GetLogIndex(); + if (end_log_index >= 0 && end_log_index > cur_log_index) { + reader->RollRLogFile(); + continue; + } + // read the opening binlog file, meet the last record, it's ok + *meet_binlog_end = true; + return 0; + } else if (status.IsEof()) { + // should read next binlog file + if (reader->GetLogIndex() != cur_log_index) { + cur_log_index = reader->GetLogIndex(); + continue; + } + // last binlog file has been read + *meet_binlog_end = true; + return 0; + } else { + // TODO(hw): should abort when read failed? + LOG(ERROR) << "read record failed, skip it, status: " << status.ToString(); + continue; + } + } + return -1; +} + +template +bool DataCollectorImpl::PackBINLOG(std::shared_ptr reader, uint64_t start_offset, Func pack_func, + datasync::SyncPoint* next_point, bool* meet_binlog_end) { + uint64_t next_offset = 0; + auto ret = PackRecords(reader, start_offset, std::move(pack_func), &next_offset, meet_binlog_end); + if (ret != 0) { + LOG(WARNING) << "PackRecords failed, ret: " << ret << ". maybe binlog is break or no data"; + return false; + } + next_point->set_type(datasync::SyncType::kBINLOG); + next_point->set_offset(next_offset); + return true; +} + +// use response and update_task iff status == true (the response may be not ok, and should delete the task) +bool DataCollectorImpl::SendDataUnlock(const datasync::AddSyncTaskRequest* task, butil::IOBuf& data, uint64_t count, + const datasync::SyncPoint& next_point, bool is_finished, + datasync::SendDataResponse* reponse, datasync::AddSyncTaskRequest* update_task) { + // pack send data request + datasync::SendDataRequest send_data_request; + send_data_request.set_tid(task->tid()); + send_data_request.set_pid(task->pid()); + send_data_request.mutable_start_point()->CopyFrom(task->sync_point()); + send_data_request.set_count(count); + send_data_request.set_token(task->token()); + send_data_request.mutable_next_point()->CopyFrom(next_point); + send_data_request.set_finished(is_finished); + + // send to sync tool + auto sync_tool_addr = task->des_endpoint(); + SyncToolClient client(sync_tool_addr); + if (!client.Init()) { + LOG(WARNING) << "fail to init sync tool client, task " << task->ShortDebugString(); + return false; + } + + if (!client.SendData(&send_data_request, data, reponse)) { + LOG(WARNING) << "fail to send data to sync tool, task " << task->ShortDebugString(); + return false; + } + + // update, if response is not ok, this new update task shouldn't be added to task map + if (update_task) { + update_task->CopyFrom(*task); + update_task->mutable_sync_point()->CopyFrom(next_point); + } + + return true; +} + +// recover from disk, report the failed tasks, but it's not mean the data collector shouldn't run +void DataCollectorImpl::Recover() { + // read sync task info from disk, get binlog path in the runtime(if tablet is invalid, no need to recover sync + // task) and do some validations + LOG(INFO) << "start to recover sync task"; + // walk through the sync task dir FLAGS_collector_datadir + if (!fs::exists(FLAGS_collector_datadir)) { + LOG(INFO) << "collector data dir not exist, skip recover. dir: " << FLAGS_collector_datadir; + return; + } + // datadir//task.progress + std::vector failed_tasks; + for (const auto& entry : fs::directory_iterator(FLAGS_collector_datadir)) { + if (!entry.is_directory()) { + LOG(WARNING) << "irrelavant file? " << entry.path(); + continue; + } + LOG(INFO) << "try recover: " << entry.path(); + // get the last directory name, parse tid-pid + auto name = entry.path().filename(); + + // read task.progress + datasync::AddSyncTaskRequest recovered_task; + if (!LoadTaskInfoFromDisk(entry.path().string() + "/", &recovered_task)) { + LOG(WARNING) << "fail to load task info from disk, skip. task dir: " << entry.path(); + failed_tasks.push_back(name); + continue; + } + // create task backup + datasync::GeneralResponse response; + CreateTaskEnv(&recovered_task, &response); + if (response.code() != 0) { + LOG(WARNING) << "fail to create task env for recovery, skip. task dir: " << entry.path(); + failed_tasks.push_back(name); + } + } + if (!failed_tasks.empty()) { + LOG(WARNING) << "fail to recover some tasks, please check the log. failed tasks: " + << absl::StrJoin(failed_tasks, ","); + } +} + +void DataCollectorImpl::CleanTaskEnv(const std::string& name) { + // destroy the task env + { + std::lock_guard lock(process_map_mutex_); + process_map_.erase(name); + snapshot_map_.erase(name); + replicator_map_.erase(name); + } + { + std::lock_guard lock(cache_mutex_); + binlog_path_map_.erase(name); + } + // TODO(hw): store some history data? + fs::remove_all(GetWorkDir(name)); +} +} // namespace openmldb::datacollector diff --git a/src/datacollector/data_collector.h b/src/datacollector/data_collector.h new file mode 100644 index 00000000000..8907ee31f04 --- /dev/null +++ b/src/datacollector/data_collector.h @@ -0,0 +1,185 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef SRC_DATACOLLECTOR_DATA_COLLECTOR_H_ +#define SRC_DATACOLLECTOR_DATA_COLLECTOR_H_ + +#include +#include +#include +#include + +#include "common/thread_pool.h" + +#include "base/file_util.h" +#include "client/tablet_client.h" +#include "log/log_reader.h" +#include "proto/data_sync.pb.h" +#include "proto/tablet.pb.h" +#include "replica/log_replicator.h" +#include "storage/disk_table.h" +#include "storage/snapshot.h" +#include "zk/zk_client.h" + +DECLARE_int32(request_timeout_ms); +namespace openmldb::datacollector { + +static const char kDataCollectorRegisterPath[] = "/sync_tool/collector"; + +// To hold log parts, we get log reader from log replicator +std::shared_ptr genLogReplicatorFromBinlog(const std::string& binlog_path); + +std::shared_ptr getLogReader(std::shared_ptr replicator, uint64_t start_offset, + bool* no_binlog_file); + +// two path need end with '/' +bool HardLinkSnapshot(const std::string& snapshot_path, const std::string& dest, uint64_t* snapshot_offset); + +bool SaveTaskInfoInDisk(const datasync::AddSyncTaskRequest* info, const std::string& path); +class SyncToolClient { + public: + explicit SyncToolClient(const std::string& endpoint) : client_(endpoint) {} + ~SyncToolClient() {} + bool Init() { + if (client_.Init() != 0) return false; + return true; + } + // if ok, check repsonse, otherwise do not use response + bool SendData(const datasync::SendDataRequest* request, butil::IOBuf& data, // NOLINT + datasync::SendDataResponse* response) { + // TODO(hw): IOBufAppender is better? + auto st = client_.SendRequestSt(&datasync::SyncTool_Stub::SendData, + [&data](brpc::Controller* cntl) { cntl->request_attachment().swap(data); }, + request, response, FLAGS_request_timeout_ms, 1); + if (!st.OK()) { + LOG(WARNING) << "send data rpc failed, " << st.GetCode() << ", " << st.GetMsg(); + return false; + } + return true; + } + + private: + RpcClient client_; +}; + +class DataCollectorImpl : public datasync::DataCollector { + public: + DataCollectorImpl(); + ~DataCollectorImpl(); + // setup zk client, recover if process exist + bool Init(const std::string& endpoint); + bool Init(const std::string& zk_cluster, const std::string& zk_path, const std::string& endpoint); + + // should call after Init(setup zk client) + bool RegisterZK(); + void CheckZkClient(); + + void AddSyncTask(::google::protobuf::RpcController* controller, + const ::openmldb::datasync::AddSyncTaskRequest* request, + ::openmldb::datasync::GeneralResponse* response, ::google::protobuf::Closure* done) override; + // limited by rpc size, so we need to split task + void SyncOnce(uint32_t tid, uint32_t pid); + + // if not exist, return empty, you can check it by has_tid() + datasync::AddSyncTaskRequest GetTaskInfo(uint32_t tid, uint32_t pid) { + std::lock_guard lock(process_map_mutex_); + auto name = EncodeId(tid, pid); + // warning if not exist + if (process_map_.find(name) == process_map_.end()) { + LOG(WARNING) << "task not exist, tid: " << tid << ", pid: " << pid; + return {}; + } + return process_map_[name]; + } + + private: + void Recover(); + + // check status by using response + void CreateTaskEnv(const datasync::AddSyncTaskRequest* request, datasync::GeneralResponse* response); + + void CleanTaskEnv(const std::string& name); + + std::string EncodeId(uint32_t tid, uint32_t pid) { return std::to_string(tid) + "-" + std::to_string(pid); } + + std::string GetWorkDir(const std::string& name); + + bool AddTaskInfoUnlocked(const datasync::AddSyncTaskRequest* request, bool replace = false) { + // "tid-pid" as key + auto name = EncodeId(request->tid(), request->pid()); + auto it = process_map_.find(name); + if (it != process_map_.end()) { + LOG(INFO) << "task replace, old: " << it->second.ShortDebugString() + << ", new: " << request->ShortDebugString(); + if (!replace && it->second.token() == request->token()) { + LOG(ERROR) << "token is same, ignore"; + return false; + } + } + process_map_.insert_or_assign(name, *request); + return true; + } + + bool ValidateTableStatus(const api::TableStatus& table_status); + bool CreateSnapshotEnvUnlocked(const std::string& name, const api::TableMeta& meta, + const std::string& snapshot_path); + bool FetchBinlogUnlocked(const std::string& name, const std::string& binlog_path, bool* updated); + + bool PackData(const datasync::AddSyncTaskRequest& task, butil::IOBuf* io_buf, uint64_t* count, + datasync::SyncPoint* next_point, bool* meet_binlog_end); + + template + bool PackSNAPSHOT(std::shared_ptr it, Func pack_func, datasync::SyncPoint* next_point); + template + bool PackBINLOG(std::shared_ptr reader, uint64_t start_offset, Func pack_func, + datasync::SyncPoint* next_point, bool* meet_binlog_end); + + // data will be cleared, whether send success or not + bool SendDataUnlock(const datasync::AddSyncTaskRequest* task, butil::IOBuf& data, uint64_t count, // NOLINT + const datasync::SyncPoint& next_point, bool is_finished, datasync::SendDataResponse* reponse, + datasync::AddSyncTaskRequest* update_task); + + std::shared_ptr GetReplicator(const std::string& name) { + std::lock_guard lock(process_map_mutex_); + auto it = replicator_map_.find(name); + if (it == replicator_map_.end()) { + LOG(WARNING) << "replicator not exist, name: " << name; + return nullptr; + } + return it->second; + } + + private: + std::shared_ptr zk_client_; + // zk background thread + baidu::common::ThreadPool keep_alive_pool_; + // protect 4 maps + std::mutex process_map_mutex_; + // tid-pid -> ..., we use tid-pid as key cuz data collector doesn't need to know the status of whole tid + std::map process_map_; + std::map> snapshot_map_; + // store LogRepliactor, so we can get LogParts to seek and LogReader to read + std::map> replicator_map_; + // tablet_endpoint -> tablet_client + std::map> tablet_client_map_; + + // smaller lock, cache the binlog path, don't use the big lock process_map_mutex_ to lock it + std::mutex cache_mutex_; + std::map binlog_path_map_; + baidu::common::ThreadPool task_pool_; +}; +} // namespace openmldb::datacollector + +#endif // SRC_DATACOLLECTOR_DATA_COLLECTOR_H_ diff --git a/src/datacollector/data_collector_server.cc b/src/datacollector/data_collector_server.cc new file mode 100644 index 00000000000..95e717102f5 --- /dev/null +++ b/src/datacollector/data_collector_server.cc @@ -0,0 +1,64 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "datacollector/data_collector.h" + +#include "brpc/server.h" +#include "gflags/gflags.h" + +#include "base/glog_wrapper.h" +#include "version.h" // NOLINT + +DECLARE_string(endpoint); +DECLARE_int32(thread_pool_size); + +const std::string OPENMLDB_VERSION = std::to_string(OPENMLDB_VERSION_MAJOR) + "." + // NOLINT + std::to_string(OPENMLDB_VERSION_MINOR) + "." + + std::to_string(OPENMLDB_VERSION_BUG) + "-" + OPENMLDB_COMMIT_ID; + +int main(int argc, char* argv[]) { + ::google::SetVersionString(OPENMLDB_VERSION); + ::google::ParseCommandLineFlags(&argc, &argv, true); + FLAGS_role = "data_collector"; + openmldb::base::SetupGlog(); + // no logic about real_endpoint + auto data_collector = new openmldb::datacollector::DataCollectorImpl(); + if (!data_collector->Init(FLAGS_endpoint)) { + LOG(WARNING) << "init data collector failed"; + exit(1); + } + if (!data_collector->RegisterZK()) { + LOG(WARNING) << "Fail to register zk"; + exit(1); + } + + brpc::ServerOptions options; + options.num_threads = FLAGS_thread_pool_size; + brpc::Server server; + if (server.AddService(data_collector, brpc::SERVER_DOESNT_OWN_SERVICE) != 0) { + LOG(WARNING) << "Fail to add service"; + exit(1); + } + + if (server.Start(FLAGS_endpoint.c_str(), &options) != 0) { + LOG(WARNING) << "Fail to start server"; + exit(1); + } + LOG(INFO) << "start data collector on endpoint " << FLAGS_endpoint << " with version " << OPENMLDB_VERSION; + server.set_version(OPENMLDB_VERSION.c_str()); + server.RunUntilAskedToQuit(); + return 0; +} diff --git a/src/datacollector/data_collector_te1st.cc b/src/datacollector/data_collector_te1st.cc new file mode 100644 index 00000000000..efff494a8b5 --- /dev/null +++ b/src/datacollector/data_collector_te1st.cc @@ -0,0 +1,287 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "datacollector/data_collector.h" + +#include "gflags/gflags.h" +#include "gtest/gtest.h" + +#include "codec/sdk_codec.h" +#include "vm/engine.h" + +using openmldb::client::TabletClient; +namespace openmldb::datacollector { + +class MockClosure : public ::google::protobuf::Closure { + public: + MockClosure() {} + ~MockClosure() {} + void Run() override {} +}; + +class DataCollectorTest : public ::testing::Test { + public: + DataCollectorTest() {} + ~DataCollectorTest() {} + + api::TableStatus GetTableStatus(uint32_t tid, uint32_t pid) { + std::shared_ptr tablet_client = std::make_shared("172.24.4.27:7126", ""); + tablet_client->Init(); + api::TableStatus table_status; + tablet_client->GetTableStatus(tid, pid, table_status); + LOG(INFO) << "table status: " << table_status.ShortDebugString(); + return table_status; + } + + api::TableMeta GetTableSchema(uint32_t tid, uint32_t pid) { + std::shared_ptr tablet_client = std::make_shared("172.24.4.27:7126", ""); + tablet_client->Init(); + api::TableMeta table_meta; + tablet_client->GetTableSchema(tid, pid, table_meta); + LOG(INFO) << "table meta: " << table_meta.ShortDebugString(); + return table_meta; + } + + std::string VecToString(std::vector vec) { + std::string str; + for (auto& s : vec) { + str += s + " "; + } + return str; + } + + void TraverseAndPrint(storage::TraverseIterator* it, api::TableMeta meta, uint64_t limit = 0, + uint64_t start_ts = 0) { + codec::SDKCodec codec(meta); + std::vector vrow; + uint64_t i = 0; + while (it->Valid() && (limit == 0 || i++ < limit)) { + if (start_ts != 0 && it->GetKey() < start_ts) { + // ts is lower than start_ts, skip this pk + it->NextPK(); + continue; + } + vrow.clear(); + codec.DecodeRow(it->GetValue().ToString(), &vrow); + LOG(INFO) << "pk " << it->GetPK() << ", ts " << it->GetKey() << ", value: " << VecToString(vrow); + it->Next(); + } + } +}; + +TEST_F(DataCollectorTest, getTableDataPath) { + // try get tid-pid 6-0 data paths + // must be the disk table + auto table_status = GetTableStatus(6, 0); + + ASSERT_NE(table_status.storage_mode(), common::StorageMode::kMemory); + ASSERT_FALSE(table_status.snapshot_path().empty()); + ASSERT_FALSE(table_status.binlog_path().empty()); + ASSERT_TRUE(std::filesystem::path(table_status.snapshot_path()).is_absolute()); + ASSERT_TRUE(std::filesystem::path(table_status.binlog_path()).is_absolute()); +} + +TEST_F(DataCollectorTest, readSnapshot) { + auto snapshot_path = GetTableStatus(6, 0).snapshot_path(); + // hard link, notice hard link can't cross different disk + std::string data_collector_tmp_path = "/home/huangwei/tmp/data_collector_test/"; + auto snapshot_hardlink_path = data_collector_tmp_path + "6_0/"; + ASSERT_TRUE(HardLinkSnapshot(snapshot_path, snapshot_hardlink_path, nullptr)); + // the snapshot has the data offset [0, offset_in_manifest] + // read the manifest to get the offset + api::Manifest manifest; + std::string manifest_file = snapshot_hardlink_path + "MANIFEST"; + if (storage::Snapshot::GetLocalManifest(manifest_file, manifest) != 0) { + LOG(FATAL) << "get manifest failed"; + } + LOG(INFO) << "manifest: " << manifest.ShortDebugString(); + // mock table + auto meta = GetTableSchema(6, 0); + // mapping empty? ttl&ttl type, get from namserver? + storage::DiskTable mock_table(meta, snapshot_hardlink_path); + mock_table.Init(); + LOG(INFO) << mock_table.GetOffset(); + // traverse will go through all data, not only current pk + auto it = mock_table.NewTraverseIterator(0); + // assume start sync point is [snapshot, no pk&ts(means start from 0)] + it->SeekToFirst(); + TraverseAndPrint(it, meta); + + // if request start sync point is [snapshot, pk&ts], seek to it, read include it + // assume start sync point is [snapshot, pk&ts] + // order is [big, small]: 3, 2, 1, so print 2, 1 + auto pk = "a"; + uint64_t ts = 2; // == key + LOG(INFO) << "seek to " << pk << ", " << ts; + it->Seek(pk, ts); + LOG(INFO) << "read two row"; + TraverseAndPrint(it, meta, 2); + // after read in limited, the current it is not read, or it may be invalid + if (it->Valid()) { + LOG(INFO) << "has next value in snapshot, point(next to read) is " << it->GetPK() << ", " << it->GetKey(); + } + + // mode 2 every pk start from ts, filter all ts >= start ts + it->SeekToFirst(); + LOG(INFO) << "mode 2, start ts " << ts; + LOG(INFO) << "read two row in mode 2"; + TraverseAndPrint(it, meta, 2, ts); + if (it->Valid()) { + LOG(INFO) << "has next value in snapshot, point(next to read) is " << it->GetPK() << ", " << it->GetKey() + << ". In mode 2, it may be filtered"; + it->Seek(it->GetPK(), it->GetKey()); + LOG(INFO) << "read all left row in mode 2"; + TraverseAndPrint(it, meta, UINT64_MAX, ts); + } + + if (!it->Valid()) { + // [binlog, start_offset(offset in snapshot and +1)] + LOG(INFO) << "snapshot end, read binlog, next to read(include me) is " << manifest.offset() + 1; + } + base::RemoveDir(data_collector_tmp_path); +} + +TEST_F(DataCollectorTest, readBinlog) { + auto binglog_path = GetTableStatus(6, 0).binlog_path(); + // assume start sync point is [binlog, 3] + auto replicator = genLogReplicatorFromBinlog(binglog_path); + // Set start offset only seek to the right log part, e.g. 3 is in log 0 [0, 100] + uint64_t start_offset = 3; + bool no_binlog = false; + auto reader = getLogReader(replicator, 3, &no_binlog); + ASSERT_TRUE(reader) << "can't gen log reader"; + // TODO(hw): make some binlog files to test + LOG(INFO) << "binlog min offset: " << reader->GetMinOffset(); + + ::openmldb::api::LogEntry entry; + std::string buffer; + ::openmldb::base::Slice record; + // this binlog file may have some records before start_offset, so we still need to filter + int cur_log_index = reader->GetLogIndex(); + + auto meta = GetTableSchema(6, 0); + codec::SDKCodec codec(meta); + std::vector vrow; + int i = 0; + while (true) { + buffer.clear(); + LOG(INFO) << "read " << i; + ::openmldb::log::Status status = reader->ReadNextRecord(&record, &buffer); + if (status.IsEof()) { + // should read next binlog file + if (reader->GetLogIndex() != cur_log_index) { + cur_log_index = reader->GetLogIndex(); + continue; + } + // last binlog file has been read + break; + } + // may meet a unfinished binlog file + if (status.IsWaitRecord()) { + // wait for next record + break; + } + + ASSERT_TRUE(status.ok()) << i << ": " << status.ToString(); + entry.ParseFromString(record.ToString()); + // parse entry + vrow.clear(); + codec.DecodeRow(entry.value(), &vrow); + LOG(INFO) << i << " value: " << VecToString(vrow); + if (entry.log_index() >= start_offset) { + // TODO(hw): which is key in entry? dim0 ts? + LOG(INFO) << "match >= start_offset: " << VecToString(vrow); + } + i++; + // what it in mode 2, start_ts = 2, filter all ts >= 2 + } + LOG(INFO) << "read binlog entry num: " << i; + // TODO(hw): what if read when writing? +} + +// init.sh and use playground disk_test.sql to create tables +// and trigger tid 6 to create snapshot +TEST_F(DataCollectorTest, taskAndRate) { + // test about tasks, no need to init zk + DataCollectorImpl dc; + MockClosure closure; + datasync::AddSyncTaskRequest request; + request.set_tid(6); + request.set_pid(0); + request.set_mode(datasync::SyncMode::FULL); + auto sync_point = request.mutable_sync_point(); + sync_point->set_type(datasync::SyncType::kSNAPSHOT); + // no pk&ts, means start from 0 + request.set_tablet_endpoint("172.24.4.27:7126"); + datasync::GeneralResponse response; + dc.AddSyncTask(nullptr, &request, &response, &closure); + ASSERT_EQ(response.code(), 0); + auto task = dc.GetTaskInfo(6, 0); + LOG(INFO) << task.ShortDebugString(); + ASSERT_EQ(task.tid(), 6); + ASSERT_EQ(task.pid(), 0); + // sleep to call SyncOnce more + sleep(1); + // new token task but snapshot in db doesn't change, so the snapshot env is not changed + request.set_mode(datasync::SyncMode::INCREMENTAL_BY_TIMESTAMP); + request.set_start_ts(3); + request.set_token("newer_one"); + dc.AddSyncTask(nullptr, &request, &response, &closure); + sleep(1); + + // a table doesn't have snapshot, make sure the snapshot path in db is empty + request.set_tid(7); + request.set_des_endpoint(""); + dc.AddSyncTask(nullptr, &request, &response, &closure); + sleep(1); + ASSERT_EQ(response.code(), 0); + + // a table doesn't exist + request.set_tid(10086); + dc.AddSyncTask(nullptr, &request, &response, &closure); + sleep(1); + ASSERT_EQ(response.code(), -1); + LOG(INFO) << "got resp: " << response.msg(); +} +} // namespace openmldb::datacollector + +int main(int argc, char** argv) { + // init google test first for gtest_xxx flags + ::testing::InitGoogleTest(&argc, argv); + ::google::ParseCommandLineFlags(&argc, &argv, true); + ::hybridse::vm::Engine::InitializeGlobalLLVM(); + ::openmldb::base::SetupGlog(true); + + // FLAGS_zk_session_timeout = 100000; + // ::openmldb::sdk::MiniCluster mc(6181); + // ::openmldb::sdk::mc_ = &mc; + // FLAGS_enable_distsql = true; + // int ok = ::openmldb::sdk::mc_->SetUp(3); + // sleep(5); + + // ::openmldb::sdk::router_ = ::openmldb::sdk::GetNewSQLRouter(); + // if (nullptr == ::openmldb::sdk::router_) { + // LOG(ERROR) << "Test failed with NULL SQL router"; + // return -1; + // } + + srand(time(nullptr)); + // TODO(hw): skip this test, cuz it needs sync tool. It's better to mock the sync tool + bool ok = true; + // ok = RUN_ALL_TESTS(); + // ::openmldb::sdk::mc_->Close(); + return ok; +} diff --git a/src/proto/data_sync.proto b/src/proto/data_sync.proto new file mode 100644 index 00000000000..f2e07cf5164 --- /dev/null +++ b/src/proto/data_sync.proto @@ -0,0 +1,120 @@ +/* + * Copyright 2021 4Paradigm + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +syntax = "proto2"; + +// datasync is a tool to sync data from online to offline +// need java package cus SyncTool is a java class +package openmldb.datasync; +option java_package = "com._4paradigm.openmldb.proto"; +option java_outer_classname = "DataSync"; +option cc_generic_services = true; + +enum SyncType { + kSNAPSHOT = 0; + kBINLOG = 1; +} + +enum SyncMode { + kFull = 0; + kIncrementalByTimestamp = 1; + kFullAndContinuous = 2; +} + +message GeneralResponse { + optional int32 code = 1; + optional string msg = 2; +} + +message SyncPoint { + required SyncType type = 1; + optional string pk = 2; + optional uint64 ts = 3; + optional uint64 offset = 4; +} + +message AddSyncTaskRequest { + optional uint32 tid = 1; + optional uint32 pid = 2; + optional SyncMode mode = 3; // 3 modes + optional uint64 start_ts = 4; // mode 1 INCREMENTAL_BY_TIMESTAMP start ts + optional SyncPoint sync_point = 5; // DataCollector should read data from this point, include it + optional string tablet_endpoint = 6; // the tablet server connect to + optional string des_endpoint = 7; // SyncTool address(which SendData to), may be worker + optional string token = 8; // token, DataCollector sends data with it, SyncTool will check it + optional string dest = 9; // offline path for sync tool persistency, data collector won't use it +} + +message CreateSyncTaskRequest { + optional string db = 1; + optional string name = 2; + optional SyncMode mode = 3; // 3 modes + optional uint64 start_ts = 4; // mode 1 INCREMENTAL_BY_TIMESTAMP start ts + optional string dest = 5; // offline path +} + +// only one sync task per table +message DeleteSyncTaskRequest { + optional string db = 1; + optional string name = 2; +} + +message TaskStatusRequest { + optional bool clear_all = 1; +} + +// used as backup data +message Task { + optional AddSyncTaskRequest progress = 1; + optional string data_collector = 2; + optional uint64 count = 3; + optional string status = 4; +} + +message TaskStatusResponse { + optional GeneralResponse response = 1; + // all tid-pid tasks + repeated Task task = 2; + repeated string readableInfo = 3; +} + +message SendDataRequest { + optional uint32 tid = 1; + optional uint32 pid = 2; + optional SyncPoint start_point = 3; // The start point of this data + optional uint64 count = 4; // the row count in attachment + optional string token = 5; + optional bool finished = 6; // if mode 0 and data collector meet the binlog end,tell SyncTool + optional SyncPoint next_point = 7; // The NEXT point of this partition, to let SyncTool know the next start point + // The data is in attachment +} + +message SendDataResponse { + required GeneralResponse response = 1; + optional bool delete_task = 2; // TODO(hw): delete task here. No extra delete task way. +} + +service DataCollector { + rpc AddSyncTask(AddSyncTaskRequest) returns (GeneralResponse); + // TODO(hw): GetTaskInfo in runtime +}; + +service SyncTool { + rpc CreateSyncTask(CreateSyncTaskRequest) returns (GeneralResponse); + rpc DelelteSyncTask(DeleteSyncTaskRequest) returns (GeneralResponse); + rpc TaskStatus(TaskStatusRequest) returns (TaskStatusResponse); + rpc SendData(SendDataRequest) returns (SendDataResponse); +} diff --git a/src/proto/tablet.proto b/src/proto/tablet.proto index c760a0d94a3..321719a8333 100755 --- a/src/proto/tablet.proto +++ b/src/proto/tablet.proto @@ -481,6 +481,8 @@ message TableStatus { optional uint32 skiplist_height = 18; optional uint64 diskused = 19 [default = 0]; optional openmldb.common.StorageMode storage_mode = 20 [default = kMemory]; + optional string snapshot_path = 21; + optional string binlog_path = 22; } message GetTableStatusResponse { diff --git a/src/replica/log_replicator.h b/src/replica/log_replicator.h index f4740cf9371..cfe4eddccdb 100644 --- a/src/replica/log_replicator.h +++ b/src/replica/log_replicator.h @@ -110,6 +110,8 @@ class LogReplicator { const std::string& GetLogPath() {return log_path_;} + uint64_t GetSnapshotLastOffset() { return snapshot_last_offset_.load(std::memory_order_relaxed); } + private: bool OpenSeqFile(const std::string& path, SequentialFile** sf); diff --git a/src/rpc/rpc_client.h b/src/rpc/rpc_client.h index cd9a6caabd8..375240dff50 100644 --- a/src/rpc/rpc_client.h +++ b/src/rpc/rpc_client.h @@ -147,7 +147,14 @@ class RpcClient { template base::Status SendRequestSt(void (T::*func)(google::protobuf::RpcController*, const Request*, Response*, Callback*), - const Request* request, Response* response, uint64_t rpc_timeout, int retry_times) { + const Request* request, Response* response, uint64_t rpc_timeout, int retry_times) { + return SendRequestSt(func, [](brpc::Controller* cntl) {}, request, response, rpc_timeout, retry_times); + } + + template + base::Status SendRequestSt(void (T::*func)(google::protobuf::RpcController*, const Request*, Response*, Callback*), + Func manual_set_cntl, const Request* request, Response* response, uint64_t rpc_timeout, + int retry_times) { base::Status status; brpc::Controller cntl; cntl.set_log_id(log_id_++); @@ -157,6 +164,8 @@ class RpcClient { if (retry_times > 0) { cntl.set_max_retry(retry_times); } + manual_set_cntl(&cntl); + VLOG(1) << "request attachment size: " << cntl.request_attachment().size(); if (stub_ == NULL) { PDLOG(WARNING, "stub is null. client must be init before send request"); return {base::ReturnCode::kServerConnError, "stub is null"}; diff --git a/src/sdk/CMakeLists.txt b/src/sdk/CMakeLists.txt index 440b9db04a2..836ec4926fc 100644 --- a/src/sdk/CMakeLists.txt +++ b/src/sdk/CMakeLists.txt @@ -203,6 +203,20 @@ if(SQL_JAVASDK_ENABLE) install(FILES ${TASKMANAGER_LIB} DESTINATION taskmanager/lib/ PERMISSIONS OWNER_WRITE OWNER_READ GROUP_READ WORLD_READ) + + # Install SyncTool binary and libraries + FILE(GLOB SYNCTOOL_BIN "${PROJECT_SOURCE_DIR}/java/openmldb-synctool/target/openmldb-synctool-binary/bin/*") + FILE(GLOB SYNCTOOL_CONF "${PROJECT_SOURCE_DIR}/java/openmldb-synctool/target/openmldb-synctool-binary/conf/log4j*") + FILE(GLOB SYNCTOOL_LIB "${PROJECT_SOURCE_DIR}/java/openmldb-synctool/target/openmldb-synctool-binary/lib/*") + install(FILES ${SYNCTOOL_BIN} + DESTINATION synctool/bin/ + PERMISSIONS OWNER_EXECUTE OWNER_WRITE OWNER_READ GROUP_EXECUTE GROUP_READ WORLD_EXECUTE WORLD_READ) + install(FILES ${SYNCTOOL_CONF} + DESTINATION synctool/conf/ + PERMISSIONS OWNER_WRITE OWNER_READ GROUP_READ WORLD_READ) + install(FILES ${SYNCTOOL_LIB} + DESTINATION synctool/lib/ + PERMISSIONS OWNER_WRITE OWNER_READ GROUP_READ WORLD_READ) endif() # c++ sdk @@ -299,7 +313,7 @@ COMMENT "pack static cxx sdk and all depend static libs into one" if(TESTING_ENABLE) add_executable(openmldb_api_test openmldb_api_test.cc) - target_link_libraries(openmldb_api_test ${BIN_LIBS} ${GTEST_LIBRARIES}) + target_link_libraries(openmldb_api_test ${EXPORTER_LIBS} ${GTEST_LIBRARIES}) endif() FILE(GLOB USER_HEADER diff --git a/src/storage/snapshot.h b/src/storage/snapshot.h index 2a5055cc8ce..65dbe3adb45 100644 --- a/src/storage/snapshot.h +++ b/src/storage/snapshot.h @@ -41,7 +41,7 @@ class Snapshot { int GenManifest(const std::string& snapshot_name, uint64_t key_count, uint64_t offset, uint64_t term); static int GetLocalManifest(const std::string& full_path, ::openmldb::api::Manifest& manifest); // NOLINT - + std::string GetSnapshotPath() { return snapshot_path_; } protected: uint32_t tid_; uint32_t pid_; diff --git a/src/tablet/tablet_impl.cc b/src/tablet/tablet_impl.cc index c7c9404594a..3059bb64f6a 100644 --- a/src/tablet/tablet_impl.cc +++ b/src/tablet/tablet_impl.cc @@ -18,6 +18,7 @@ #include #include +#include #include #include "absl/time/clock.h" #include "absl/time/time.h" @@ -28,14 +29,14 @@ #include #include // NOLINT +#include #include #include -#include #include "absl/cleanup/cleanup.h" #include "boost/bind.hpp" #include "boost/container/deque.hpp" -#include "config.h" // NOLINT +#include "config.h" // NOLINT #ifdef TCMALLOC_ENABLE #include "gperftools/malloc_extension.h" #endif @@ -58,17 +59,17 @@ #include "google/protobuf/text_format.h" #include "schema/schema_adapter.h" #include "storage/binlog.h" +#include "storage/disk_table_snapshot.h" #include "storage/segment.h" -#include "tablet/file_sender.h" #include "storage/table.h" -#include "storage/disk_table_snapshot.h" +#include "tablet/file_sender.h" -using google::protobuf::RepeatedPtrField; using ::openmldb::base::ReturnCode; using ::openmldb::codec::SchemaCodec; using ::openmldb::storage::DataBlock; -using ::openmldb::storage::Table; using ::openmldb::storage::DiskTable; +using ::openmldb::storage::Table; +using google::protobuf::RepeatedPtrField; DECLARE_int32(gc_interval); DECLARE_int32(gc_pool_size); @@ -339,7 +340,7 @@ void TabletImpl::UpdateTTL(RpcController* ctrl, const ::openmldb::api::UpdateTTL response->set_code(::openmldb::base::ReturnCode::kTtlTypeMismatch); response->set_msg("ttl type mismatch"); PDLOG(WARNING, "ttl type mismatch request type %d current type %d. tid %u, pid %u", - ::openmldb::storage::TTLSt::ConvertTTLType(ttl.ttl_type()), index->GetTTLType(), tid, pid); + ::openmldb::storage::TTLSt::ConvertTTLType(ttl.ttl_type()), index->GetTTLType(), tid, pid); return; } } @@ -388,8 +389,8 @@ void TabletImpl::UpdateTTL(RpcController* ctrl, const ::openmldb::api::UpdateTTL base::SetResponseStatus(base::ReturnCode::kWriteDataFailed, "write meta data failed", response); return; } - PDLOG(INFO, "update table tid %u pid %u ttl to abs_ttl %lu lat_ttl %lu index_name %s", - tid, pid, abs_ttl, lat_ttl, index_name.c_str()); + PDLOG(INFO, "update table tid %u pid %u ttl to abs_ttl %lu lat_ttl %lu index_name %s", tid, pid, abs_ttl, lat_ttl, + index_name.c_str()); response->set_code(::openmldb::base::ReturnCode::kOk); response->set_msg("ok"); } @@ -740,9 +741,9 @@ void TabletImpl::Put(RpcController* controller, const ::openmldb::api::PutReques return; } if (table->GetStorageMode() == ::openmldb::common::StorageMode::kMemory && - memory_used_.load(std::memory_order_relaxed) > FLAGS_max_memory_mb) { + memory_used_.load(std::memory_order_relaxed) > FLAGS_max_memory_mb) { PDLOG(WARNING, "current memory %lu MB exceed max memory limit %lu MB. tid %u, pid %u", - memory_used_.load(std::memory_order_relaxed), FLAGS_max_memory_mb, tid, pid); + memory_used_.load(std::memory_order_relaxed), FLAGS_max_memory_mb, tid, pid); response->set_code(::openmldb::base::ReturnCode::kExceedMaxMemory); response->set_msg("exceed max memory"); return; @@ -755,8 +756,7 @@ void TabletImpl::Put(RpcController* controller, const ::openmldb::api::PutReques response->set_msg("invalid dimension parameter"); return; } - DLOG(INFO) << "put data to tid " << tid << " pid " << pid << " with key " - << request->dimensions(0).key(); + DLOG(INFO) << "put data to tid " << tid << " pid " << pid << " with key " << request->dimensions(0).key(); ok = table->Put(request->time(), request->value(), request->dimensions()); } if (!ok) { @@ -789,8 +789,8 @@ void TabletImpl::Put(RpcController* controller, const ::openmldb::api::PutReques // so the update should be protected within the replicator lock // in case there will be other Put jump into the middle auto update_aggr = [this, &request, &ok, &entry]() { - ok = UpdateAggrs(request->tid(), request->pid(), request->value(), - request->dimensions(), entry.log_index()); + ok = + UpdateAggrs(request->tid(), request->pid(), request->value(), request->dimensions(), entry.log_index()); }; UpdateAggrClosure closure(update_aggr); replicator->AppendEntry(entry, &closure); @@ -1453,8 +1453,8 @@ void TabletImpl::Traverse(RpcController* controller, const ::openmldb::api::Trav traverse_it->NextPK(); } else if (traverse_it->GetKey() == last_time) { uint32_t skip_cnt = request->has_ts_pos() ? request->ts_pos() : 1; - while (skip_cnt > 0 && traverse_it->Valid() && - traverse_it->GetPK() == last_pk && traverse_it->GetKey() == last_time) { + while (skip_cnt > 0 && traverse_it->Valid() && traverse_it->GetPK() == last_pk && + traverse_it->GetKey() == last_time) { traverse_it->Next(); skip_cnt--; } @@ -1538,14 +1538,13 @@ void TabletImpl::Traverse(RpcController* controller, const ::openmldb::api::Trav } for (const auto& pair : iter->second) { DLOG(INFO) << "encode pk " << key << " ts " << pair.first << " size " << pair.second.size(); - ::openmldb::codec::EncodeFull(key, pair.first, pair.second.data(), pair.second.size(), rbuffer, - offset); + ::openmldb::codec::EncodeFull(key, pair.first, pair.second.data(), pair.second.size(), rbuffer, offset); offset += (4 + 4 + 8 + key.length() + pair.second.size()); } } delete it; DLOG(INFO) << "tid " << tid << " pid " << pid << " traverse count " << scount << " last_pk " << last_pk - << " last_time " << last_time << " ts_pos " << ts_pos; + << " last_time " << last_time << " ts_pos " << ts_pos; response->set_code(::openmldb::base::ReturnCode::kOk); response->set_count(scount); response->set_pk(last_pk); @@ -2227,9 +2226,8 @@ void TabletImpl::AppendEntries(RpcController* controller, const ::openmldb::api: uint64_t term = replicator->GetLeaderTerm(); if (!follower_.load(std::memory_order_relaxed)) { if (!FLAGS_zk_cluster.empty() && request->term() < term) { - PDLOG(WARNING, - "leader id not match. request term %lu, cur term %lu, tid %u, pid %u", - request->term(), term, tid, pid); + PDLOG(WARNING, "leader id not match. request term %lu, cur term %lu, tid %u, pid %u", request->term(), + term, tid, pid); response->set_code(::openmldb::base::ReturnCode::kFailToAppendEntriesToReplicator); response->set_msg("fail to append entries to replicator"); return; @@ -2242,8 +2240,8 @@ void TabletImpl::AppendEntries(RpcController* controller, const ::openmldb::api: response->set_log_offset(last_log_offset); if (!FLAGS_zk_cluster.empty() && request->term() > term) { replicator->SetLeaderTerm(request->term()); - PDLOG(INFO, "get log_offset %lu and set term %lu. tid %u, pid %u", - last_log_offset, request->term(), tid, pid); + PDLOG(INFO, "get log_offset %lu and set term %lu. tid %u, pid %u", last_log_offset, request->term(), tid, + pid); return; } PDLOG(INFO, "first sync log_index! log_offset[%lu] tid[%u] pid[%u]", last_log_offset, tid, pid); @@ -2253,7 +2251,7 @@ void TabletImpl::AppendEntries(RpcController* controller, const ::openmldb::api: const auto& entry = request->entries(i); if (entry.log_index() <= last_log_offset) { PDLOG(WARNING, "entry log_index %lu cur log_offset %lu tid %u pid %u", request->entries(i).log_index(), - last_log_offset, tid, pid); + last_log_offset, tid, pid); continue; } if (!replicator->ApplyEntry(entry)) { @@ -2435,6 +2433,32 @@ void TabletImpl::GetTableStatus(RpcController* controller, const ::openmldb::api } status->set_idx_cnt(record_idx_cnt); } + } else { + // status about disk table's data paths + // snapshot path from DiskTableSnapshot + auto snapshot = GetSnapshotUnLock(table->GetId(), table->GetPid()); + if (snapshot) { + status->set_snapshot_path(snapshot->GetSnapshotPath()); + } else { + LOG(WARNING) << "snapshot is null. tid " << table->GetId() << " pid " << table->GetPid(); + } + + // binlog path from LogReplicator + auto log_rep = GetReplicatorUnLock(table->GetId(), table->GetPid()); + if (log_rep) { + // LogPath may be relative path, so we need to convert it to absolute path + std::filesystem::path p = log_rep->GetLogPath(); + std::error_code ec; + auto abs_path = std::filesystem::absolute(p, ec); + if (ec) { + LOG(WARNING) << "log_rep path is not absolute path. tid " << table->GetId() << " pid " + << table->GetPid(); + } else { + status->set_binlog_path(abs_path); + } + } else { + LOG(WARNING) << "log_rep is null. tid " << table->GetId() << " pid " << table->GetPid(); + } } } } @@ -3316,7 +3340,7 @@ int32_t TabletImpl::DeleteTableInternal(uint32_t tid, uint32_t pid, uint32_t base_tid = table->GetTableMeta()->base_table_tid(); if (base_tid > 0) { std::lock_guard spin_lock(spin_mutex_); - uint64_t uid = (uint64_t) base_tid << 32 | pid; + uint64_t uid = (uint64_t)base_tid << 32 | pid; auto it = aggregators_.find(uid); if (it != aggregators_.end()) { auto aggrs = *it->second; @@ -3506,7 +3530,7 @@ void TabletImpl::GetTableFollower(RpcController* controller, const ::openmldb::a } int32_t TabletImpl::GetSnapshotOffset(uint32_t tid, uint32_t pid, openmldb::common::StorageMode storageMode, - std::string& msg, uint64_t& term, uint64_t& offset) { + std::string& msg, uint64_t& term, uint64_t& offset) { std::string db_root_path; bool ok = ChooseDBRootPath(tid, pid, storageMode, db_root_path); if (!ok) { @@ -3871,8 +3895,7 @@ int TabletImpl::CreateTableInternal(const ::openmldb::api::TableMeta* table_meta std::shared_ptr replicator; if (table->IsLeader()) { - replicator = - std::make_shared(tid, pid, table_db_path, real_ep_map, ReplicatorRole::kLeaderNode); + replicator = std::make_shared(tid, pid, table_db_path, real_ep_map, ReplicatorRole::kLeaderNode); } else { replicator = std::make_shared(tid, pid, table_db_path, std::map(), ReplicatorRole::kFollowerNode); @@ -4188,7 +4211,7 @@ std::shared_ptr TabletImpl::GetAggregators(uint32_t tid, uint32_t pid) { } std::shared_ptr TabletImpl::GetAggregatorsUnLock(uint32_t tid, uint32_t pid) { - uint64_t uid = (uint64_t) tid << 32 | pid; + uint64_t uid = (uint64_t)tid << 32 | pid; auto it = aggregators_.find(uid); if (it != aggregators_.end()) { return it->second; @@ -4197,7 +4220,7 @@ std::shared_ptr TabletImpl::GetAggregatorsUnLock(uint32_t tid, uint32_t p } bool TabletImpl::UpdateAggrs(uint32_t tid, uint32_t pid, const std::string& value, - const ::openmldb::storage::Dimensions& dimensions, uint64_t log_offset) { + const ::openmldb::storage::Dimensions& dimensions, uint64_t log_offset) { auto aggrs = GetAggregators(tid, pid); if (!aggrs) { return true; @@ -4209,8 +4232,8 @@ bool TabletImpl::UpdateAggrs(uint32_t tid, uint32_t pid, const std::string& valu } auto ok = aggr->Update(iter->key(), value, log_offset); if (!ok) { - PDLOG(WARNING, "update aggr failed. tid[%u] pid[%u] index[%u] key[%s] value[%s]", - tid, pid, iter->idx(), iter->key().c_str(), value.c_str()); + PDLOG(WARNING, "update aggr failed. tid[%u] pid[%u] index[%u] key[%s] value[%s]", tid, pid, iter->idx(), + iter->key().c_str(), value.c_str()); return false; } } @@ -4218,7 +4241,6 @@ bool TabletImpl::UpdateAggrs(uint32_t tid, uint32_t pid, const std::string& valu return true; } - void TabletImpl::ShowMemPool(RpcController* controller, const ::openmldb::api::HttpRequest* request, ::openmldb::api::HttpResponse* response, Closure* done) { brpc::ClosureGuard done_guard(done); @@ -4272,14 +4294,14 @@ bool TabletImpl::RefreshSingleTable(uint32_t tid) { void TabletImpl::UpdateGlobalVarTable() { auto table_handler = catalog_->GetTable(nameserver::INFORMATION_SCHEMA_DB, nameserver::GLOBAL_VARIABLES); if (!table_handler) { - LOG(WARNING) << "fail to get table handler. db " << nameserver::INFORMATION_SCHEMA_DB << - " table " << nameserver::GLOBAL_VARIABLES; + LOG(WARNING) << "fail to get table handler. db " << nameserver::INFORMATION_SCHEMA_DB << " table " + << nameserver::GLOBAL_VARIABLES; return; } auto it = table_handler->GetIterator(); if (!it) { - LOG(WARNING) << "fail to get full iterator. db " << nameserver::INFORMATION_SCHEMA_DB << - " table " << nameserver::GLOBAL_VARIABLES; + LOG(WARNING) << "fail to get full iterator. db " << nameserver::INFORMATION_SCHEMA_DB << " table " + << nameserver::GLOBAL_VARIABLES; return; } auto old_global_var = std::atomic_load_explicit(&global_variables_, std::memory_order_relaxed); @@ -5101,8 +5123,8 @@ void TabletImpl::LoadIndexDataInternal(uint32_t tid, uint32_t pid, uint32_t cur_ } void TabletImpl::ExtractMultiIndexData(RpcController* controller, - const ::openmldb::api::ExtractMultiIndexDataRequest* request, - ::openmldb::api::GeneralResponse* response, Closure* done) { + const ::openmldb::api::ExtractMultiIndexDataRequest* request, + ::openmldb::api::GeneralResponse* response, Closure* done) { brpc::ClosureGuard done_guard(done); uint32_t tid = request->tid(); uint32_t pid = request->pid(); @@ -5117,17 +5139,16 @@ void TabletImpl::ExtractMultiIndexData(RpcController* controller, return; } if (table->GetTableStat() != ::openmldb::storage::kNormal) { - PDLOG(WARNING, "table state is %d, cannot extract index data. tid %u, pid %u", - table->GetTableStat(), tid, pid); - base::SetResponseStatus(base::ReturnCode::kTableStatusIsNotKnormal, - "table status is not kNormal", response); + PDLOG(WARNING, "table state is %d, cannot extract index data. tid %u, pid %u", table->GetTableStat(), tid, + pid); + base::SetResponseStatus(base::ReturnCode::kTableStatusIsNotKnormal, "table status is not kNormal", + response); return; } snapshot = GetSnapshotUnLock(tid, pid); if (!snapshot) { PDLOG(WARNING, "snapshot does not exist. tid %u pid %u", tid, pid); - base::SetResponseStatus(base::ReturnCode::kSnapshotIsNotExist, - "table snapshot does not exist", response); + base::SetResponseStatus(base::ReturnCode::kSnapshotIsNotExist, "table snapshot does not exist", response); return; } } @@ -5179,17 +5200,17 @@ void TabletImpl::ExtractIndexData(RpcController* controller, const ::openmldb::a break; } if (table->GetTableStat() != ::openmldb::storage::kNormal) { - PDLOG(WARNING, "table state is %d, cannot extract index data. tid %u, pid %u", - table->GetTableStat(), tid, pid); - base::SetResponseStatus(base::ReturnCode::kTableStatusIsNotKnormal, - "table status is not kNormal", response); + PDLOG(WARNING, "table state is %d, cannot extract index data. tid %u, pid %u", table->GetTableStat(), + tid, pid); + base::SetResponseStatus(base::ReturnCode::kTableStatusIsNotKnormal, "table status is not kNormal", + response); break; } snapshot = GetSnapshotUnLock(tid, pid); if (!snapshot) { PDLOG(WARNING, "snapshot does not exist. tid %u pid %u", tid, pid); - base::SetResponseStatus(base::ReturnCode::kSnapshotIsNotExist, - "table snapshot does not exist", response); + base::SetResponseStatus(base::ReturnCode::kSnapshotIsNotExist, "table snapshot does not exist", + response); break; } } @@ -5738,7 +5759,7 @@ void TabletImpl::BulkLoad(RpcController* controller, const ::openmldb::api::Bulk } void TabletImpl::CreateFunction(RpcController* controller, const openmldb::api::CreateFunctionRequest* request, - openmldb::api::CreateFunctionResponse* response, Closure* done) { + openmldb::api::CreateFunctionResponse* response, Closure* done) { brpc::ClosureGuard done_guard(done); auto status = CreateFunctionInternal(request->fun()); response->set_code(status.code); @@ -5758,8 +5779,8 @@ base::Status TabletImpl::CreateFunctionInternal(const ::openmldb::common::Extern openmldb::schema::SchemaAdapter::ConvertType(fun.arg_type(idx), &data_type); arg_types.emplace_back(data_type); } - auto status = engine_->RegisterExternalFunction(fun.name(), return_type, fun.return_nullable(), - arg_types, fun.arg_nullable(), fun.is_aggregate(), fun.file()); + auto status = engine_->RegisterExternalFunction(fun.name(), return_type, fun.return_nullable(), arg_types, + fun.arg_nullable(), fun.is_aggregate(), fun.file()); if (status.isOK()) { LOG(INFO) << "create function success. name " << fun.name() << " path " << fun.file(); return {}; @@ -5769,7 +5790,7 @@ base::Status TabletImpl::CreateFunctionInternal(const ::openmldb::common::Extern } void TabletImpl::DropFunction(RpcController* controller, const openmldb::api::DropFunctionRequest* request, - openmldb::api::DropFunctionResponse* response, Closure* done) { + openmldb::api::DropFunctionResponse* response, Closure* done) { brpc::ClosureGuard done_guard(done); const auto& fun = request->fun(); std::vector arg_types; @@ -5791,7 +5812,7 @@ void TabletImpl::DropFunction(RpcController* controller, const openmldb::api::Dr } void TabletImpl::CreateAggregator(RpcController* controller, const ::openmldb::api::CreateAggregatorRequest* request, - ::openmldb::api::CreateAggregatorResponse* response, Closure* done) { + ::openmldb::api::CreateAggregatorResponse* response, Closure* done) { brpc::ClosureGuard done_guard(done); std::string msg; @@ -5804,8 +5825,8 @@ void TabletImpl::CreateAggregator(RpcController* controller, const ::openmldb::a if (!ok) { response->set_code(::openmldb::base::ReturnCode::kFailToGetDbRootPath); response->set_msg("fail to get pre-aggr table db root path"); - PDLOG(WARNING, "pre-aggr table db path is not found. tid %u, pid %u", - request->aggr_table_tid(), request->aggr_table_pid()); + PDLOG(WARNING, "pre-aggr table db path is not found. tid %u, pid %u", request->aggr_table_tid(), + request->aggr_table_pid()); return; } std::string aggr_table_path = GetDBPath(aggr_path, request->aggr_table_tid(), request->aggr_table_pid()); @@ -5841,8 +5862,7 @@ void TabletImpl::CreateAggregator(RpcController* controller, const ::openmldb::a return; } -bool TabletImpl::CreateAggregatorInternal(const ::openmldb::api::CreateAggregatorRequest* request, - std::string& msg) { +bool TabletImpl::CreateAggregatorInternal(const ::openmldb::api::CreateAggregatorRequest* request, std::string& msg) { const ::openmldb::api::TableMeta* base_meta = &request->base_table_meta(); std::shared_ptr aggr_table = GetTable(request->aggr_table_tid(), request->aggr_table_pid()); if (!aggr_table) { @@ -5851,11 +5871,9 @@ bool TabletImpl::CreateAggregatorInternal(const ::openmldb::api::CreateAggregato return false; } auto aggr_replicator = GetReplicator(request->aggr_table_tid(), request->aggr_table_pid()); - auto aggregator = ::openmldb::storage::CreateAggregator(*base_meta, *aggr_table->GetTableMeta(), - aggr_table, aggr_replicator, request->index_pos(), - request->aggr_col(), request->aggr_func(), - request->order_by_col(), request->bucket_size(), - request->filter_col()); + auto aggregator = ::openmldb::storage::CreateAggregator( + *base_meta, *aggr_table->GetTableMeta(), aggr_table, aggr_replicator, request->index_pos(), request->aggr_col(), + request->aggr_func(), request->order_by_col(), request->bucket_size(), request->filter_col()); if (!aggregator) { msg.assign("create aggregator failed"); return false; @@ -5865,7 +5883,7 @@ bool TabletImpl::CreateAggregatorInternal(const ::openmldb::api::CreateAggregato if (!aggregator->Init(base_replicator)) { PDLOG(WARNING, "aggregator init failed"); } - uint64_t uid = (uint64_t) base_meta->tid() << 32 | base_meta->pid(); + uint64_t uid = (uint64_t)base_meta->tid() << 32 | base_meta->pid(); { std::lock_guard spin_lock(spin_mutex_); if (aggregators_.find(uid) == aggregators_.end()) { diff --git a/tools/synctool_helper.py b/tools/synctool_helper.py new file mode 100644 index 00000000000..99f66b7e277 --- /dev/null +++ b/tools/synctool_helper.py @@ -0,0 +1,155 @@ +from datetime import datetime +import json +import os +import requests +import configparser as cfg +import argparse + +parser = argparse.ArgumentParser() +parser.add_argument('action', type=str, help='create, delete, status, tool-status') +parser.add_argument('-t', type=str, help='task id(db.table)') +parser.add_argument('-m', '--mode', type=int, help='task mode(0,1,2: FULL, INCREMENTAL_BY_TIMESTAMP, FULL_AND_CONTINUOUS)') +parser.add_argument('-ts', type=int, help='start timestamp if mode is INCREMENTAL_BY_TIMESTAMP') +parser.add_argument('-d', '--dest', type=str, help='sync destination, should be the path, exclude hdfs header, e.g. hdfs://:9000/tmp/a/b/c, should be /tmp/a/b/c') + +parser.add_argument('-s', '--server', type=str, default='127.0.0.1:8848',help='sync tool host:port') +parser.add_argument('-f', '--synctool_conf', type=str, default='conf/synctool.properties', help='synctool config file path, we need it to get the sync tool address and persist path') + +def status_parse(status): + assert status['response']['code'] == 0, status + # todo: check if part tasks are alive, if one unalive task, the whole table sync task is unalive + if not 'readableInfo' in status: + print('no task') + return + size = len(status['readableInfo']) + print(size, 'tasks(tid-pid)') + failed_tids = set() + tid_pid_task = {} + for i in range(size): + info = status['readableInfo'][i] + d = dict([item.split('=') for item in info.split(';')]) + task = status['task'][i] + # print('task ', task) + progress = task.pop('progress') + d.update(progress) + d.update(task) + # print(d) + + # more readable ts + d['lastUpdateTime'] = str(datetime.fromtimestamp(int(d['lastUpdateTime'])/1000)) + if d['status'] == 'FAILED': + failed_tids.add(d['tid']) + if d['tid'] not in tid_pid_task: + tid_pid_task[d['tid']] = {} + tid_pid_task[d['tid']][d['pid']] = d + print(d) + + print('table scope') + print('running tasks:') + for tid, pid_tasks in tid_pid_task.items(): + if tid not in failed_tids: + print(f'tid {tid} task is running, pid parts: {pid_tasks.keys()}') + if failed_tids: + print('tasks contains failed part:', failed_tids) + for tid in failed_tids: + print('tid:', tid) + for p in tid_pid_task[tid].values(): + if p['status'] == 'FAILED': + print('failed part:', p) + else: + print('all tasks are running') + +if __name__ == "__main__": + args = parser.parse_args() + if args.action in ['create', 'delete', 'status']: + sync_tool_url = f'http://{args.server}/openmldb.datasync.SyncTool' + if args.action == 'create': + def create_sync_task(db, table, mode, ts, dest): + create_task = requests.post(f'{sync_tool_url}/CreateSyncTask', json={ + "db": db, "name": table, "mode": mode, "start_ts": ts, "dest": dest}) + print(create_task.text) + task = args.t + assert len(task) > 0, 'task should not be empty' + db, table = task.split('.') + # just use int for mode + mode = args.mode # default is full + ts = 0 # it's safe to set ts starts from 0 when mode is 0 or 2 + if args.mode == 1: + ts = args.ts + assert len(args.dest) > 0, 'dest should not be empty' + dest = args.dest + create_sync_task(db, table, mode, ts, dest) + elif args.action == 'delete': + def delete_sync_task(db, table): + delete_task = requests.post(f'{sync_tool_url}/DeleteSyncTask', json={ + "db": db, "name": table}) + print(delete_task.text) + task = args.t + db, table = task.split('.') + delete_sync_task(db, table) + else: + status = requests.get(f'{sync_tool_url}/TaskStatus', json={}) + status = json.loads(status.text) + status_parse(status) + elif args.action == 'tool-status': + # check sync tool status, needs properties file + config_path = args.synctool_conf + cf = cfg.ConfigParser() + with open(config_path, "r") as f: + config_string = '[dummy_section]\n' + f.read() + cf.read_string(config_string) + cfgs = dict(cf.items('dummy_section')) + print(cfgs) + sync_tool_addr = cfgs['server.host']+ ':' + cfgs['server.port'] + synctool_progress_root = cfgs['sync_task.progress_path'] + data_cache_path = cfgs['data.cache_path'] + sync_tool_url = f'http://{sync_tool_addr}/openmldb.datasync.SyncTool' + + print('sync task progress in sync tool fs') + # List the files in the directory + directory = synctool_progress_root + # Walk the directory tree + for dirpath, dirnames, filenames in os.walk(directory): + # Loop over the files in the current directory + for filename in filenames: + # Construct the full path to the file + filepath = os.path.join(dirpath, filename) + print(filepath) + # Open the file for reading read protobuf message is dangerous + +# about data collector config or cache files, it is not necessary at the moment + +# data cache +# print(""" +# data cache, save the data which data collector send to sync tool +# and sync tool will watch it and sink +# """) +# print('data cache path:', data_cache_path) +# try: +# assert os.path.isdir(data_cache_path) +# cache_status = {} +# for dirpath, dirnames, filenames in os.walk(data_cache_path): +# if not dirnames: +# # the last path +# print(dirpath) +# # tid = dirpath.split('/')[-2] +# # ts = int(dirpath.split('/')[-1]) +# # cache_status[tid] = max(cache_status.get(tid, -1), ts) + +# # print('data latest cache status(including finished/deleted task)') +# # print([(tid, str(datetime.fromtimestamp(ts/1000))) for tid, ts in cache_status.items()]) +# except Exception as e: +# pass + +# data collector status(runtime needs rpc in data collector, file needs run on data collector host) +# try: +# data_collector = '172.24.4.27:8888' +# # no status rpc now +# # data_collector_url = f'http://{data_collector}/DataCollector/' +# # check dir collector_datadir datacollector/- +# collector_datadir = '/home/huangwei/tmp/openmldb/datacollector' +# assert os.path.isdir(collector_datadir) +# except Exception as e: +# pass + +# task progress in data collector no rpc now