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 2021/09/24 07:26:45 UTC

[GitHub] [iotdb] choubenson opened a new pull request #4024: [IOTDB-1635] settle TsFiles and mods

choubenson opened a new pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024


   Complete the settle tool, including online settleTool and offline settleTool. Also modify the online upgrade tool : remove the deleted data when rewriting tsfiles.


-- 
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



[GitHub] [iotdb] HTHou commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r719010897



##########
File path: server/src/test/java/org/apache/iotdb/db/integration/IoTDBSettleIT.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.db.integration;
+
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+public class IoTDBSettleIT {
+  private static List<String> sqls = new ArrayList<>();
+  private static Connection connection;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    EnvironmentUtils.closeStatMonitor();
+    initCreateSQLStatement();
+    EnvironmentUtils.envSetUp();
+    executeSql();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    close();
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void onlineSettleTest() {
+    PartialPath sg = null;
+    try {
+      sg = new PartialPath("root.st1");
+      StorageEngine.getInstance().settleAll(sg);
+    } catch (IllegalPathException | StorageEngineException | WriteProcessException e) {
+      Assert.fail(e.getMessage());
+    }
+  }
+
+  private static void close() throws SQLException {
+    if (Objects.nonNull(connection)) {
+      try {
+        connection.close();
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }
+  }
+
+  private static void initCreateSQLStatement() {
+    sqls.add("SET STORAGE GROUP TO root.st1");
+    sqls.add("CREATE TIMESERIES root.st1.wf01.wt01.status WITH DATATYPE=BOOLEAN, ENCODING=PLAIN");
+    for (int i = 1; i <= 10; i++) {
+      sqls.add("insert into root.st1.wf01.wt01(timestamp,status) values(" + 100 * i + ",false)");
+    }
+    sqls.add("flush");
+    sqls.add("delete from root.st1.wf01.wt01.* where time<500");
+  }
+
+  private static void executeSql() throws ClassNotFoundException, SQLException {
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    connection =
+        DriverManager.getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+    Statement statement = connection.createStatement();
+
+    for (String sql : sqls) {
+      statement.execute(sql);
+    }
+
+    statement.close();

Review comment:
       Use try with resource to close connection and statement automatically.




-- 
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



[GitHub] [iotdb] HTHou commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r715417144



##########
File path: docs/zh/SystemDesign/Tools/Settle.md
##########
@@ -0,0 +1,96 @@
+## TsFile整理工具
+
+### 介绍
+

Review comment:
       Same




-- 
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



[GitHub] [iotdb] choubenson commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
choubenson commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r720835994



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -171,18 +178,21 @@
   private final TreeMap<Long, TsFileProcessor> workSequenceTsFileProcessors = new TreeMap<>();
   /** time partition id in the storage group -> tsFileProcessor for this time partition */
   private final TreeMap<Long, TsFileProcessor> workUnsequenceTsFileProcessors = new TreeMap<>();
+
+  private final Deque<ByteBuffer> walByteBufferPool = new LinkedList<>();
   /** compactionMergeWorking is used to wait for last compaction to be done. */
   private volatile boolean compactionMergeWorking = false;
   // upgrading sequence TsFile resource list
   private List<TsFileResource> upgradeSeqFileList = new LinkedList<>();
-
   /** sequence tsfile processors which are closing */
   private CopyOnReadLinkedList<TsFileProcessor> closingSequenceTsFileProcessor =
       new CopyOnReadLinkedList<>();
-
   // upgrading unsequence TsFile resource list
   private List<TsFileResource> upgradeUnseqFileList = new LinkedList<>();
 
+  private Map<String, TsFileResource> settleSeqFileList = new HashMap<>();
+  private Map<String, TsFileResource> settleUnseqFileList = new HashMap<>();

Review comment:
       It's to estimate whether the resource to be settled has been added in the settleSeqFileMap.




-- 
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



[GitHub] [iotdb] HTHou commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r715416910



##########
File path: docs/UserGuide/System-Tools/SettleTool.md
##########
@@ -0,0 +1,60 @@
+
+
+## TsFile Settle Tool
+
+### Introduction

Review comment:
       Add the Apache license header?




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43209471/badge)](https://coveralls.io/builds/43209471)
   
   Coverage increased (+0.01%) to 67.765% when pulling **efb4db115aa18a599c4089ff52a530fd50b8122a on choubenson:settleTsFileAndMod** into **4b34a9cd298217917711a28bac038ebd894d4dee on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43386891/badge)](https://coveralls.io/builds/43386891)
   
   Coverage decreased (-0.02%) to 67.807% when pulling **4a0557ae1f92484b4430429f74c6edea151181c0 on choubenson:settleTsFileAndMod** into **2cb933600f3ad944850c7dbebb7ee9e22a6ccea7 on apache:master**.
   


-- 
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



[GitHub] [iotdb] HTHou commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r718999533



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1874,7 +1924,6 @@ public void writeUnlock() {
    * @param path the timeseries path of the to be deleted.
    * @param startTime the startTime of delete range.
    * @param endTime the endTime of delete range.
-   * @param timePartitionFilter

Review comment:
       Why remove this? I see there is the timePartitionFilter...




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43147925/badge)](https://coveralls.io/builds/43147925)
   
   Coverage increased (+0.2%) to 67.754% when pulling **08fc753f765b467a1b57d559983b2a13e2830dab on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43398876/badge)](https://coveralls.io/builds/43398876)
   
   Coverage increased (+0.01%) to 67.808% when pulling **a97fff6f797a2106c147d7d5375263379af1ab81 on choubenson:settleTsFileAndMod** into **c1e43e25267a63cf087f07bfc203b4e955198ecd on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43140435/badge)](https://coveralls.io/builds/43140435)
   
   Coverage increased (+0.2%) to 67.747% when pulling **38a954df12e2dfec140108a7a1aca9400ae63575 on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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



[GitHub] [iotdb] choubenson commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
choubenson commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r720833973



##########
File path: docs/zh/SystemDesign/Tools/Settle.md
##########
@@ -0,0 +1,119 @@
+<!--
+
+```
+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.
+```
+
+-->
+
+## TsFile整理工具
+
+### 介绍
+
+1. 整理工具可以帮助你整理已封口的tsFile文件和.mods文件,过滤掉删除的数据并生成新的tsFile。整理完后会将本地旧的.mods文件删除,若某tsFile里的数据被全部删除了,则会删除本地该tsFile文件和对应的.resource文件。
+
+2. 整理工具只能针对v0.12版本的IOTDB来使用,即只能针对版本为V3的tsFile进行整理,若低于此版本则必须先用在线升级工具将tsFile升级到V3版本。
+3. tsFile整理工具分为在线整理工具和离线整理工具,两者在整理的时候都会记录日志,供下次恢复先前整理失败的文件。在执行整理的时候会先去检测日志里是否存在失败的文件,有的话则优先整理它们。
+
+### 1. 离线整理工具
+
+离线整理工具是以命令行的方式来启动,该工具的启动脚本settle.bat和settle.sh在编译了server后会生成至server\target\iotdb-server-{version}\tools\tsFileToolSet目录中。在使用的时候必须保证IOTDB Server是停止运行的,否则会出现整理错误。具体使用方式详见用户手册。
+
+#### 1.1 涉及的相关类
+
+整理工具类:org.apache.iotdb.db.tools.settle.tsFileAndModSettleTool
+
+### 2.在线整理工具
+
+在线整理工具是当用户在IOTDB客户端输入了settle命令后,会在后台注册启动一个整理服务Settle Service,该服务会去寻找指定存储组下的所有tsFile文件,并为每个tsFile开启一个整理线程进行整理。下面讲解整理线程的工作流程。

Review comment:
       There is one parameter to control the size of settleThreadPool, which value is 1.




-- 
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



[GitHub] [iotdb] HTHou commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r715416723



##########
File path: docs/SystemDesign/Tools/Settle.md
##########
@@ -0,0 +1,95 @@
+## TsFile Settle Tool
+
+### Introduction
+

Review comment:
       Add the Apache license header?




-- 
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



[GitHub] [iotdb] HTHou commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r719011485



##########
File path: server/src/test/java/org/apache/iotdb/db/integration/IoTDBSettleIT.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.db.integration;
+
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+public class IoTDBSettleIT {
+  private static List<String> sqls = new ArrayList<>();
+  private static Connection connection;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    EnvironmentUtils.closeStatMonitor();
+    initCreateSQLStatement();
+    EnvironmentUtils.envSetUp();
+    executeSql();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    close();
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void onlineSettleTest() {
+    PartialPath sg = null;
+    try {
+      sg = new PartialPath("root.st1");
+      StorageEngine.getInstance().settleAll(sg);
+    } catch (IllegalPathException | StorageEngineException | WriteProcessException e) {
+      Assert.fail(e.getMessage());
+    }
+  }

Review comment:
       As an IT, we should execute sql `settle root.st1` here to test the feature...




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43130268/badge)](https://coveralls.io/builds/43130268)
   
   Coverage decreased (-0.1%) to 67.446% when pulling **2acb8c2a6c6adc84cff3d9b8f877c908c57ffdcf on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43130268/badge)](https://coveralls.io/builds/43130268)
   
   Coverage decreased (-0.1%) to 67.446% when pulling **2acb8c2a6c6adc84cff3d9b8f877c908c57ffdcf on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43148223/badge)](https://coveralls.io/builds/43148223)
   
   Coverage increased (+0.2%) to 67.735% when pulling **08fc753f765b467a1b57d559983b2a13e2830dab on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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



[GitHub] [iotdb] choubenson commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
choubenson commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r720834694



##########
File path: server/src/main/java/org/apache/iotdb/db/service/SettleService.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.db.service;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.settle.SettleLog;
+import org.apache.iotdb.db.engine.settle.SettleTask;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.tools.settle.TsFileAndModSettleTool;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class SettleService implements IService {

Review comment:
       The serivceThread api is implemented to consider that it will run in the backgroud periodically in the future. But currently the function of settle command is executed in the main thread.




-- 
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



[GitHub] [iotdb] HTHou commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r718989905



##########
File path: codecov.yml
##########
@@ -28,22 +28,22 @@ coverage:
     changes: no
 
 ignore:
-  #  - "*/interface/thrift/**/*.java"  # ignore thrift-gen folders and all its contents
-  - "**/generated-classes/**/*.java"
-  - "**/generated-sources/**/*.java"
-  - "grafana/**/*.java"
-  - "spark/**/*.java"
-  - "hadoop/**/*.java"
-  - "**/pom.xml"
-  - "**/*.md"
-  - "**/*.sh"
-  - "**/*.cmd"
-  - "**/*.bat"
-  - "**/*.yml"
-  - ".mvn"
-  - "mvnw"
-  - "NOTICE"
-  - "NOTICE-binary"
-  - "License"
-  - "LICENSE-binary"
-  - "DISCLAIMER"
+#  - "*/interface/thrift/**/*.java"  # ignore thrift-gen folders and all its contents
+- "**/generated-classes/**/*.java"
+- "**/generated-sources/**/*.java"
+- "grafana/**/*.java"
+- "spark/**/*.java"
+- "hadoop/**/*.java"
+- "**/pom.xml"
+- "**/*.md"
+- "**/*.sh"
+- "**/*.cmd"
+- "**/*.bat"
+- "**/*.yml"
+- ".mvn"
+- "mvnw"
+- "NOTICE"
+- "NOTICE-binary"
+- "License"
+- "LICENSE-binary"
+- "DISCLAIMER"

Review comment:
       I'm not sure whether the codecov will work if you change the format of this file. Better to keep the original format....




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43066043/badge)](https://coveralls.io/builds/43066043)
   
   Coverage decreased (-0.005%) to 67.551% when pulling **850929eeb22b789e3bc334debf42d2e3f3cc1f36 on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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



[GitHub] [iotdb] choubenson commented on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
choubenson commented on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-927614296


   Hi,i've writen the documents in [wiki](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=191332375) 


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43045139/badge)](https://coveralls.io/builds/43045139)
   
   Coverage decreased (-0.01%) to 67.543% when pulling **1a1bfc0f036020ed8cce0a0e9a72e75133e10b20 on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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



[GitHub] [iotdb] wangchao316 commented on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
wangchao316 commented on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926486252


   hi , I feel you should write a Design Documents in wiki.  
   So  can review your code better.


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43141027/badge)](https://coveralls.io/builds/43141027)
   
   Coverage increased (+0.2%) to 67.75% when pulling **84b25bd9133bd1d49f2837ed6224c905f1bac53c on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls commented on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls commented on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43044100/badge)](https://coveralls.io/builds/43044100)
   
   Coverage decreased (-0.01%) to 67.546% when pulling **c0bedd2565679ef905ae433c7c3931bf02fcb481 on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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



[GitHub] [iotdb] choubenson commented on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
choubenson commented on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926474197


   First time to contribute for IOTDB,there may be many problems, including code format and specification, etc. Welcome to give me 
    correction !


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43239324/badge)](https://coveralls.io/builds/43239324)
   
   Coverage increased (+0.02%) to 67.802% when pulling **39a5565ce43a119e508ff4065e7e02c443eb099c on choubenson:settleTsFileAndMod** into **cbbdc6caf51660e5817a4e9c854831d820315b72 on apache:master**.
   


-- 
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



[GitHub] [iotdb] JackieTien97 merged pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
JackieTien97 merged pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024


   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43141562/badge)](https://coveralls.io/builds/43141562)
   
   Coverage increased (+0.2%) to 67.745% when pulling **27c0584d69af1efbfd3f2c1abf8f5a25871f19dc on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43045861/badge)](https://coveralls.io/builds/43045861)
   
   Coverage increased (+0.04%) to 67.592% when pulling **dc5620dba0cd5648e692c36d2cae9a01de4d8bd0 on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43141668/badge)](https://coveralls.io/builds/43141668)
   
   Coverage increased (+0.2%) to 67.745% when pulling **27c0584d69af1efbfd3f2c1abf8f5a25871f19dc on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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



[GitHub] [iotdb] HTHou commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r718990595



##########
File path: docs/SystemDesign/Tools/Settle.md
##########
@@ -0,0 +1,118 @@
+<!--
+
+```
+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.
+```
+
+-->
+
+## TsFile Settle Tool
+
+### 1. Introduction
+

Review comment:
       How about move this user guide to docs/UserGuide/System-Tools ?




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982






-- 
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



[GitHub] [iotdb] HTHou commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r718991048



##########
File path: docs/UserGuide/System-Tools/SettleTool.md
##########
@@ -0,0 +1,81 @@
+<!--
+
+```
+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.
+```

Review comment:
       Don't use ``` here...




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43070788/badge)](https://coveralls.io/builds/43070788)
   
   Coverage decreased (-0.01%) to 67.545% when pulling **ef74f23b3704d7b3f025fc7460c39d288934715a on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43300449/badge)](https://coveralls.io/builds/43300449)
   
   Coverage increased (+0.03%) to 67.813% when pulling **b5ffda7b684aaa6fb11e25a700df6779695e5e0d on choubenson:settleTsFileAndMod** into **cbbdc6caf51660e5817a4e9c854831d820315b72 on apache:master**.
   


-- 
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



[GitHub] [iotdb] JackieTien97 commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r719002457



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/settle/SettleLog.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.db.engine.settle;
+
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class SettleLog {
+  private static final Logger logger = LoggerFactory.getLogger(SettleLog.class);
+  public static final String COMMA_SEPERATOR = ",";
+  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private static final String SETTLE_DIR = "settle";
+  private static final String SETTLE_LOG_NAME = "settle.txt";
+  private static BufferedWriter settleLogWriter;
+  private static File settleLogPath = // the path of upgrade log is "data/system/settle/settle.txt"
+      SystemFileFactory.INSTANCE.getFile(
+          SystemFileFactory.INSTANCE.getFile(config.getSystemDir(), SETTLE_DIR), SETTLE_LOG_NAME);
+
+  private static final ReadWriteLock settleLogFileLock = new ReentrantReadWriteLock();
+
+  public static boolean createSettleLog() {
+    try {
+      if (!settleLogPath.getParentFile().exists()) {
+        settleLogPath.getParentFile().mkdirs();
+      }
+      settleLogPath.createNewFile();
+      settleLogWriter = new BufferedWriter(new FileWriter(getSettleLogPath(), true));
+      return true;
+    } catch (IOException e) {
+      logger.error("meet error when creating settle log, file path:{}", settleLogPath, e);
+      return false;
+    }
+  }
+
+  public static boolean writeSettleLog(String content) {
+    settleLogFileLock.writeLock().lock();
+    try {
+      settleLogWriter.write(content); // Todo:bug,settleLogWriter null
+      settleLogWriter.newLine();
+      settleLogWriter.flush();
+      return true;
+    } catch (IOException e) {
+      logger.error("write settle log file failed, the log file:{}", getSettleLogPath(), e);
+      return false;
+    } finally {
+      settleLogFileLock.writeLock().unlock();
+    }
+  }
+
+  public static void closeLogWriter() {
+    try {
+      if (settleLogWriter != null) {
+        settleLogWriter.close();
+      }
+    } catch (IOException e) {
+      logger.error("close upgrade log file failed, the log file:{}", getSettleLogPath(), e);
+    }
+  }
+
+  public static String getSettleLogPath() { // "data/system/settle/settle.txt"
+    return settleLogPath.getAbsolutePath();
+  }
+
+  public static void setSettleLogPath(File settleLogPath) {
+    SettleLog.settleLogPath = settleLogPath;
+  }
+
+  public static enum SettleCheckStatus {

Review comment:
       ```suggestion
     public enum SettleCheckStatus {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/settle/SettleLog.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.db.engine.settle;
+
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class SettleLog {
+  private static final Logger logger = LoggerFactory.getLogger(SettleLog.class);
+  public static final String COMMA_SEPERATOR = ",";
+  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+  private static final String SETTLE_DIR = "settle";
+  private static final String SETTLE_LOG_NAME = "settle.txt";
+  private static BufferedWriter settleLogWriter;
+  private static File settleLogPath = // the path of upgrade log is "data/system/settle/settle.txt"
+      SystemFileFactory.INSTANCE.getFile(
+          SystemFileFactory.INSTANCE.getFile(config.getSystemDir(), SETTLE_DIR), SETTLE_LOG_NAME);
+
+  private static final ReadWriteLock settleLogFileLock = new ReentrantReadWriteLock();
+
+  public static boolean createSettleLog() {
+    try {
+      if (!settleLogPath.getParentFile().exists()) {
+        settleLogPath.getParentFile().mkdirs();
+      }
+      settleLogPath.createNewFile();
+      settleLogWriter = new BufferedWriter(new FileWriter(getSettleLogPath(), true));
+      return true;
+    } catch (IOException e) {
+      logger.error("meet error when creating settle log, file path:{}", settleLogPath, e);
+      return false;
+    }
+  }
+
+  public static boolean writeSettleLog(String content) {
+    settleLogFileLock.writeLock().lock();
+    try {
+      settleLogWriter.write(content); // Todo:bug,settleLogWriter null

Review comment:
       Has the TODO been fixed?

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.db.tools.settle;
+
+import org.apache.iotdb.db.engine.settle.SettleLog;
+import org.apache.iotdb.db.engine.settle.SettleLog.SettleCheckStatus;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+/**
+ * Offline Settle tool, which is used to settle TsFile and its corresponding mods file to a new
+ * TsFile.
+ */
+public class TsFileAndModSettleTool {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileAndModSettleTool.class);
+  // TsFilePath -> SettleCheckStatus
+  public static Map<String, Integer> recoverSettleFileMap = new HashMap<>();
+
+  public static void main(String[] args) {
+    Map<String, TsFileResource> oldTsFileResources = new HashMap<>();
+    // List<TsFileResource> oldTsFileResources = new ArrayList<>();
+    findFilesToBeRecovered();
+    for (Map.Entry<String, Integer> entry : recoverSettleFileMap.entrySet()) {
+      String path = entry.getKey();
+      TsFileResource resource = new TsFileResource(new File(path));
+      resource.setClosed(true);
+      oldTsFileResources.put(resource.getTsFile().getName(), resource);
+    }
+    List<File> tsFiles = checkArgs(args);
+    if (tsFiles != null) {
+      for (File file : tsFiles) {
+        if (!oldTsFileResources.containsKey(file.getName())) {
+          if (new File(file + TsFileResource.RESOURCE_SUFFIX).exists()) {
+            TsFileResource resource = new TsFileResource(file);
+            resource.setClosed(true);
+            oldTsFileResources.put(file.getName(), resource);
+          }
+        }
+      }
+    }
+    System.out.println(
+        "Totally find "
+            + oldTsFileResources.size()
+            + " tsFiles to be settled, including "
+            + recoverSettleFileMap.size()
+            + " tsFiles to be recovered.");
+    settleTsFilesAndMods(oldTsFileResources);
+  }
+
+  public static List<File> checkArgs(String[] args) {
+    String filePath = "test.tsfile";
+    List<File> files = new ArrayList<>();
+    if (args.length == 0) {
+      return null;
+    } else {
+      for (String arg : args) {
+        if (arg.endsWith(TSFILE_SUFFIX)) { // it's a file
+          File f = new File(arg);
+          if (!f.exists()) {
+            logger.warn("Cannot find TsFile : " + arg);
+            continue;
+          }
+          files.add(f);
+        } else { // it's a dir
+          List<File> tmpFiles = getAllFilesInOneDirBySuffix(arg, TSFILE_SUFFIX);
+          if (tmpFiles == null) {
+            continue;
+          }
+          files.addAll(tmpFiles);
+        }
+      }
+    }
+    return files;
+  }
+
+  private static List<File> getAllFilesInOneDirBySuffix(String dirPath, String suffix) {
+    File dir = new File(dirPath);
+    if (!dir.isDirectory()) {
+      logger.warn("It's not a directory path : " + dirPath);
+      return null;
+    }
+    if (!dir.exists()) {
+      logger.warn("Cannot find Directory : " + dirPath);
+      return null;
+    }
+    List<File> tsFiles = new ArrayList<>();
+    tsFiles.addAll(
+        Arrays.asList(FSFactoryProducer.getFSFactory().listFilesBySuffix(dirPath, suffix)));
+    List<File> tmpFiles = Arrays.asList(dir.listFiles());
+    for (File f : tmpFiles) {
+      if (f.isDirectory()) {
+        tsFiles.addAll(getAllFilesInOneDirBySuffix(f.getAbsolutePath(), suffix));
+      }
+    }
+    return tsFiles;
+  }
+
+  /**
+   * This method is used to settle tsFiles and mods files, so that each old TsFile corresponds to
+   * one or several new TsFiles. This method is only applicable to V3 TsFile.
+   *
+   * @return Each old TsFile corresponds to one or several new TsFileResources of the new TsFiles
+   */
+  public static Map<String, List<TsFileResource>> settleTsFilesAndMods(

Review comment:
       Return value of the method is never used

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.db.tools.settle;
+
+import org.apache.iotdb.db.engine.settle.SettleLog;
+import org.apache.iotdb.db.engine.settle.SettleLog.SettleCheckStatus;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+/**
+ * Offline Settle tool, which is used to settle TsFile and its corresponding mods file to a new
+ * TsFile.
+ */
+public class TsFileAndModSettleTool {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileAndModSettleTool.class);
+  // TsFilePath -> SettleCheckStatus
+  public static Map<String, Integer> recoverSettleFileMap = new HashMap<>();
+
+  public static void main(String[] args) {
+    Map<String, TsFileResource> oldTsFileResources = new HashMap<>();
+    // List<TsFileResource> oldTsFileResources = new ArrayList<>();
+    findFilesToBeRecovered();
+    for (Map.Entry<String, Integer> entry : recoverSettleFileMap.entrySet()) {
+      String path = entry.getKey();
+      TsFileResource resource = new TsFileResource(new File(path));
+      resource.setClosed(true);
+      oldTsFileResources.put(resource.getTsFile().getName(), resource);
+    }
+    List<File> tsFiles = checkArgs(args);
+    if (tsFiles != null) {
+      for (File file : tsFiles) {
+        if (!oldTsFileResources.containsKey(file.getName())) {
+          if (new File(file + TsFileResource.RESOURCE_SUFFIX).exists()) {
+            TsFileResource resource = new TsFileResource(file);
+            resource.setClosed(true);
+            oldTsFileResources.put(file.getName(), resource);
+          }
+        }
+      }
+    }
+    System.out.println(
+        "Totally find "
+            + oldTsFileResources.size()
+            + " tsFiles to be settled, including "
+            + recoverSettleFileMap.size()
+            + " tsFiles to be recovered.");
+    settleTsFilesAndMods(oldTsFileResources);
+  }
+
+  public static List<File> checkArgs(String[] args) {
+    String filePath = "test.tsfile";
+    List<File> files = new ArrayList<>();
+    if (args.length == 0) {
+      return null;
+    } else {
+      for (String arg : args) {
+        if (arg.endsWith(TSFILE_SUFFIX)) { // it's a file
+          File f = new File(arg);
+          if (!f.exists()) {
+            logger.warn("Cannot find TsFile : " + arg);
+            continue;
+          }
+          files.add(f);
+        } else { // it's a dir
+          List<File> tmpFiles = getAllFilesInOneDirBySuffix(arg, TSFILE_SUFFIX);
+          if (tmpFiles == null) {
+            continue;
+          }
+          files.addAll(tmpFiles);
+        }
+      }
+    }
+    return files;
+  }
+
+  private static List<File> getAllFilesInOneDirBySuffix(String dirPath, String suffix) {
+    File dir = new File(dirPath);
+    if (!dir.isDirectory()) {
+      logger.warn("It's not a directory path : " + dirPath);
+      return null;
+    }
+    if (!dir.exists()) {
+      logger.warn("Cannot find Directory : " + dirPath);
+      return null;
+    }
+    List<File> tsFiles = new ArrayList<>();
+    tsFiles.addAll(
+        Arrays.asList(FSFactoryProducer.getFSFactory().listFilesBySuffix(dirPath, suffix)));
+    List<File> tmpFiles = Arrays.asList(dir.listFiles());
+    for (File f : tmpFiles) {
+      if (f.isDirectory()) {
+        tsFiles.addAll(getAllFilesInOneDirBySuffix(f.getAbsolutePath(), suffix));
+      }
+    }
+    return tsFiles;
+  }
+
+  /**
+   * This method is used to settle tsFiles and mods files, so that each old TsFile corresponds to
+   * one or several new TsFiles. This method is only applicable to V3 TsFile.
+   *
+   * @return Each old TsFile corresponds to one or several new TsFileResources of the new TsFiles
+   */
+  public static Map<String, List<TsFileResource>> settleTsFilesAndMods(
+      Map<String, TsFileResource> resourcesToBeSettled) {
+    int successCount = 0;
+    Map<String, List<TsFileResource>> newTsFileResources = new HashMap<>();
+    SettleLog.createSettleLog();
+    for (Map.Entry<String, TsFileResource> entry : resourcesToBeSettled.entrySet()) {
+      TsFileResource resourceToBeSettled = entry.getValue();
+      List<TsFileResource> settledTsFileResources = new ArrayList<>();
+      try {
+        TsFileAndModSettleTool tsFileAndModSettleTool = new TsFileAndModSettleTool();

Review comment:
       You can change the TsFileAndModSettleTool as a singleton

##########
File path: server/src/main/java/org/apache/iotdb/db/service/SettleService.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.db.service;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.settle.SettleLog;
+import org.apache.iotdb.db.engine.settle.SettleTask;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.tools.settle.TsFileAndModSettleTool;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class SettleService implements IService {

Review comment:
       BTW, you should rethink about the design of this class, it should be thread-safe.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -171,18 +178,21 @@
   private final TreeMap<Long, TsFileProcessor> workSequenceTsFileProcessors = new TreeMap<>();
   /** time partition id in the storage group -> tsFileProcessor for this time partition */
   private final TreeMap<Long, TsFileProcessor> workUnsequenceTsFileProcessors = new TreeMap<>();
+
+  private final Deque<ByteBuffer> walByteBufferPool = new LinkedList<>();
   /** compactionMergeWorking is used to wait for last compaction to be done. */
   private volatile boolean compactionMergeWorking = false;
   // upgrading sequence TsFile resource list
   private List<TsFileResource> upgradeSeqFileList = new LinkedList<>();
-
   /** sequence tsfile processors which are closing */
   private CopyOnReadLinkedList<TsFileProcessor> closingSequenceTsFileProcessor =
       new CopyOnReadLinkedList<>();
-
   // upgrading unsequence TsFile resource list
   private List<TsFileResource> upgradeUnseqFileList = new LinkedList<>();
 
+  private Map<String, TsFileResource> settleSeqFileList = new HashMap<>();
+  private Map<String, TsFileResource> settleUnseqFileList = new HashMap<>();

Review comment:
       Why you name the Map<> type as List. And why we need a Map here, the map's key is for what?

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -2033,4 +2046,10 @@ private QueryDataSet processShowQueryProcesslist() {
     }
     return noExistSg;
   }
+
+  private void settle(SettlePlan plan) throws WriteProcessException, StorageEngineException {
+    PartialPath sgPath = plan.getSgPath();
+    SettleService.setStorageGroupPath(sgPath);

Review comment:
       What if I call the settle command twice continuously? SettleService is not a thread-safe class

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.db.tools.settle;
+
+import org.apache.iotdb.db.engine.settle.SettleLog;
+import org.apache.iotdb.db.engine.settle.SettleLog.SettleCheckStatus;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+/**
+ * Offline Settle tool, which is used to settle TsFile and its corresponding mods file to a new
+ * TsFile.
+ */
+public class TsFileAndModSettleTool {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileAndModSettleTool.class);
+  // TsFilePath -> SettleCheckStatus
+  public static Map<String, Integer> recoverSettleFileMap = new HashMap<>();
+
+  public static void main(String[] args) {
+    Map<String, TsFileResource> oldTsFileResources = new HashMap<>();
+    // List<TsFileResource> oldTsFileResources = new ArrayList<>();
+    findFilesToBeRecovered();
+    for (Map.Entry<String, Integer> entry : recoverSettleFileMap.entrySet()) {
+      String path = entry.getKey();
+      TsFileResource resource = new TsFileResource(new File(path));
+      resource.setClosed(true);
+      oldTsFileResources.put(resource.getTsFile().getName(), resource);
+    }
+    List<File> tsFiles = checkArgs(args);
+    if (tsFiles != null) {
+      for (File file : tsFiles) {
+        if (!oldTsFileResources.containsKey(file.getName())) {
+          if (new File(file + TsFileResource.RESOURCE_SUFFIX).exists()) {
+            TsFileResource resource = new TsFileResource(file);
+            resource.setClosed(true);
+            oldTsFileResources.put(file.getName(), resource);
+          }
+        }
+      }
+    }
+    System.out.println(
+        "Totally find "
+            + oldTsFileResources.size()
+            + " tsFiles to be settled, including "
+            + recoverSettleFileMap.size()
+            + " tsFiles to be recovered.");
+    settleTsFilesAndMods(oldTsFileResources);
+  }
+
+  public static List<File> checkArgs(String[] args) {
+    String filePath = "test.tsfile";
+    List<File> files = new ArrayList<>();
+    if (args.length == 0) {
+      return null;
+    } else {
+      for (String arg : args) {
+        if (arg.endsWith(TSFILE_SUFFIX)) { // it's a file
+          File f = new File(arg);
+          if (!f.exists()) {
+            logger.warn("Cannot find TsFile : " + arg);
+            continue;
+          }
+          files.add(f);
+        } else { // it's a dir
+          List<File> tmpFiles = getAllFilesInOneDirBySuffix(arg, TSFILE_SUFFIX);
+          if (tmpFiles == null) {
+            continue;
+          }
+          files.addAll(tmpFiles);
+        }
+      }
+    }
+    return files;
+  }
+
+  private static List<File> getAllFilesInOneDirBySuffix(String dirPath, String suffix) {
+    File dir = new File(dirPath);
+    if (!dir.isDirectory()) {
+      logger.warn("It's not a directory path : " + dirPath);
+      return null;
+    }
+    if (!dir.exists()) {
+      logger.warn("Cannot find Directory : " + dirPath);
+      return null;
+    }
+    List<File> tsFiles = new ArrayList<>();
+    tsFiles.addAll(
+        Arrays.asList(FSFactoryProducer.getFSFactory().listFilesBySuffix(dirPath, suffix)));
+    List<File> tmpFiles = Arrays.asList(dir.listFiles());
+    for (File f : tmpFiles) {
+      if (f.isDirectory()) {
+        tsFiles.addAll(getAllFilesInOneDirBySuffix(f.getAbsolutePath(), suffix));
+      }
+    }
+    return tsFiles;
+  }
+
+  /**
+   * This method is used to settle tsFiles and mods files, so that each old TsFile corresponds to
+   * one or several new TsFiles. This method is only applicable to V3 TsFile.
+   *
+   * @return Each old TsFile corresponds to one or several new TsFileResources of the new TsFiles
+   */
+  public static Map<String, List<TsFileResource>> settleTsFilesAndMods(
+      Map<String, TsFileResource> resourcesToBeSettled) {
+    int successCount = 0;
+    Map<String, List<TsFileResource>> newTsFileResources = new HashMap<>();
+    SettleLog.createSettleLog();
+    for (Map.Entry<String, TsFileResource> entry : resourcesToBeSettled.entrySet()) {
+      TsFileResource resourceToBeSettled = entry.getValue();
+      List<TsFileResource> settledTsFileResources = new ArrayList<>();
+      try {
+        TsFileAndModSettleTool tsFileAndModSettleTool = new TsFileAndModSettleTool();
+        System.out.println("Start settling for tsFile : " + resourceToBeSettled.getTsFilePath());
+        if (tsFileAndModSettleTool.isSettledFileGenerated(resourceToBeSettled)) {
+          settledTsFileResources = tsFileAndModSettleTool.findSettledFile(resourceToBeSettled);
+          newTsFileResources.put(resourceToBeSettled.getTsFile().getName(), settledTsFileResources);
+        } else {
+          // Write Settle Log, Status 1
+          SettleLog.writeSettleLog(
+              resourceToBeSettled.getTsFilePath()
+                  + SettleLog.COMMA_SEPERATOR
+                  + SettleCheckStatus.BEGIN_SETTLE_FILE);
+          tsFileAndModSettleTool.settleOneTsFileAndMod(resourceToBeSettled, settledTsFileResources);
+          // Write Settle Log, Status 2
+          SettleLog.writeSettleLog(
+              resourceToBeSettled.getTsFilePath()
+                  + SettleLog.COMMA_SEPERATOR
+                  + SettleCheckStatus.AFTER_SETTLE_FILE);
+          newTsFileResources.put(resourceToBeSettled.getTsFile().getName(), settledTsFileResources);
+        }
+
+        moveNewTsFile(resourceToBeSettled, settledTsFileResources);
+        // Write Settle Log, Status 3
+        SettleLog.writeSettleLog(
+            resourceToBeSettled.getTsFilePath()
+                + SettleLog.COMMA_SEPERATOR
+                + SettleCheckStatus.SETTLE_SUCCESS);
+        System.out.println(
+            "Finish settling successfully for tsFile : " + resourceToBeSettled.getTsFilePath());
+        successCount++;
+      } catch (Exception e) {
+        System.out.println(
+            "Meet error while settling the tsFile : " + resourceToBeSettled.getTsFilePath());
+        e.printStackTrace();
+      }
+    }
+    if (resourcesToBeSettled.size() == successCount) {
+      SettleLog.closeLogWriter();
+      FSFactoryProducer.getFSFactory().getFile(SettleLog.getSettleLogPath()).delete();
+      System.out.println("Finish settling all tsfiles Successfully!");
+    } else {
+      System.out.println(
+          "Finish Settling, "
+              + (resourcesToBeSettled.size() - successCount)
+              + " tsfiles meet errors.");
+    }
+    return newTsFileResources;
+  }
+
+  /**
+   * The size of settledResources will be 0 in one of the following conditions: (1) old TsFile is
+   * not closed (2) old ModFile is not existed (3) all data in the old tsfile is being deleted after
+   * settling
+   */
+  public void settleOneTsFileAndMod(
+      TsFileResource resourceToBeSettled, List<TsFileResource> settledResources) throws Exception {
+    if (!resourceToBeSettled.isClosed()) {
+      logger.warn(
+          "The tsFile {} should be sealed when rewritting.", resourceToBeSettled.getTsFilePath());
+      return;
+    }
+    // if no deletions to this tsfile, then return.
+    if (!resourceToBeSettled.getModFile().exists()) {
+      return;
+    }
+    try (TsFileRewriteTool tsFileRewriteTool = new TsFileRewriteTool(resourceToBeSettled)) {
+      tsFileRewriteTool.parseAndRewriteFile(settledResources);
+    }
+    if (settledResources.size() == 0) { // if all the data in this tsfile has been deleted
+      resourceToBeSettled.readUnlock();
+      resourceToBeSettled.writeLock();
+      resourceToBeSettled.delete();
+      resourceToBeSettled.writeUnlock();
+      resourceToBeSettled.readLock();
+    }
+    return;
+  }
+
+  public static void findFilesToBeRecovered() {
+    if (FSFactoryProducer.getFSFactory().getFile(SettleLog.getSettleLogPath()).exists()) {
+      try (BufferedReader settleLogReader =
+          new BufferedReader(
+              new FileReader(
+                  FSFactoryProducer.getFSFactory().getFile(SettleLog.getSettleLogPath())))) {
+        String line = null;
+        while ((line = settleLogReader.readLine()) != null && !line.equals("")) {
+          String oldFilePath = line.split(SettleLog.COMMA_SEPERATOR)[0];
+          Integer settleCheckStatus = Integer.parseInt(line.split(SettleLog.COMMA_SEPERATOR)[1]);

Review comment:
       ```suggestion
             int settleCheckStatus = Integer.parseInt(line.split(SettleLog.COMMA_SEPERATOR)[1]);
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.db.tools.settle;
+
+import org.apache.iotdb.db.engine.settle.SettleLog;
+import org.apache.iotdb.db.engine.settle.SettleLog.SettleCheckStatus;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+/**
+ * Offline Settle tool, which is used to settle TsFile and its corresponding mods file to a new
+ * TsFile.
+ */
+public class TsFileAndModSettleTool {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileAndModSettleTool.class);
+  // TsFilePath -> SettleCheckStatus
+  public static Map<String, Integer> recoverSettleFileMap = new HashMap<>();
+
+  public static void main(String[] args) {
+    Map<String, TsFileResource> oldTsFileResources = new HashMap<>();
+    // List<TsFileResource> oldTsFileResources = new ArrayList<>();
+    findFilesToBeRecovered();
+    for (Map.Entry<String, Integer> entry : recoverSettleFileMap.entrySet()) {
+      String path = entry.getKey();
+      TsFileResource resource = new TsFileResource(new File(path));
+      resource.setClosed(true);
+      oldTsFileResources.put(resource.getTsFile().getName(), resource);
+    }
+    List<File> tsFiles = checkArgs(args);
+    if (tsFiles != null) {
+      for (File file : tsFiles) {
+        if (!oldTsFileResources.containsKey(file.getName())) {
+          if (new File(file + TsFileResource.RESOURCE_SUFFIX).exists()) {
+            TsFileResource resource = new TsFileResource(file);
+            resource.setClosed(true);
+            oldTsFileResources.put(file.getName(), resource);
+          }
+        }
+      }
+    }
+    System.out.println(
+        "Totally find "
+            + oldTsFileResources.size()
+            + " tsFiles to be settled, including "
+            + recoverSettleFileMap.size()
+            + " tsFiles to be recovered.");
+    settleTsFilesAndMods(oldTsFileResources);
+  }
+
+  public static List<File> checkArgs(String[] args) {
+    String filePath = "test.tsfile";
+    List<File> files = new ArrayList<>();
+    if (args.length == 0) {
+      return null;
+    } else {
+      for (String arg : args) {
+        if (arg.endsWith(TSFILE_SUFFIX)) { // it's a file
+          File f = new File(arg);
+          if (!f.exists()) {
+            logger.warn("Cannot find TsFile : " + arg);
+            continue;
+          }
+          files.add(f);
+        } else { // it's a dir
+          List<File> tmpFiles = getAllFilesInOneDirBySuffix(arg, TSFILE_SUFFIX);
+          if (tmpFiles == null) {
+            continue;
+          }
+          files.addAll(tmpFiles);
+        }
+      }
+    }
+    return files;
+  }
+
+  private static List<File> getAllFilesInOneDirBySuffix(String dirPath, String suffix) {
+    File dir = new File(dirPath);
+    if (!dir.isDirectory()) {
+      logger.warn("It's not a directory path : " + dirPath);
+      return null;
+    }
+    if (!dir.exists()) {
+      logger.warn("Cannot find Directory : " + dirPath);
+      return null;
+    }
+    List<File> tsFiles = new ArrayList<>();
+    tsFiles.addAll(
+        Arrays.asList(FSFactoryProducer.getFSFactory().listFilesBySuffix(dirPath, suffix)));
+    List<File> tmpFiles = Arrays.asList(dir.listFiles());
+    for (File f : tmpFiles) {
+      if (f.isDirectory()) {
+        tsFiles.addAll(getAllFilesInOneDirBySuffix(f.getAbsolutePath(), suffix));
+      }
+    }
+    return tsFiles;

Review comment:
       ```suggestion
       File dir = new File(dirPath);
       if (!dir.isDirectory()) {
         logger.warn("It's not a directory path : " + dirPath);
         return Collections.emptyList();
       }
       if (!dir.exists()) {
         logger.warn("Cannot find Directory : " + dirPath);
         return Collections.emptyList();
       }
       List<File> tsFiles = new ArrayList<>(
           Arrays.asList(FSFactoryProducer.getFSFactory().listFilesBySuffix(dirPath, suffix)));
       File[] tmpFiles = dir.listFiles();
       if (tmpFiles != null) {
         for (File f : tmpFiles) {
           if (f.isDirectory()) {
             tsFiles.addAll(getAllFilesInOneDirBySuffix(f.getAbsolutePath(), suffix));
           }
         }
       }
       return tsFiles;
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SettlePlan.java
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.db.qp.physical.sys;
+
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+
+import java.util.List;
+
+public class SettlePlan extends PhysicalPlan {
+  PartialPath sgPath;
+
+  public SettlePlan(PartialPath sgPath) {
+    super(false, OperatorType.SETTLE);
+    this.sgPath = sgPath;
+  }
+
+  @Override
+  public List<PartialPath> getPaths() {
+    return null;

Review comment:
       ```suggestion
       return  Collections.singletonList(sgPath);
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/service/SettleService.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.db.service;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.settle.SettleLog;
+import org.apache.iotdb.db.engine.settle.SettleTask;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.WriteProcessException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.tools.settle.TsFileAndModSettleTool;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class SettleService implements IService {

Review comment:
       Why the settle need to be an IService? Should that be a background thread and periodically started to do settle job? It seems that only when we call the settle command, then it will start.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/settle/SettleTask.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.db.engine.settle;
+
+import org.apache.iotdb.db.concurrent.WrappedRunnable;
+import org.apache.iotdb.db.engine.settle.SettleLog.SettleCheckStatus;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.service.SettleService;
+import org.apache.iotdb.db.tools.settle.TsFileAndModSettleTool;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class SettleTask extends WrappedRunnable {
+
+  private static final Logger logger = LoggerFactory.getLogger(SettleTask.class);
+  private TsFileResource resourceToBeSettled;
+  private FSFactory fsFactory = FSFactoryProducer.getFSFactory();

Review comment:
       ```suggestion
     private final TsFileResource resourceToBeSettled;
     private final FSFactory fsFactory = FSFactoryProducer.getFSFactory();
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.db.tools.settle;
+
+import org.apache.iotdb.db.engine.settle.SettleLog;
+import org.apache.iotdb.db.engine.settle.SettleLog.SettleCheckStatus;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+/**
+ * Offline Settle tool, which is used to settle TsFile and its corresponding mods file to a new
+ * TsFile.
+ */
+public class TsFileAndModSettleTool {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileAndModSettleTool.class);
+  // TsFilePath -> SettleCheckStatus
+  public static Map<String, Integer> recoverSettleFileMap = new HashMap<>();
+
+  public static void main(String[] args) {
+    Map<String, TsFileResource> oldTsFileResources = new HashMap<>();
+    // List<TsFileResource> oldTsFileResources = new ArrayList<>();
+    findFilesToBeRecovered();
+    for (Map.Entry<String, Integer> entry : recoverSettleFileMap.entrySet()) {
+      String path = entry.getKey();
+      TsFileResource resource = new TsFileResource(new File(path));
+      resource.setClosed(true);
+      oldTsFileResources.put(resource.getTsFile().getName(), resource);
+    }
+    List<File> tsFiles = checkArgs(args);
+    if (tsFiles != null) {
+      for (File file : tsFiles) {
+        if (!oldTsFileResources.containsKey(file.getName())) {
+          if (new File(file + TsFileResource.RESOURCE_SUFFIX).exists()) {
+            TsFileResource resource = new TsFileResource(file);
+            resource.setClosed(true);
+            oldTsFileResources.put(file.getName(), resource);
+          }
+        }
+      }
+    }
+    System.out.println(
+        "Totally find "
+            + oldTsFileResources.size()
+            + " tsFiles to be settled, including "
+            + recoverSettleFileMap.size()
+            + " tsFiles to be recovered.");
+    settleTsFilesAndMods(oldTsFileResources);
+  }
+
+  public static List<File> checkArgs(String[] args) {
+    String filePath = "test.tsfile";
+    List<File> files = new ArrayList<>();
+    if (args.length == 0) {
+      return null;
+    } else {
+      for (String arg : args) {
+        if (arg.endsWith(TSFILE_SUFFIX)) { // it's a file
+          File f = new File(arg);
+          if (!f.exists()) {
+            logger.warn("Cannot find TsFile : " + arg);
+            continue;
+          }
+          files.add(f);
+        } else { // it's a dir
+          List<File> tmpFiles = getAllFilesInOneDirBySuffix(arg, TSFILE_SUFFIX);
+          if (tmpFiles == null) {
+            continue;
+          }
+          files.addAll(tmpFiles);
+        }
+      }
+    }
+    return files;
+  }
+
+  private static List<File> getAllFilesInOneDirBySuffix(String dirPath, String suffix) {
+    File dir = new File(dirPath);
+    if (!dir.isDirectory()) {
+      logger.warn("It's not a directory path : " + dirPath);
+      return null;
+    }
+    if (!dir.exists()) {
+      logger.warn("Cannot find Directory : " + dirPath);
+      return null;
+    }
+    List<File> tsFiles = new ArrayList<>();
+    tsFiles.addAll(
+        Arrays.asList(FSFactoryProducer.getFSFactory().listFilesBySuffix(dirPath, suffix)));
+    List<File> tmpFiles = Arrays.asList(dir.listFiles());
+    for (File f : tmpFiles) {
+      if (f.isDirectory()) {
+        tsFiles.addAll(getAllFilesInOneDirBySuffix(f.getAbsolutePath(), suffix));
+      }
+    }
+    return tsFiles;
+  }
+
+  /**
+   * This method is used to settle tsFiles and mods files, so that each old TsFile corresponds to
+   * one or several new TsFiles. This method is only applicable to V3 TsFile.
+   *
+   * @return Each old TsFile corresponds to one or several new TsFileResources of the new TsFiles
+   */
+  public static Map<String, List<TsFileResource>> settleTsFilesAndMods(
+      Map<String, TsFileResource> resourcesToBeSettled) {
+    int successCount = 0;
+    Map<String, List<TsFileResource>> newTsFileResources = new HashMap<>();
+    SettleLog.createSettleLog();
+    for (Map.Entry<String, TsFileResource> entry : resourcesToBeSettled.entrySet()) {
+      TsFileResource resourceToBeSettled = entry.getValue();
+      List<TsFileResource> settledTsFileResources = new ArrayList<>();
+      try {
+        TsFileAndModSettleTool tsFileAndModSettleTool = new TsFileAndModSettleTool();
+        System.out.println("Start settling for tsFile : " + resourceToBeSettled.getTsFilePath());
+        if (tsFileAndModSettleTool.isSettledFileGenerated(resourceToBeSettled)) {
+          settledTsFileResources = tsFileAndModSettleTool.findSettledFile(resourceToBeSettled);
+          newTsFileResources.put(resourceToBeSettled.getTsFile().getName(), settledTsFileResources);
+        } else {
+          // Write Settle Log, Status 1
+          SettleLog.writeSettleLog(
+              resourceToBeSettled.getTsFilePath()
+                  + SettleLog.COMMA_SEPERATOR
+                  + SettleCheckStatus.BEGIN_SETTLE_FILE);
+          tsFileAndModSettleTool.settleOneTsFileAndMod(resourceToBeSettled, settledTsFileResources);
+          // Write Settle Log, Status 2
+          SettleLog.writeSettleLog(
+              resourceToBeSettled.getTsFilePath()
+                  + SettleLog.COMMA_SEPERATOR
+                  + SettleCheckStatus.AFTER_SETTLE_FILE);
+          newTsFileResources.put(resourceToBeSettled.getTsFile().getName(), settledTsFileResources);
+        }
+
+        moveNewTsFile(resourceToBeSettled, settledTsFileResources);
+        // Write Settle Log, Status 3
+        SettleLog.writeSettleLog(
+            resourceToBeSettled.getTsFilePath()
+                + SettleLog.COMMA_SEPERATOR
+                + SettleCheckStatus.SETTLE_SUCCESS);
+        System.out.println(
+            "Finish settling successfully for tsFile : " + resourceToBeSettled.getTsFilePath());
+        successCount++;
+      } catch (Exception e) {
+        System.out.println(
+            "Meet error while settling the tsFile : " + resourceToBeSettled.getTsFilePath());
+        e.printStackTrace();
+      }
+    }
+    if (resourcesToBeSettled.size() == successCount) {
+      SettleLog.closeLogWriter();
+      FSFactoryProducer.getFSFactory().getFile(SettleLog.getSettleLogPath()).delete();
+      System.out.println("Finish settling all tsfiles Successfully!");
+    } else {
+      System.out.println(
+          "Finish Settling, "
+              + (resourcesToBeSettled.size() - successCount)
+              + " tsfiles meet errors.");
+    }
+    return newTsFileResources;
+  }
+
+  /**
+   * The size of settledResources will be 0 in one of the following conditions: (1) old TsFile is
+   * not closed (2) old ModFile is not existed (3) all data in the old tsfile is being deleted after
+   * settling
+   */
+  public void settleOneTsFileAndMod(
+      TsFileResource resourceToBeSettled, List<TsFileResource> settledResources) throws Exception {
+    if (!resourceToBeSettled.isClosed()) {
+      logger.warn(
+          "The tsFile {} should be sealed when rewritting.", resourceToBeSettled.getTsFilePath());
+      return;
+    }
+    // if no deletions to this tsfile, then return.
+    if (!resourceToBeSettled.getModFile().exists()) {
+      return;
+    }
+    try (TsFileRewriteTool tsFileRewriteTool = new TsFileRewriteTool(resourceToBeSettled)) {
+      tsFileRewriteTool.parseAndRewriteFile(settledResources);
+    }
+    if (settledResources.size() == 0) { // if all the data in this tsfile has been deleted
+      resourceToBeSettled.readUnlock();
+      resourceToBeSettled.writeLock();
+      resourceToBeSettled.delete();
+      resourceToBeSettled.writeUnlock();
+      resourceToBeSettled.readLock();
+    }
+    return;

Review comment:
       ```suggestion
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.db.tools.settle;
+
+import org.apache.iotdb.db.engine.settle.SettleLog;
+import org.apache.iotdb.db.engine.settle.SettleLog.SettleCheckStatus;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+/**
+ * Offline Settle tool, which is used to settle TsFile and its corresponding mods file to a new
+ * TsFile.
+ */
+public class TsFileAndModSettleTool {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileAndModSettleTool.class);
+  // TsFilePath -> SettleCheckStatus
+  public static Map<String, Integer> recoverSettleFileMap = new HashMap<>();
+
+  public static void main(String[] args) {
+    Map<String, TsFileResource> oldTsFileResources = new HashMap<>();
+    // List<TsFileResource> oldTsFileResources = new ArrayList<>();
+    findFilesToBeRecovered();
+    for (Map.Entry<String, Integer> entry : recoverSettleFileMap.entrySet()) {
+      String path = entry.getKey();
+      TsFileResource resource = new TsFileResource(new File(path));
+      resource.setClosed(true);
+      oldTsFileResources.put(resource.getTsFile().getName(), resource);
+    }
+    List<File> tsFiles = checkArgs(args);
+    if (tsFiles != null) {
+      for (File file : tsFiles) {
+        if (!oldTsFileResources.containsKey(file.getName())) {
+          if (new File(file + TsFileResource.RESOURCE_SUFFIX).exists()) {
+            TsFileResource resource = new TsFileResource(file);
+            resource.setClosed(true);
+            oldTsFileResources.put(file.getName(), resource);
+          }
+        }
+      }
+    }
+    System.out.println(
+        "Totally find "
+            + oldTsFileResources.size()
+            + " tsFiles to be settled, including "
+            + recoverSettleFileMap.size()
+            + " tsFiles to be recovered.");
+    settleTsFilesAndMods(oldTsFileResources);
+  }
+
+  public static List<File> checkArgs(String[] args) {
+    String filePath = "test.tsfile";
+    List<File> files = new ArrayList<>();
+    if (args.length == 0) {
+      return null;
+    } else {
+      for (String arg : args) {
+        if (arg.endsWith(TSFILE_SUFFIX)) { // it's a file
+          File f = new File(arg);
+          if (!f.exists()) {
+            logger.warn("Cannot find TsFile : " + arg);
+            continue;
+          }
+          files.add(f);
+        } else { // it's a dir
+          List<File> tmpFiles = getAllFilesInOneDirBySuffix(arg, TSFILE_SUFFIX);
+          if (tmpFiles == null) {
+            continue;
+          }
+          files.addAll(tmpFiles);
+        }
+      }
+    }
+    return files;
+  }
+
+  private static List<File> getAllFilesInOneDirBySuffix(String dirPath, String suffix) {
+    File dir = new File(dirPath);
+    if (!dir.isDirectory()) {
+      logger.warn("It's not a directory path : " + dirPath);
+      return null;
+    }
+    if (!dir.exists()) {
+      logger.warn("Cannot find Directory : " + dirPath);
+      return null;
+    }
+    List<File> tsFiles = new ArrayList<>();
+    tsFiles.addAll(
+        Arrays.asList(FSFactoryProducer.getFSFactory().listFilesBySuffix(dirPath, suffix)));
+    List<File> tmpFiles = Arrays.asList(dir.listFiles());
+    for (File f : tmpFiles) {
+      if (f.isDirectory()) {
+        tsFiles.addAll(getAllFilesInOneDirBySuffix(f.getAbsolutePath(), suffix));
+      }
+    }
+    return tsFiles;
+  }
+
+  /**
+   * This method is used to settle tsFiles and mods files, so that each old TsFile corresponds to
+   * one or several new TsFiles. This method is only applicable to V3 TsFile.
+   *
+   * @return Each old TsFile corresponds to one or several new TsFileResources of the new TsFiles
+   */
+  public static Map<String, List<TsFileResource>> settleTsFilesAndMods(
+      Map<String, TsFileResource> resourcesToBeSettled) {
+    int successCount = 0;
+    Map<String, List<TsFileResource>> newTsFileResources = new HashMap<>();
+    SettleLog.createSettleLog();

Review comment:
       Where did you delete the old settle log file?

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.db.tools.settle;
+
+import org.apache.iotdb.db.engine.settle.SettleLog;
+import org.apache.iotdb.db.engine.settle.SettleLog.SettleCheckStatus;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+/**
+ * Offline Settle tool, which is used to settle TsFile and its corresponding mods file to a new
+ * TsFile.
+ */
+public class TsFileAndModSettleTool {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileAndModSettleTool.class);
+  // TsFilePath -> SettleCheckStatus
+  public static Map<String, Integer> recoverSettleFileMap = new HashMap<>();
+
+  public static void main(String[] args) {
+    Map<String, TsFileResource> oldTsFileResources = new HashMap<>();
+    // List<TsFileResource> oldTsFileResources = new ArrayList<>();
+    findFilesToBeRecovered();
+    for (Map.Entry<String, Integer> entry : recoverSettleFileMap.entrySet()) {
+      String path = entry.getKey();
+      TsFileResource resource = new TsFileResource(new File(path));
+      resource.setClosed(true);
+      oldTsFileResources.put(resource.getTsFile().getName(), resource);
+    }
+    List<File> tsFiles = checkArgs(args);
+    if (tsFiles != null) {
+      for (File file : tsFiles) {
+        if (!oldTsFileResources.containsKey(file.getName())) {
+          if (new File(file + TsFileResource.RESOURCE_SUFFIX).exists()) {
+            TsFileResource resource = new TsFileResource(file);
+            resource.setClosed(true);
+            oldTsFileResources.put(file.getName(), resource);
+          }
+        }
+      }
+    }
+    System.out.println(
+        "Totally find "
+            + oldTsFileResources.size()
+            + " tsFiles to be settled, including "
+            + recoverSettleFileMap.size()
+            + " tsFiles to be recovered.");
+    settleTsFilesAndMods(oldTsFileResources);
+  }
+
+  public static List<File> checkArgs(String[] args) {
+    String filePath = "test.tsfile";
+    List<File> files = new ArrayList<>();
+    if (args.length == 0) {
+      return null;
+    } else {
+      for (String arg : args) {
+        if (arg.endsWith(TSFILE_SUFFIX)) { // it's a file
+          File f = new File(arg);
+          if (!f.exists()) {
+            logger.warn("Cannot find TsFile : " + arg);
+            continue;
+          }
+          files.add(f);
+        } else { // it's a dir
+          List<File> tmpFiles = getAllFilesInOneDirBySuffix(arg, TSFILE_SUFFIX);
+          if (tmpFiles == null) {
+            continue;
+          }
+          files.addAll(tmpFiles);
+        }
+      }
+    }
+    return files;
+  }
+
+  private static List<File> getAllFilesInOneDirBySuffix(String dirPath, String suffix) {
+    File dir = new File(dirPath);
+    if (!dir.isDirectory()) {
+      logger.warn("It's not a directory path : " + dirPath);
+      return null;
+    }
+    if (!dir.exists()) {
+      logger.warn("Cannot find Directory : " + dirPath);
+      return null;
+    }
+    List<File> tsFiles = new ArrayList<>();
+    tsFiles.addAll(
+        Arrays.asList(FSFactoryProducer.getFSFactory().listFilesBySuffix(dirPath, suffix)));
+    List<File> tmpFiles = Arrays.asList(dir.listFiles());
+    for (File f : tmpFiles) {
+      if (f.isDirectory()) {
+        tsFiles.addAll(getAllFilesInOneDirBySuffix(f.getAbsolutePath(), suffix));
+      }
+    }
+    return tsFiles;
+  }
+
+  /**
+   * This method is used to settle tsFiles and mods files, so that each old TsFile corresponds to
+   * one or several new TsFiles. This method is only applicable to V3 TsFile.
+   *
+   * @return Each old TsFile corresponds to one or several new TsFileResources of the new TsFiles
+   */
+  public static Map<String, List<TsFileResource>> settleTsFilesAndMods(
+      Map<String, TsFileResource> resourcesToBeSettled) {
+    int successCount = 0;
+    Map<String, List<TsFileResource>> newTsFileResources = new HashMap<>();
+    SettleLog.createSettleLog();
+    for (Map.Entry<String, TsFileResource> entry : resourcesToBeSettled.entrySet()) {
+      TsFileResource resourceToBeSettled = entry.getValue();
+      List<TsFileResource> settledTsFileResources = new ArrayList<>();
+      try {
+        TsFileAndModSettleTool tsFileAndModSettleTool = new TsFileAndModSettleTool();
+        System.out.println("Start settling for tsFile : " + resourceToBeSettled.getTsFilePath());
+        if (tsFileAndModSettleTool.isSettledFileGenerated(resourceToBeSettled)) {
+          settledTsFileResources = tsFileAndModSettleTool.findSettledFile(resourceToBeSettled);
+          newTsFileResources.put(resourceToBeSettled.getTsFile().getName(), settledTsFileResources);
+        } else {
+          // Write Settle Log, Status 1
+          SettleLog.writeSettleLog(
+              resourceToBeSettled.getTsFilePath()
+                  + SettleLog.COMMA_SEPERATOR
+                  + SettleCheckStatus.BEGIN_SETTLE_FILE);
+          tsFileAndModSettleTool.settleOneTsFileAndMod(resourceToBeSettled, settledTsFileResources);
+          // Write Settle Log, Status 2
+          SettleLog.writeSettleLog(
+              resourceToBeSettled.getTsFilePath()
+                  + SettleLog.COMMA_SEPERATOR
+                  + SettleCheckStatus.AFTER_SETTLE_FILE);
+          newTsFileResources.put(resourceToBeSettled.getTsFile().getName(), settledTsFileResources);
+        }
+
+        moveNewTsFile(resourceToBeSettled, settledTsFileResources);
+        // Write Settle Log, Status 3
+        SettleLog.writeSettleLog(
+            resourceToBeSettled.getTsFilePath()
+                + SettleLog.COMMA_SEPERATOR
+                + SettleCheckStatus.SETTLE_SUCCESS);
+        System.out.println(
+            "Finish settling successfully for tsFile : " + resourceToBeSettled.getTsFilePath());
+        successCount++;
+      } catch (Exception e) {
+        System.out.println(
+            "Meet error while settling the tsFile : " + resourceToBeSettled.getTsFilePath());
+        e.printStackTrace();
+      }
+    }
+    if (resourcesToBeSettled.size() == successCount) {
+      SettleLog.closeLogWriter();
+      FSFactoryProducer.getFSFactory().getFile(SettleLog.getSettleLogPath()).delete();
+      System.out.println("Finish settling all tsfiles Successfully!");
+    } else {
+      System.out.println(
+          "Finish Settling, "
+              + (resourcesToBeSettled.size() - successCount)
+              + " tsfiles meet errors.");
+    }
+    return newTsFileResources;
+  }
+
+  /**
+   * The size of settledResources will be 0 in one of the following conditions: (1) old TsFile is
+   * not closed (2) old ModFile is not existed (3) all data in the old tsfile is being deleted after
+   * settling
+   */
+  public void settleOneTsFileAndMod(
+      TsFileResource resourceToBeSettled, List<TsFileResource> settledResources) throws Exception {
+    if (!resourceToBeSettled.isClosed()) {
+      logger.warn(
+          "The tsFile {} should be sealed when rewritting.", resourceToBeSettled.getTsFilePath());
+      return;
+    }
+    // if no deletions to this tsfile, then return.
+    if (!resourceToBeSettled.getModFile().exists()) {
+      return;
+    }
+    try (TsFileRewriteTool tsFileRewriteTool = new TsFileRewriteTool(resourceToBeSettled)) {
+      tsFileRewriteTool.parseAndRewriteFile(settledResources);
+    }
+    if (settledResources.size() == 0) { // if all the data in this tsfile has been deleted
+      resourceToBeSettled.readUnlock();
+      resourceToBeSettled.writeLock();
+      resourceToBeSettled.delete();
+      resourceToBeSettled.writeUnlock();
+      resourceToBeSettled.readLock();
+    }
+    return;
+  }
+
+  public static void findFilesToBeRecovered() {
+    if (FSFactoryProducer.getFSFactory().getFile(SettleLog.getSettleLogPath()).exists()) {
+      try (BufferedReader settleLogReader =
+          new BufferedReader(
+              new FileReader(
+                  FSFactoryProducer.getFSFactory().getFile(SettleLog.getSettleLogPath())))) {
+        String line = null;
+        while ((line = settleLogReader.readLine()) != null && !line.equals("")) {
+          String oldFilePath = line.split(SettleLog.COMMA_SEPERATOR)[0];
+          Integer settleCheckStatus = Integer.parseInt(line.split(SettleLog.COMMA_SEPERATOR)[1]);
+          if (settleCheckStatus == SettleCheckStatus.SETTLE_SUCCESS.getCheckStatus()) {
+            recoverSettleFileMap.remove(oldFilePath);
+            continue;
+          }
+          recoverSettleFileMap.put(oldFilePath, settleCheckStatus);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "meet error when recover settle process, file path:{}",
+            SettleLog.getSettleLogPath(),
+            e);
+      }
+    }
+  }
+
+  /** this method is used to check whether the new file is settled when recovering old tsFile. */
+  public boolean isSettledFileGenerated(TsFileResource oldTsFileResource) {
+    String oldFilePath = oldTsFileResource.getTsFilePath();
+    return TsFileAndModSettleTool.recoverSettleFileMap.containsKey(oldFilePath)
+        && TsFileAndModSettleTool.recoverSettleFileMap.get(oldFilePath)
+            == SettleCheckStatus.AFTER_SETTLE_FILE.getCheckStatus();
+  }
+
+  /** when the new file is settled , we need to find and deserialize it. */
+  public List<TsFileResource> findSettledFile(TsFileResource resourceToBeSettled)
+      throws IOException {
+    List<TsFileResource> settledTsFileResources = new ArrayList<>();
+    SettleLog.writeSettleLog(
+        resourceToBeSettled.getTsFilePath()
+            + SettleLog.COMMA_SEPERATOR
+            + SettleCheckStatus.BEGIN_SETTLE_FILE);
+
+    for (File tempPartitionDir : resourceToBeSettled.getTsFile().getParentFile().listFiles()) {
+      if (tempPartitionDir.isDirectory()
+          && FSFactoryProducer.getFSFactory()
+              .getFile(
+                  tempPartitionDir,
+                  resourceToBeSettled.getTsFile().getName() + TsFileResource.RESOURCE_SUFFIX)
+              .exists()) {
+        TsFileResource settledTsFileResource =
+            new TsFileResource(
+                FSFactoryProducer.getFSFactory()
+                    .getFile(tempPartitionDir, resourceToBeSettled.getTsFile().getName()));
+        settledTsFileResource.deserialize();
+        settledTsFileResources.add(settledTsFileResource);
+      }
+    }
+    SettleLog.writeSettleLog(
+        resourceToBeSettled.getTsFilePath()
+            + SettleLog.COMMA_SEPERATOR
+            + SettleCheckStatus.AFTER_SETTLE_FILE);
+    return settledTsFileResources;
+  }
+
+  /**
+   * This method is used to move a new TsFile and its corresponding resource file to the correct
+   * folder.
+   *
+   * @param oldTsFileResource
+   * @param newTsFileResources if the old TsFile has not any deletions or all the data in which has
+   *     been deleted or its modFile does not exist, then this size will be 0.
+   * @throws IOException
+   */
+  public static void moveNewTsFile(
+      TsFileResource oldTsFileResource, List<TsFileResource> newTsFileResources)
+      throws IOException {
+    // delete old mods
+    oldTsFileResource.removeModFile();
+
+    File newPartionDir =

Review comment:
       ```suggestion
       File newPartitionDir =
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.db.tools.settle;
+
+import org.apache.iotdb.db.engine.settle.SettleLog;
+import org.apache.iotdb.db.engine.settle.SettleLog.SettleCheckStatus;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+/**
+ * Offline Settle tool, which is used to settle TsFile and its corresponding mods file to a new
+ * TsFile.
+ */
+public class TsFileAndModSettleTool {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileAndModSettleTool.class);
+  // TsFilePath -> SettleCheckStatus
+  public static Map<String, Integer> recoverSettleFileMap = new HashMap<>();
+
+  public static void main(String[] args) {
+    Map<String, TsFileResource> oldTsFileResources = new HashMap<>();
+    // List<TsFileResource> oldTsFileResources = new ArrayList<>();
+    findFilesToBeRecovered();
+    for (Map.Entry<String, Integer> entry : recoverSettleFileMap.entrySet()) {
+      String path = entry.getKey();
+      TsFileResource resource = new TsFileResource(new File(path));
+      resource.setClosed(true);
+      oldTsFileResources.put(resource.getTsFile().getName(), resource);
+    }
+    List<File> tsFiles = checkArgs(args);
+    if (tsFiles != null) {
+      for (File file : tsFiles) {
+        if (!oldTsFileResources.containsKey(file.getName())) {
+          if (new File(file + TsFileResource.RESOURCE_SUFFIX).exists()) {
+            TsFileResource resource = new TsFileResource(file);
+            resource.setClosed(true);
+            oldTsFileResources.put(file.getName(), resource);
+          }
+        }
+      }
+    }
+    System.out.println(
+        "Totally find "
+            + oldTsFileResources.size()
+            + " tsFiles to be settled, including "
+            + recoverSettleFileMap.size()
+            + " tsFiles to be recovered.");
+    settleTsFilesAndMods(oldTsFileResources);
+  }
+
+  public static List<File> checkArgs(String[] args) {
+    String filePath = "test.tsfile";
+    List<File> files = new ArrayList<>();
+    if (args.length == 0) {
+      return null;
+    } else {
+      for (String arg : args) {
+        if (arg.endsWith(TSFILE_SUFFIX)) { // it's a file
+          File f = new File(arg);
+          if (!f.exists()) {
+            logger.warn("Cannot find TsFile : " + arg);
+            continue;
+          }
+          files.add(f);
+        } else { // it's a dir
+          List<File> tmpFiles = getAllFilesInOneDirBySuffix(arg, TSFILE_SUFFIX);
+          if (tmpFiles == null) {
+            continue;
+          }
+          files.addAll(tmpFiles);
+        }
+      }
+    }
+    return files;
+  }
+
+  private static List<File> getAllFilesInOneDirBySuffix(String dirPath, String suffix) {
+    File dir = new File(dirPath);
+    if (!dir.isDirectory()) {
+      logger.warn("It's not a directory path : " + dirPath);
+      return null;
+    }
+    if (!dir.exists()) {
+      logger.warn("Cannot find Directory : " + dirPath);
+      return null;
+    }
+    List<File> tsFiles = new ArrayList<>();
+    tsFiles.addAll(
+        Arrays.asList(FSFactoryProducer.getFSFactory().listFilesBySuffix(dirPath, suffix)));
+    List<File> tmpFiles = Arrays.asList(dir.listFiles());
+    for (File f : tmpFiles) {
+      if (f.isDirectory()) {
+        tsFiles.addAll(getAllFilesInOneDirBySuffix(f.getAbsolutePath(), suffix));
+      }
+    }
+    return tsFiles;

Review comment:
       The previous way will cause NPE and unnecessary list resize.

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.db.tools.settle;
+
+import org.apache.iotdb.db.engine.settle.SettleLog;
+import org.apache.iotdb.db.engine.settle.SettleLog.SettleCheckStatus;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+/**
+ * Offline Settle tool, which is used to settle TsFile and its corresponding mods file to a new
+ * TsFile.
+ */
+public class TsFileAndModSettleTool {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileAndModSettleTool.class);
+  // TsFilePath -> SettleCheckStatus
+  public static Map<String, Integer> recoverSettleFileMap = new HashMap<>();
+
+  public static void main(String[] args) {
+    Map<String, TsFileResource> oldTsFileResources = new HashMap<>();
+    // List<TsFileResource> oldTsFileResources = new ArrayList<>();
+    findFilesToBeRecovered();
+    for (Map.Entry<String, Integer> entry : recoverSettleFileMap.entrySet()) {
+      String path = entry.getKey();
+      TsFileResource resource = new TsFileResource(new File(path));
+      resource.setClosed(true);
+      oldTsFileResources.put(resource.getTsFile().getName(), resource);
+    }
+    List<File> tsFiles = checkArgs(args);
+    if (tsFiles != null) {
+      for (File file : tsFiles) {
+        if (!oldTsFileResources.containsKey(file.getName())) {
+          if (new File(file + TsFileResource.RESOURCE_SUFFIX).exists()) {
+            TsFileResource resource = new TsFileResource(file);
+            resource.setClosed(true);
+            oldTsFileResources.put(file.getName(), resource);
+          }
+        }
+      }
+    }
+    System.out.println(
+        "Totally find "
+            + oldTsFileResources.size()
+            + " tsFiles to be settled, including "
+            + recoverSettleFileMap.size()
+            + " tsFiles to be recovered.");
+    settleTsFilesAndMods(oldTsFileResources);
+  }
+
+  public static List<File> checkArgs(String[] args) {
+    String filePath = "test.tsfile";
+    List<File> files = new ArrayList<>();
+    if (args.length == 0) {
+      return null;
+    } else {
+      for (String arg : args) {
+        if (arg.endsWith(TSFILE_SUFFIX)) { // it's a file
+          File f = new File(arg);
+          if (!f.exists()) {
+            logger.warn("Cannot find TsFile : " + arg);
+            continue;
+          }
+          files.add(f);
+        } else { // it's a dir
+          List<File> tmpFiles = getAllFilesInOneDirBySuffix(arg, TSFILE_SUFFIX);
+          if (tmpFiles == null) {
+            continue;
+          }
+          files.addAll(tmpFiles);
+        }
+      }
+    }
+    return files;
+  }
+
+  private static List<File> getAllFilesInOneDirBySuffix(String dirPath, String suffix) {
+    File dir = new File(dirPath);
+    if (!dir.isDirectory()) {
+      logger.warn("It's not a directory path : " + dirPath);
+      return null;
+    }
+    if (!dir.exists()) {
+      logger.warn("Cannot find Directory : " + dirPath);
+      return null;
+    }
+    List<File> tsFiles = new ArrayList<>();
+    tsFiles.addAll(
+        Arrays.asList(FSFactoryProducer.getFSFactory().listFilesBySuffix(dirPath, suffix)));
+    List<File> tmpFiles = Arrays.asList(dir.listFiles());
+    for (File f : tmpFiles) {
+      if (f.isDirectory()) {
+        tsFiles.addAll(getAllFilesInOneDirBySuffix(f.getAbsolutePath(), suffix));
+      }
+    }
+    return tsFiles;
+  }
+
+  /**
+   * This method is used to settle tsFiles and mods files, so that each old TsFile corresponds to
+   * one or several new TsFiles. This method is only applicable to V3 TsFile.
+   *
+   * @return Each old TsFile corresponds to one or several new TsFileResources of the new TsFiles
+   */
+  public static Map<String, List<TsFileResource>> settleTsFilesAndMods(
+      Map<String, TsFileResource> resourcesToBeSettled) {
+    int successCount = 0;
+    Map<String, List<TsFileResource>> newTsFileResources = new HashMap<>();
+    SettleLog.createSettleLog();
+    for (Map.Entry<String, TsFileResource> entry : resourcesToBeSettled.entrySet()) {
+      TsFileResource resourceToBeSettled = entry.getValue();
+      List<TsFileResource> settledTsFileResources = new ArrayList<>();
+      try {
+        TsFileAndModSettleTool tsFileAndModSettleTool = new TsFileAndModSettleTool();
+        System.out.println("Start settling for tsFile : " + resourceToBeSettled.getTsFilePath());
+        if (tsFileAndModSettleTool.isSettledFileGenerated(resourceToBeSettled)) {
+          settledTsFileResources = tsFileAndModSettleTool.findSettledFile(resourceToBeSettled);
+          newTsFileResources.put(resourceToBeSettled.getTsFile().getName(), settledTsFileResources);
+        } else {
+          // Write Settle Log, Status 1
+          SettleLog.writeSettleLog(
+              resourceToBeSettled.getTsFilePath()
+                  + SettleLog.COMMA_SEPERATOR
+                  + SettleCheckStatus.BEGIN_SETTLE_FILE);
+          tsFileAndModSettleTool.settleOneTsFileAndMod(resourceToBeSettled, settledTsFileResources);
+          // Write Settle Log, Status 2
+          SettleLog.writeSettleLog(
+              resourceToBeSettled.getTsFilePath()
+                  + SettleLog.COMMA_SEPERATOR
+                  + SettleCheckStatus.AFTER_SETTLE_FILE);
+          newTsFileResources.put(resourceToBeSettled.getTsFile().getName(), settledTsFileResources);
+        }
+
+        moveNewTsFile(resourceToBeSettled, settledTsFileResources);
+        // Write Settle Log, Status 3
+        SettleLog.writeSettleLog(
+            resourceToBeSettled.getTsFilePath()
+                + SettleLog.COMMA_SEPERATOR
+                + SettleCheckStatus.SETTLE_SUCCESS);
+        System.out.println(
+            "Finish settling successfully for tsFile : " + resourceToBeSettled.getTsFilePath());
+        successCount++;
+      } catch (Exception e) {
+        System.out.println(
+            "Meet error while settling the tsFile : " + resourceToBeSettled.getTsFilePath());
+        e.printStackTrace();
+      }
+    }
+    if (resourcesToBeSettled.size() == successCount) {
+      SettleLog.closeLogWriter();
+      FSFactoryProducer.getFSFactory().getFile(SettleLog.getSettleLogPath()).delete();
+      System.out.println("Finish settling all tsfiles Successfully!");
+    } else {
+      System.out.println(
+          "Finish Settling, "
+              + (resourcesToBeSettled.size() - successCount)
+              + " tsfiles meet errors.");
+    }
+    return newTsFileResources;
+  }
+
+  /**
+   * The size of settledResources will be 0 in one of the following conditions: (1) old TsFile is
+   * not closed (2) old ModFile is not existed (3) all data in the old tsfile is being deleted after
+   * settling
+   */
+  public void settleOneTsFileAndMod(
+      TsFileResource resourceToBeSettled, List<TsFileResource> settledResources) throws Exception {
+    if (!resourceToBeSettled.isClosed()) {
+      logger.warn(
+          "The tsFile {} should be sealed when rewritting.", resourceToBeSettled.getTsFilePath());
+      return;
+    }
+    // if no deletions to this tsfile, then return.
+    if (!resourceToBeSettled.getModFile().exists()) {
+      return;
+    }
+    try (TsFileRewriteTool tsFileRewriteTool = new TsFileRewriteTool(resourceToBeSettled)) {
+      tsFileRewriteTool.parseAndRewriteFile(settledResources);
+    }
+    if (settledResources.size() == 0) { // if all the data in this tsfile has been deleted
+      resourceToBeSettled.readUnlock();
+      resourceToBeSettled.writeLock();
+      resourceToBeSettled.delete();
+      resourceToBeSettled.writeUnlock();
+      resourceToBeSettled.readLock();
+    }
+    return;
+  }
+
+  public static void findFilesToBeRecovered() {
+    if (FSFactoryProducer.getFSFactory().getFile(SettleLog.getSettleLogPath()).exists()) {
+      try (BufferedReader settleLogReader =
+          new BufferedReader(
+              new FileReader(
+                  FSFactoryProducer.getFSFactory().getFile(SettleLog.getSettleLogPath())))) {
+        String line = null;
+        while ((line = settleLogReader.readLine()) != null && !line.equals("")) {
+          String oldFilePath = line.split(SettleLog.COMMA_SEPERATOR)[0];
+          Integer settleCheckStatus = Integer.parseInt(line.split(SettleLog.COMMA_SEPERATOR)[1]);
+          if (settleCheckStatus == SettleCheckStatus.SETTLE_SUCCESS.getCheckStatus()) {
+            recoverSettleFileMap.remove(oldFilePath);
+            continue;
+          }
+          recoverSettleFileMap.put(oldFilePath, settleCheckStatus);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "meet error when recover settle process, file path:{}",
+            SettleLog.getSettleLogPath(),
+            e);
+      }
+    }
+  }
+
+  /** this method is used to check whether the new file is settled when recovering old tsFile. */
+  public boolean isSettledFileGenerated(TsFileResource oldTsFileResource) {
+    String oldFilePath = oldTsFileResource.getTsFilePath();
+    return TsFileAndModSettleTool.recoverSettleFileMap.containsKey(oldFilePath)
+        && TsFileAndModSettleTool.recoverSettleFileMap.get(oldFilePath)
+            == SettleCheckStatus.AFTER_SETTLE_FILE.getCheckStatus();
+  }
+
+  /** when the new file is settled , we need to find and deserialize it. */
+  public List<TsFileResource> findSettledFile(TsFileResource resourceToBeSettled)
+      throws IOException {
+    List<TsFileResource> settledTsFileResources = new ArrayList<>();
+    SettleLog.writeSettleLog(
+        resourceToBeSettled.getTsFilePath()
+            + SettleLog.COMMA_SEPERATOR
+            + SettleCheckStatus.BEGIN_SETTLE_FILE);
+
+    for (File tempPartitionDir : resourceToBeSettled.getTsFile().getParentFile().listFiles()) {

Review comment:
       Dereference of `resourceToBeSettled.getTsFile().getParentFile().listFiles()` may produce `NullPointerException`.

##########
File path: docs/zh/SystemDesign/Tools/Settle.md
##########
@@ -0,0 +1,119 @@
+<!--
+
+```
+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.
+```
+
+-->
+
+## TsFile整理工具
+
+### 介绍
+
+1. 整理工具可以帮助你整理已封口的tsFile文件和.mods文件,过滤掉删除的数据并生成新的tsFile。整理完后会将本地旧的.mods文件删除,若某tsFile里的数据被全部删除了,则会删除本地该tsFile文件和对应的.resource文件。
+
+2. 整理工具只能针对v0.12版本的IOTDB来使用,即只能针对版本为V3的tsFile进行整理,若低于此版本则必须先用在线升级工具将tsFile升级到V3版本。
+3. tsFile整理工具分为在线整理工具和离线整理工具,两者在整理的时候都会记录日志,供下次恢复先前整理失败的文件。在执行整理的时候会先去检测日志里是否存在失败的文件,有的话则优先整理它们。
+
+### 1. 离线整理工具
+
+离线整理工具是以命令行的方式来启动,该工具的启动脚本settle.bat和settle.sh在编译了server后会生成至server\target\iotdb-server-{version}\tools\tsFileToolSet目录中。在使用的时候必须保证IOTDB Server是停止运行的,否则会出现整理错误。具体使用方式详见用户手册。
+
+#### 1.1 涉及的相关类
+
+整理工具类:org.apache.iotdb.db.tools.settle.tsFileAndModSettleTool
+
+### 2.在线整理工具
+
+在线整理工具是当用户在IOTDB客户端输入了settle命令后,会在后台注册启动一个整理服务Settle Service,该服务会去寻找指定存储组下的所有tsFile文件,并为每个tsFile开启一个整理线程进行整理。下面讲解整理线程的工作流程。

Review comment:
       Why we need to create a thread for each tsfile? Maybe we need a parameter to control the granularity of concurrency. Otherwise, if the number of mods file is large, it will influence the common write and read throughput.

##########
File path: server/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java
##########
@@ -0,0 +1,353 @@
+/*
+ * 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.db.tools.settle;
+
+import org.apache.iotdb.db.engine.settle.SettleLog;
+import org.apache.iotdb.db.engine.settle.SettleLog.SettleCheckStatus;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+/**
+ * Offline Settle tool, which is used to settle TsFile and its corresponding mods file to a new
+ * TsFile.
+ */
+public class TsFileAndModSettleTool {
+  private static final Logger logger = LoggerFactory.getLogger(TsFileAndModSettleTool.class);
+  // TsFilePath -> SettleCheckStatus
+  public static Map<String, Integer> recoverSettleFileMap = new HashMap<>();
+
+  public static void main(String[] args) {
+    Map<String, TsFileResource> oldTsFileResources = new HashMap<>();
+    // List<TsFileResource> oldTsFileResources = new ArrayList<>();
+    findFilesToBeRecovered();
+    for (Map.Entry<String, Integer> entry : recoverSettleFileMap.entrySet()) {
+      String path = entry.getKey();
+      TsFileResource resource = new TsFileResource(new File(path));
+      resource.setClosed(true);
+      oldTsFileResources.put(resource.getTsFile().getName(), resource);
+    }
+    List<File> tsFiles = checkArgs(args);
+    if (tsFiles != null) {
+      for (File file : tsFiles) {
+        if (!oldTsFileResources.containsKey(file.getName())) {
+          if (new File(file + TsFileResource.RESOURCE_SUFFIX).exists()) {
+            TsFileResource resource = new TsFileResource(file);
+            resource.setClosed(true);
+            oldTsFileResources.put(file.getName(), resource);
+          }
+        }
+      }
+    }
+    System.out.println(
+        "Totally find "
+            + oldTsFileResources.size()
+            + " tsFiles to be settled, including "
+            + recoverSettleFileMap.size()
+            + " tsFiles to be recovered.");
+    settleTsFilesAndMods(oldTsFileResources);
+  }
+
+  public static List<File> checkArgs(String[] args) {
+    String filePath = "test.tsfile";
+    List<File> files = new ArrayList<>();
+    if (args.length == 0) {
+      return null;
+    } else {
+      for (String arg : args) {
+        if (arg.endsWith(TSFILE_SUFFIX)) { // it's a file
+          File f = new File(arg);
+          if (!f.exists()) {
+            logger.warn("Cannot find TsFile : " + arg);
+            continue;
+          }
+          files.add(f);
+        } else { // it's a dir
+          List<File> tmpFiles = getAllFilesInOneDirBySuffix(arg, TSFILE_SUFFIX);
+          if (tmpFiles == null) {
+            continue;
+          }
+          files.addAll(tmpFiles);
+        }
+      }
+    }
+    return files;
+  }
+
+  private static List<File> getAllFilesInOneDirBySuffix(String dirPath, String suffix) {
+    File dir = new File(dirPath);
+    if (!dir.isDirectory()) {
+      logger.warn("It's not a directory path : " + dirPath);
+      return null;
+    }
+    if (!dir.exists()) {
+      logger.warn("Cannot find Directory : " + dirPath);
+      return null;
+    }
+    List<File> tsFiles = new ArrayList<>();
+    tsFiles.addAll(
+        Arrays.asList(FSFactoryProducer.getFSFactory().listFilesBySuffix(dirPath, suffix)));
+    List<File> tmpFiles = Arrays.asList(dir.listFiles());
+    for (File f : tmpFiles) {
+      if (f.isDirectory()) {
+        tsFiles.addAll(getAllFilesInOneDirBySuffix(f.getAbsolutePath(), suffix));
+      }
+    }
+    return tsFiles;
+  }
+
+  /**
+   * This method is used to settle tsFiles and mods files, so that each old TsFile corresponds to
+   * one or several new TsFiles. This method is only applicable to V3 TsFile.
+   *
+   * @return Each old TsFile corresponds to one or several new TsFileResources of the new TsFiles
+   */
+  public static Map<String, List<TsFileResource>> settleTsFilesAndMods(
+      Map<String, TsFileResource> resourcesToBeSettled) {
+    int successCount = 0;
+    Map<String, List<TsFileResource>> newTsFileResources = new HashMap<>();
+    SettleLog.createSettleLog();
+    for (Map.Entry<String, TsFileResource> entry : resourcesToBeSettled.entrySet()) {
+      TsFileResource resourceToBeSettled = entry.getValue();
+      List<TsFileResource> settledTsFileResources = new ArrayList<>();
+      try {
+        TsFileAndModSettleTool tsFileAndModSettleTool = new TsFileAndModSettleTool();
+        System.out.println("Start settling for tsFile : " + resourceToBeSettled.getTsFilePath());
+        if (tsFileAndModSettleTool.isSettledFileGenerated(resourceToBeSettled)) {
+          settledTsFileResources = tsFileAndModSettleTool.findSettledFile(resourceToBeSettled);
+          newTsFileResources.put(resourceToBeSettled.getTsFile().getName(), settledTsFileResources);
+        } else {
+          // Write Settle Log, Status 1
+          SettleLog.writeSettleLog(
+              resourceToBeSettled.getTsFilePath()
+                  + SettleLog.COMMA_SEPERATOR
+                  + SettleCheckStatus.BEGIN_SETTLE_FILE);
+          tsFileAndModSettleTool.settleOneTsFileAndMod(resourceToBeSettled, settledTsFileResources);
+          // Write Settle Log, Status 2
+          SettleLog.writeSettleLog(
+              resourceToBeSettled.getTsFilePath()
+                  + SettleLog.COMMA_SEPERATOR
+                  + SettleCheckStatus.AFTER_SETTLE_FILE);
+          newTsFileResources.put(resourceToBeSettled.getTsFile().getName(), settledTsFileResources);
+        }
+
+        moveNewTsFile(resourceToBeSettled, settledTsFileResources);
+        // Write Settle Log, Status 3
+        SettleLog.writeSettleLog(
+            resourceToBeSettled.getTsFilePath()
+                + SettleLog.COMMA_SEPERATOR
+                + SettleCheckStatus.SETTLE_SUCCESS);
+        System.out.println(
+            "Finish settling successfully for tsFile : " + resourceToBeSettled.getTsFilePath());
+        successCount++;
+      } catch (Exception e) {
+        System.out.println(
+            "Meet error while settling the tsFile : " + resourceToBeSettled.getTsFilePath());
+        e.printStackTrace();
+      }
+    }
+    if (resourcesToBeSettled.size() == successCount) {
+      SettleLog.closeLogWriter();
+      FSFactoryProducer.getFSFactory().getFile(SettleLog.getSettleLogPath()).delete();
+      System.out.println("Finish settling all tsfiles Successfully!");
+    } else {
+      System.out.println(
+          "Finish Settling, "
+              + (resourcesToBeSettled.size() - successCount)
+              + " tsfiles meet errors.");
+    }
+    return newTsFileResources;
+  }
+
+  /**
+   * The size of settledResources will be 0 in one of the following conditions: (1) old TsFile is
+   * not closed (2) old ModFile is not existed (3) all data in the old tsfile is being deleted after
+   * settling
+   */
+  public void settleOneTsFileAndMod(
+      TsFileResource resourceToBeSettled, List<TsFileResource> settledResources) throws Exception {
+    if (!resourceToBeSettled.isClosed()) {
+      logger.warn(
+          "The tsFile {} should be sealed when rewritting.", resourceToBeSettled.getTsFilePath());
+      return;
+    }
+    // if no deletions to this tsfile, then return.
+    if (!resourceToBeSettled.getModFile().exists()) {
+      return;
+    }
+    try (TsFileRewriteTool tsFileRewriteTool = new TsFileRewriteTool(resourceToBeSettled)) {
+      tsFileRewriteTool.parseAndRewriteFile(settledResources);
+    }
+    if (settledResources.size() == 0) { // if all the data in this tsfile has been deleted
+      resourceToBeSettled.readUnlock();
+      resourceToBeSettled.writeLock();
+      resourceToBeSettled.delete();
+      resourceToBeSettled.writeUnlock();
+      resourceToBeSettled.readLock();
+    }
+    return;
+  }
+
+  public static void findFilesToBeRecovered() {
+    if (FSFactoryProducer.getFSFactory().getFile(SettleLog.getSettleLogPath()).exists()) {
+      try (BufferedReader settleLogReader =
+          new BufferedReader(
+              new FileReader(
+                  FSFactoryProducer.getFSFactory().getFile(SettleLog.getSettleLogPath())))) {
+        String line = null;
+        while ((line = settleLogReader.readLine()) != null && !line.equals("")) {
+          String oldFilePath = line.split(SettleLog.COMMA_SEPERATOR)[0];
+          Integer settleCheckStatus = Integer.parseInt(line.split(SettleLog.COMMA_SEPERATOR)[1]);
+          if (settleCheckStatus == SettleCheckStatus.SETTLE_SUCCESS.getCheckStatus()) {
+            recoverSettleFileMap.remove(oldFilePath);
+            continue;
+          }
+          recoverSettleFileMap.put(oldFilePath, settleCheckStatus);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "meet error when recover settle process, file path:{}",
+            SettleLog.getSettleLogPath(),
+            e);
+      }
+    }
+  }
+
+  /** this method is used to check whether the new file is settled when recovering old tsFile. */
+  public boolean isSettledFileGenerated(TsFileResource oldTsFileResource) {
+    String oldFilePath = oldTsFileResource.getTsFilePath();
+    return TsFileAndModSettleTool.recoverSettleFileMap.containsKey(oldFilePath)
+        && TsFileAndModSettleTool.recoverSettleFileMap.get(oldFilePath)
+            == SettleCheckStatus.AFTER_SETTLE_FILE.getCheckStatus();
+  }
+
+  /** when the new file is settled , we need to find and deserialize it. */
+  public List<TsFileResource> findSettledFile(TsFileResource resourceToBeSettled)
+      throws IOException {
+    List<TsFileResource> settledTsFileResources = new ArrayList<>();
+    SettleLog.writeSettleLog(
+        resourceToBeSettled.getTsFilePath()
+            + SettleLog.COMMA_SEPERATOR
+            + SettleCheckStatus.BEGIN_SETTLE_FILE);

Review comment:
       Since the new file is already settled, why should we change the log status of it back to BEGIN_SETTLE_FILE




-- 
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



[GitHub] [iotdb] HTHou commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r718998318



##########
File path: jenkins.pom
##########
@@ -19,49 +19,49 @@
     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">
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xmlns="http://maven.apache.org/POM/4.0.0"

Review comment:
       Why do you change the order?




-- 
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



[GitHub] [iotdb] wangchao316 commented on a change in pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
wangchao316 commented on a change in pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#discussion_r715418428



##########
File path: docs/zh/SystemDesign/Tools/Settle.md
##########
@@ -0,0 +1,96 @@
+## TsFile整理工具
+
+### 介绍
+
+1. 整理工具可以帮助你整理已封口的TsFile文件和.mods文件,过滤掉删除的数据并生成新的TsFile。整理完后会将本地旧的.mods文件删除,若某TsFile里的数据被全部删除了,则会删除本地该TsFile文件和对应的.resource文件。
+
+2. 整理工具只能针对v0.12版本的IOTDB来使用,即只能针对版本为V3的TsFile进行整理,若低于此版本则必须先用在线升级工具将TsFile升级到V3版本。
+3. TsFile整理工具分为在线整理工具和离线整理工具,两者在整理的时候都会记录日志,供下次恢复先前整理失败的文件。在执行整理的时候会先去检测日志里是否存在失败的文件,有的话则优先整理它们。
+
+### 1. 离线整理工具
+
+离线整理工具是以命令行的方式来启动,该工具的启动脚本settle.bat和settle.sh在编译了server后会生成至server\target\iotdb-server-{version}\tools\tsfileToolSet目录中。在使用的时候必须保证IOTDB Server是停止运行的,否则会出现整理错误。具体使用方式详见用户手册。
+
+#### 1.1 涉及的相关类
+
+整理工具类:org.apache.iotdb.db.tools.settle.TsFileAndModSettleTool
+
+### 2.在线整理工具
+
+在线整理工具是当用户在IOTDB客户端输入了settle命令后,会在后台注册启动一个整理服务Settle Service,该服务会去寻找指定存储组下的所有TsFile文件,并为每个TsFile开启一个整理线程进行整理。下面讲解整理线程的工作流程。
+
+#### 2.1 整理线程的工作流程
+
+1. 首先对该旧TsFile文件**加读锁**,避免在重写的过程中该TsFile被删除,而在删除该TsFile的本地mods文件前允许用户对该TsFile进行查询操作。【该步骤在StorageGroupProcessor类的settle()方法里】
+

Review comment:
       Thanks for your contribution. 
   I have some question. 
   If the tsfile is being merged, it needs to be deleted. What about this tsfile? Do you want to delete or not delete?
   I suggest  we should use hardlink for tsfile, when settle tsfile.
   
   This function minimizes the impact on other functions,
   In addition, compact is a frequent function.
   




-- 
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



[GitHub] [iotdb] coveralls edited a comment on pull request #4024: [IOTDB-1635] settle TsFiles and mods

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #4024:
URL: https://github.com/apache/iotdb/pull/4024#issuecomment-926447982


   
   [![Coverage Status](https://coveralls.io/builds/43046393/badge)](https://coveralls.io/builds/43046393)
   
   Coverage decreased (-0.01%) to 67.545% when pulling **dc5620dba0cd5648e692c36d2cae9a01de4d8bd0 on choubenson:settleTsFileAndMod** into **1b7add158a99052e6dfaf1fb38314ead76d20d2b on apache:master**.
   


-- 
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