You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2022/09/13 08:29:23 UTC

[GitHub] [iotdb] qiaojialin commented on a diff in pull request #7248: [To rel/13] [IOTDB-4332] data backup tool based on iotdb-session

qiaojialin commented on code in PR #7248:
URL: https://github.com/apache/iotdb/pull/7248#discussion_r969285009


##########
docs/zh/UserGuide/Maintenance-Tools/Backup-Tool.md:
##########
@@ -0,0 +1,61 @@
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you 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.
+
+-->
+
+# Backup-Tool
+IOTDB 的数据导入导出工具,导出的数据主要包含时间序列的结构和其对应的数据。

Review Comment:
   When adding new pages, remember to update the site/src/main/.vuepress in master branch.



##########
backup-tool/backup-core/src/main/java/org/apache/iotdb/backup/core/DeleteStarter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iotdb.backup.core;
+
+import org.apache.iotdb.backup.core.pipeline.context.model.DeleteModel;
+import org.apache.iotdb.backup.core.service.ExportPipelineService;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.session.Session;
+import org.apache.iotdb.session.SessionDataSet;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import reactor.core.Disposable;
+import reactor.core.publisher.Flux;
+import reactor.core.scheduler.Scheduler;
+import reactor.core.scheduler.Schedulers;
+
+/** @Author: LL @Description: @Date: create in 2022/7/26 13:51 */
+public class DeleteStarter implements Starter<DeleteModel> {
+
+  private static final Logger log = LoggerFactory.getLogger(DeleteStarter.class);
+
+  @Override
+  public Disposable start(DeleteModel model) {
+    Scheduler scheduler = Schedulers.single();
+    Disposable disposable =
+        Flux.just("")
+            .subscribeOn(scheduler)
+            .flatMap(
+                s -> {
+                  deleteTimeseries(model);
+                  return Flux.just(s);
+                })
+            .doFinally(
+                s -> {
+                  scheduler.dispose();
+                })
+            .subscribe();
+    return disposable;
+  }
+
+  public void deleteTimeseries(DeleteModel model) {
+    String version = getIotdbVersion(model.getSession());
+    if (model.getMeasurementList() != null && model.getMeasurementList().size() != 0) {
+      model
+          .getMeasurementList()
+          .forEach(
+              s -> {
+                StringBuilder sql = new StringBuilder();
+                sql.append("delete from ")
+                    .append(ExportPipelineService.formatPath(model.getIotdbPath(), version))
+                    .append(".")
+                    .append(ExportPipelineService.formatMeasurement(s));
+                if (model.getWhereClause() != null && !"".equals(model.getWhereClause())) {
+                  sql.append(" where ").append(model.getWhereClause());
+                }
+                doExecuteQuery(model.getSession(), sql.toString());
+              });
+    } else {
+      StringBuilder sql = new StringBuilder();
+      sql.append("delete  from ").append(ExportPipelineService.formatPath(model.getIotdbPath()));
+      if (model.getWhereClause() != null && !"".equals(model.getWhereClause())) {
+        sql.append(" where ").append(model.getWhereClause());
+      }
+      doExecuteQuery(model.getSession(), sql.toString());
+    }
+  }
+
+  public void doExecuteQuery(Session session, String sql) {
+    try {
+      session.executeNonQueryStatement(sql);
+    } catch (StatementExecutionException | IoTDBConnectionException e) {
+      log.error("异常SQL:{}\n异常信息:", sql, e);
+    }
+  }
+
+  private String getIotdbVersion(Session session) {
+    try {
+      String versionSql = "show version";
+      SessionDataSet dataSet = session.executeQueryStatement(versionSql);
+      String version = dataSet.next().getFields().get(0).getStringValue();
+      if (version.startsWith("0.13")) {
+        return "13";
+      } else if (version.startsWith("0.12")) {
+        return "12";
+      } else {
+        return version;
+      }
+    } catch (Exception e) {
+      log.error("获取版本异常", e);
+      return null;
+    }
+  }
+
+  @Override
+  public void shutDown() {}
+
+  @Override
+  public Double[] rateOfProcess() {
+    return new Double[0];
+  }
+
+  @Override
+  public Long finishedRowNum() {
+    return null;
+  }
+
+  // public static void main(String[] args) throws IoTDBConnectionException, InterruptedException {

Review Comment:
   remove



##########
backup-tool/pom.xml:
##########
@@ -0,0 +1,188 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you 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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.iotdb</groupId>
+        <artifactId>iotdb-parent</artifactId>
+        <version>0.13.3-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <groupId>org.apache.iotdb</groupId>
+    <artifactId>backup-tool</artifactId>
+    <version>0.13.3-SNAPSHOT</version>
+    <packaging>pom</packaging>
+    <name>Data Backup Tool Base on Session</name>
+    <modules>
+        <module>backup-core</module>
+        <module>backup-command</module>
+    </modules>
+    <properties>
+        <iotdb.version>0.13.0</iotdb.version>
+    </properties>
+    <dependencyManagement>
+        <dependencies>
+            <dependency>
+                <groupId>org.apache.iotdb</groupId>
+                <artifactId>iotdb-session</artifactId>
+                <version>${iotdb.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.iotdb</groupId>
+                <artifactId>iotdb-antlr</artifactId>
+                <version>${iotdb.version}</version>
+            </dependency>
+        </dependencies>
+    </dependencyManagement>
+    <dependencies>
+        <!--        <dependency>-->
+        <!--            <groupId>org.slf4j</groupId>-->

Review Comment:
   remove useless code



##########
backup-tool/backup-command/src/main/java/org/apache/iotdb/backup/command/ExportTool13_1.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.iotdb.backup.command;
+
+import org.apache.iotdb.backup.command.Exception.ArgsErrorException;
+import org.apache.iotdb.backup.command.utils.AbstractCsvTool;
+import org.apache.iotdb.backup.core.ExportStarter;
+import org.apache.iotdb.backup.core.exception.ParamCheckException;
+import org.apache.iotdb.backup.core.pipeline.context.model.CompressEnum;
+import org.apache.iotdb.backup.core.pipeline.context.model.ExportModel;
+import org.apache.iotdb.backup.core.pipeline.context.model.FileSinkStrategyEnum;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.StatementExecutionException;
+
+import org.apache.commons.cli.*;
+import reactor.core.Disposable;
+
+import java.io.File;
+
+/** @Author: LL @Description: @Date: create in 2022/7/7 14:29 */
+public class ExportTool13_1 extends AbstractCsvTool {

Review Comment:
   what is 13_1, 0.13.1?
   No need to show these in class name, the minor version do not change the TsFile and IoTDB structure/interface.
   So only put these code in rel/0.13 is enough.



##########
backup-tool/readme_zh.md:
##########
@@ -0,0 +1,61 @@
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you 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.
+
+-->
+
+# Backup-Tool
+IOTDB 的数据导入导出工具,导出的数据主要包含时间序列的结构和其对应的数据。
+
+## 项目结构
+本项目分为两个子项目:backup-core和backup-command项目;  
+backup-core项目包含主要的业务逻辑和实现方法,该项目可以以jar包的形式对外提供服务。  
+backup-command项目是一个工具类包,功能实现依赖于核心项目,提供命令行工具。  
+
+## 编译
+你可以使用以下命令来编译或者打包这个项目:  
+````
+mvn clean package;  
+mvn install;  
+mvn test;   
+````
+ps: 使用 '-DskipTests' 来跳过测试用例    
+由于core-jar还没有放到远程仓库,为了顺利编译工具,首先需要执行mvn install -DskipTests。  
+集成测试需要一个真正的iotdbserver,你可以在“test/resource/sesseinConfig.properties”中配置服务器.  
+
+## 特性    
+- 基于iotdb-session   
+- 基于pipeline的导入导出工具  
+- 基于project reactor框架  
+- 支持很多情形的导入导出,比如导入导出一个存储组,导入导出对应的时间序列结构,包含设备是否对齐的信息等等  
+
+## 使用
+````
+-  backup-export.bat -h 127.0.0.1 -p 6667 -u root -pw root -f d:\\validate_test\\83 -i root.** -sy true -se true -c gzip  
+-  backup-import.1.bat -h 127.0.0.1 -p 6667 -u root -pw root -f D:\validate_test\83 -se true -c gzip
+- -h  // iotdb host address
+- -p  // port
+- -u  // username
+- -pw // password
+- -f  // fileFolder
+- -sy // 是否需要时间序列结构,如果是true的话,工具会额外导出一个文件来记录它对应的时间序列结构
+- -se // 文件生成策略,1.以entity的路径为为文件名生成文件 2、额外提供一个文件来记录文件名与entity路径的对应关系

Review Comment:
   what is entity?



##########
docs/zh/UserGuide/Maintenance-Tools/Backup-Tool.md:
##########
@@ -0,0 +1,61 @@
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you 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.
+
+-->
+
+# Backup-Tool
+IOTDB 的数据导入导出工具,导出的数据主要包含时间序列的结构和其对应的数据。
+
+## 项目结构
+本项目分为两个子项目:backup-core和backup-command项目;  
+backup-core项目包含主要的业务逻辑和实现方法,该项目可以以jar包的形式对外提供服务。  
+backup-command项目是一个工具类包,功能实现依赖于核心项目,提供命令行工具。  
+
+## 编译
+你可以使用以下命令来编译或者打包这个项目:  
+````
+mvn clean package;  
+mvn install;  
+mvn test;   
+````
+ps: 使用 '-DskipTests' 来跳过测试用例    
+由于core-jar还没有放到远程仓库,为了顺利编译工具,首先需要执行mvn install -DskipTests。  
+集成测试需要一个真正的iotdbserver,你可以在“test/resource/sesseinConfig.properties”中配置服务器.  
+
+## 特性    
+- 基于iotdb-session   
+- 基于pipeline的导入导出工具  
+- 基于react框架  
+- 支持很多情形的导入导出,比如导入导出一个存储组,导入导出对应的时间序列结构,包含设备是否对齐的信息等等  
+
+## 使用
+````
+-  backup-export.bat -h 127.0.0.1 -p 6667 -u root -pw root -f d:\\validate_test\\83 -i root.** -sy true -se true -c gzip  
+-  backup-import.1.bat -h 127.0.0.1 -p 6667 -u root -pw root -f D:\validate_test\83 -se true -c gzip
+- -h  // iotdb host address
+- -p  // port
+- -u  // username
+- -pw // password
+- -f  // fileFolder
+- -sy // 是否需要时间序列结构,如果是true的话,工具会额外导出一个文件来记录它对应的时间序列结构
+- -se // 文件生成策略,1.以entity的路径为为文件名生成文件 2、额外提供一个文件来记录文件名与entity路径的对应关系

Review Comment:
   These parameters, which are used in import and which are used in export?



##########
docs/zh/UserGuide/Maintenance-Tools/Backup-Tool.md:
##########
@@ -0,0 +1,61 @@
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you 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.
+
+-->
+
+# Backup-Tool
+IOTDB 的数据导入导出工具,导出的数据主要包含时间序列的结构和其对应的数据。
+
+## 项目结构
+本项目分为两个子项目:backup-core和backup-command项目;  
+backup-core项目包含主要的业务逻辑和实现方法,该项目可以以jar包的形式对外提供服务。  
+backup-command项目是一个工具类包,功能实现依赖于核心项目,提供命令行工具。  
+
+## 编译
+你可以使用以下命令来编译或者打包这个项目:  
+````
+mvn clean package;  
+mvn install;  
+mvn test;   
+````
+ps: 使用 '-DskipTests' 来跳过测试用例    
+由于core-jar还没有放到远程仓库,为了顺利编译工具,首先需要执行mvn install -DskipTests。  
+集成测试需要一个真正的iotdbserver,你可以在“test/resource/sesseinConfig.properties”中配置服务器.  
+
+## 特性    
+- 基于iotdb-session   
+- 基于pipeline的导入导出工具  
+- 基于react框架  
+- 支持很多情形的导入导出,比如导入导出一个存储组,导入导出对应的时间序列结构,包含设备是否对齐的信息等等  
+
+## 使用
+````
+-  backup-export.bat -h 127.0.0.1 -p 6667 -u root -pw root -f d:\\validate_test\\83 -i root.** -sy true -se true -c gzip  
+-  backup-import.1.bat -h 127.0.0.1 -p 6667 -u root -pw root -f D:\validate_test\83 -se true -c gzip
+- -h  // iotdb host address
+- -p  // port
+- -u  // username
+- -pw // password
+- -f  // fileFolder
+- -sy // 是否需要时间序列结构,如果是true的话,工具会额外导出一个文件来记录它对应的时间序列结构
+- -se // 文件生成策略,1.以entity的路径为为文件名生成文件 2、额外提供一个文件来记录文件名与entity路径的对应关系
+- -c  // 压缩格式 : SQL、CSV、SNAPPY、GZIP、LZ4

Review Comment:
   why we combine the SQL, CSV with SNAPPY?
   SQL and CSV is the data structure, SNAPPY is a compresson method, are they in the same level?



##########
backup-tool/readme_zh.md:
##########
@@ -0,0 +1,61 @@
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you 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.
+
+-->
+
+# Backup-Tool
+IOTDB 的数据导入导出工具,导出的数据主要包含时间序列的结构和其对应的数据。
+
+## 项目结构
+本项目分为两个子项目:backup-core和backup-command项目;  
+backup-core项目包含主要的业务逻辑和实现方法,该项目可以以jar包的形式对外提供服务。  
+backup-command项目是一个工具类包,功能实现依赖于核心项目,提供命令行工具。  
+
+## 编译
+你可以使用以下命令来编译或者打包这个项目:  
+````
+mvn clean package;  
+mvn install;  
+mvn test;   
+````
+ps: 使用 '-DskipTests' 来跳过测试用例    
+由于core-jar还没有放到远程仓库,为了顺利编译工具,首先需要执行mvn install -DskipTests。  
+集成测试需要一个真正的iotdbserver,你可以在“test/resource/sesseinConfig.properties”中配置服务器.  
+
+## 特性    
+- 基于iotdb-session   
+- 基于pipeline的导入导出工具  
+- 基于project reactor框架  
+- 支持很多情形的导入导出,比如导入导出一个存储组,导入导出对应的时间序列结构,包含设备是否对齐的信息等等  
+
+## 使用
+````
+-  backup-export.bat -h 127.0.0.1 -p 6667 -u root -pw root -f d:\\validate_test\\83 -i root.** -sy true -se true -c gzip  
+-  backup-import.1.bat -h 127.0.0.1 -p 6667 -u root -pw root -f D:\validate_test\83 -se true -c gzip
+- -h  // iotdb host address
+- -p  // port
+- -u  // username
+- -pw // password
+- -f  // fileFolder
+- -sy // 是否需要时间序列结构,如果是true的话,工具会额外导出一个文件来记录它对应的时间序列结构
+- -se // 文件生成策略,1.以entity的路径为为文件名生成文件 2、额外提供一个文件来记录文件名与entity路径的对应关系
+- -c  // 压缩格式 : SQL、CSV、SNAPPY、GZIP、LZ4
+- -w  // where  条件
+- -i  // 要导出的路径
+````

Review Comment:
   Give some example



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org