You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by lt...@apache.org on 2019/10/29 01:47:12 UTC

[incubator-iotdb] branch master updated: [IOTDB-198]Reimplementation sync module (#372)

This is an automated email from the ASF dual-hosted git repository.

lta pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 5dcb366  [IOTDB-198]Reimplementation sync module (#372)
5dcb366 is described below

commit 5dcb3660b04b299f15d832a6641c5e227632f283
Author: Tianan Li <li...@163.com>
AuthorDate: Tue Oct 29 09:47:06 2019 +0800

    [IOTDB-198]Reimplementation sync module (#372)
    
    * add sync new code framework
    
    * complete manage module
    
    * complete all sender module
    
    * complete sync sender module
    
    * complete sync receiver end
    
    * complete load module in receiver end
    
    * add load log and clean up in receiver end
    
    * add sync log recovery in receiver end
    
    * add sync sender log analyzer
    
    * complete load tsfiles module
    
    * add snapshot unit test
    
    * add load tsfile unit test
    
    * add log analyzer unit test
    
    * finish file renaming for loaded sequence tsfiles
    
    * fix an acute bug due to unmodifiable variable publishing
    
    * fix some acute bug in testing
    
    * fix a bug in windows environment
    
    * modify log level fron info to error
    
    * add default properties
    
    * replace interface with concrete class
    
    * add definition about appropriate
    
    * update memory estimation tool doc
    
    * fix a bug of ip issue
    
    * fix issues according to pr reviews
    
    * add reject policy while there are two sender sync data of the same device to the same receiver
---
 .../UserGuide/9-System Tools/1-Sync Tool.md        |  58 +-
 .../9-System Tools/2-Memory Estimation Tool.md     |   4 +-
 .../UserGuide/9-System Tools/1-Sync Tool.md        |  85 ++-
 .../9-System Tools/2-Memory Estimation Tool.md     |   4 +-
 .../resources/conf/iotdb-engine.properties         |   8 -
 .../resources/conf/iotdb-sync-client.properties    |  16 +-
 server/src/assembly/resources/conf/logback.xml     |  20 +
 .../assembly/resources/tools/start-sync-client.bat |   2 +-
 .../assembly/resources/tools/start-sync-client.sh  |   2 +-
 .../org/apache/iotdb/db/concurrent/ThreadName.java |   1 +
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |  14 -
 .../org/apache/iotdb/db/conf/IoTDBConstant.java    |   5 +
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |   3 -
 .../db/conf/adapter/IoTDBConfigDynamicAdapter.java |   8 +-
 .../db/conf/directories/DirectoryManager.java      |   8 +-
 .../org/apache/iotdb/db/engine/StorageEngine.java  |  14 +-
 .../iotdb/db/engine/merge/task/MergeFileTask.java  |  32 +-
 .../engine/storagegroup/StorageGroupProcessor.java | 259 +++++++-
 .../db/engine/storagegroup/TsFileResource.java     |   4 +
 .../SyncDeviceOwnerConflictException.java}         |  25 +-
 .../org/apache/iotdb/db/metadata/MManager.java     |   4 +-
 .../universal/CachedPriorityMergeReader.java       |  26 +-
 .../reader/universal/PriorityMergeReader.java      |   2 +-
 .../sync/conf/{Constans.java => SyncConstant.java} |  50 +-
 .../iotdb/db/sync/conf/SyncSenderConfig.java       | 145 ++--
 .../iotdb/db/sync/conf/SyncSenderDescriptor.java   |  53 +-
 .../org/apache/iotdb/db/sync/package-info.java     |  39 ++
 .../iotdb/db/sync/receiver/SyncServerManager.java  |  17 +-
 .../iotdb/db/sync/receiver/SyncServiceImpl.java    | 737 ---------------------
 .../iotdb/db/sync/receiver/load/FileLoader.java    | 234 +++++++
 .../db/sync/receiver/load/FileLoaderManager.java   | 215 ++++++
 .../iotdb/db/sync/receiver/load/IFileLoader.java   |  60 ++
 .../iotdb/db/sync/receiver/load/ILoadLogger.java   |  62 ++
 .../iotdb/db/sync/receiver/load/LoadLogger.java    |  72 ++
 .../iotdb/db/sync/receiver/load/LoadType.java      |  61 +-
 .../receiver/recover/ISyncReceiverLogAnalyzer.java |  72 +-
 .../sync/receiver/recover/ISyncReceiverLogger.java |  54 ++
 .../receiver/recover/SyncReceiverLogAnalyzer.java  | 153 +++++
 .../sync/receiver/recover/SyncReceiverLogger.java  |  72 ++
 .../db/sync/receiver/transfer/SyncServiceImpl.java | 304 +++++++++
 .../iotdb/db/sync/sender/SyncFileManager.java      | 208 ------
 .../apache/iotdb/db/sync/sender/SyncSender.java    |  75 ---
 .../iotdb/db/sync/sender/SyncSenderImpl.java       | 545 ---------------
 .../db/sync/sender/manage/ISyncFileManager.java    |  67 ++
 .../db/sync/sender/manage/SyncFileManager.java     | 198 ++++++
 .../sender/recover/ISyncSenderLogAnalyzer.java     |  52 ++
 .../db/sync/sender/recover/ISyncSenderLogger.java  |  65 ++
 .../sync/sender/recover/SyncSenderLogAnalyzer.java | 128 ++++
 .../db/sync/sender/recover/SyncSenderLogger.java   |  72 ++
 .../sync/sender/transfer/DataTransferManager.java  | 667 +++++++++++++++++++
 .../sync/sender/transfer/IDataTransferManager.java | 104 +++
 .../java/org/apache/iotdb/db/utils/SyncUtils.java  |  40 +-
 .../apache/iotdb/db/engine/merge/MergeLogTest.java |   1 -
 .../iotdb/db/engine/merge/MergeOverLapTest.java    | 151 +++++
 .../apache/iotdb/db/engine/merge/MergeTest.java    |  21 +-
 .../storagegroup/FileNodeManagerBenchmark.java     |   2 +-
 .../db/sync/receiver/load/FileLoaderTest.java      | 330 +++++++++
 .../recover/SyncReceiverLogAnalyzerTest.java       | 212 ++++++
 .../receiver/recover/SyncReceiverLoggerTest.java   | 112 ++++
 .../db/sync/sender/MultipleClientSyncTest.java     | 226 -------
 .../iotdb/db/sync/sender/SingleClientSyncTest.java | 559 ----------------
 .../iotdb/db/sync/sender/SyncFileManagerTest.java  | 374 -----------
 .../db/sync/sender/manage/SyncFileManagerTest.java | 295 +++++++++
 .../sender/recover/SyncSenderLogAnalyzerTest.java  | 163 +++++
 .../sync/sender/recover/SyncSenderLoggerTest.java  | 111 ++++
 .../sender/transfer/DataTransferManagerTest.java   | 140 ++++
 .../apache/iotdb/db/sync/test/SyncTestClient1.java | 253 -------
 .../apache/iotdb/db/sync/test/SyncTestClient2.java | 262 --------
 .../apache/iotdb/db/sync/test/SyncTestClient3.java | 282 --------
 .../iotdb/db/{sync/test => utils}/RandomNum.java   |   2 +-
 service-rpc/src/main/thrift/sync.thrift            |  22 +-
 .../iotdb/tsfile/common/conf/TSFileConfig.java     |   1 +
 72 files changed, 4812 insertions(+), 3932 deletions(-)

diff --git a/docs/Documentation-CHN/UserGuide/9-System Tools/1-Sync Tool.md b/docs/Documentation-CHN/UserGuide/9-System Tools/1-Sync Tool.md
index eef403e..7b83bb4 100644
--- a/docs/Documentation-CHN/UserGuide/9-System Tools/1-Sync Tool.md	
+++ b/docs/Documentation-CHN/UserGuide/9-System Tools/1-Sync Tool.md	
@@ -23,8 +23,10 @@
 # 同步工具
 <!-- TOC -->
 
+- [第8章: 系统工具](#第8章-系统工具)
 - [同步工具](#同步工具)
 - [介绍](#介绍)
+- [应用场景](#应用场景)
 - [配置参数](#配置参数)
     - [同步工具接收端](#同步工具接收端)
     - [同步工具发送端](#同步工具发送端)
@@ -36,7 +38,7 @@
 
 <!-- /TOC -->
 # 介绍
-同步工具是定期将本地磁盘中和新增的已持久化的tsfile文件上传至云端并加载到IoTDB套件工具。
+同步工具是定期将本地磁盘中和新增的已持久化的tsfile文件上传至云端并加载到IoTDB的套件工具。
 
 在同步工具的发送端,同步模块是一个独立的进程,独立于本地的IoTDB。通过独立的脚本进行启动和关闭(详见章节`使用方式`),同步的频率周期可由用户设置。
 
@@ -45,6 +47,13 @@
 同步工具具有多对一的发送-接受模式,即一个同步接收端可以同时接受多个同步发送端传输的数据,一个同步发送端只能向一个同步接收端发送数据
 
 > 注意:在使用同步工具前,同步工具的接收端和发送端需要单独配置。
+# 应用场景
+以一个工厂应用为例,通常有多个分厂和多个总厂,每个分厂中使用一个IoTDB实例收集数据,然后将数据定时汇总到总厂中进行备份或者分析等,一个总厂可以接收来自多个分厂的数据,一个分厂也可以给多个总厂同步数据,在这种场景下每个IoTDB实例所管理的设备各不相同。
+
+在sync模块中,每个分厂是发送端,总厂是接收端,发送端定时将数据同步给接收端,在上述应用场景下一个设备的数据只能由一个发送端来收集,因此多个发送端同步的数据之间必须是没有设备重叠的,否则不符合sync功能的应用场景。
+
+当出现异常场景时,即两个或两个以上的发送端向同一个接收端同步相同设备(其存储组设为root.sg)的数据时,后被接收端收到的含有该设备数据的发送端的root.sg数据将会被拒绝接收。示例:发动端1向接收端同步存储组root.sg1和root.sg2, 发动端2向接收端同步存储组root.sg2和root.sg3, 
+均包括时间序列root.sg2.d0.s0, 若接收端先接收到发送端1的root.sg2.d0.s0的数据,那么接收端将拒绝发送端2的root.sg2同步的数据。
 # 配置参数
 ## 同步工具接收端
 同步工具接收端的参数配置位于IoTDB的配置文件iotdb-engine.properties中,其安装目录为$IOTDB_HOME/conf/iotdb-engine.properties。在该配置文件中,有四个参数和同步接收端有关,配置说明如下:
@@ -63,7 +72,7 @@
    </tr>
    <tr>
       <td>默认值</td>
-      <td>false</td>
+      <td>true</td>
    </tr>
    <tr>
       <td>改后生效方式</td>
@@ -95,29 +104,6 @@
 
 <table>
    <tr>
-      <td colspan="2">参数名: update_historical_data_possibility</td>
-   </tr>
-   <tr>
-      <td width="20%">描述</td>
-      <td>同步服务端在合并同步的数据时选择的处理策略。如果同步的数据对历史数据(相比本地该存储组数据的最新时间戳)更新占比超过50%,则建议选择策略1,将参数设置为true,使用该策略对IoTDB系统的写入性能产生较大影响,对机器的CPU占用较小;如果同步的数据对历史数据更新占比少于50%,则建议选择策略2,将参数设置为false,使用该策略对IoTDB系统的写入性能产生较小影响,对机器CPU的占用较大。<br/>
-</td>
-   </tr>
-   <tr>
-      <td>类型</td>
-      <td>Boolean</td>
-   </tr>
-   <tr>
-      <td>默认值</td>
-      <td>false</td>
-   </tr>
-   <tr>
-      <td>改后生效方式</td>
-      <td>重启服务器生效</td>
-   </tr>
-</table>
-
-<table>
-   <tr>
       <td colspan="2">参数名: sync_server_port</td>
    </tr>
    <tr>
@@ -208,17 +194,21 @@
 
 <table>
    <tr>
-      <td colspan="2">参数名: iotdb_schema_directory</td>
+      <td colspan="2">参数名: sync_storage_groups</td>
    </tr>
    <tr>
       <td width="20%">描述</td>
-      <td>同步发送端的IoTDB schema文件的绝对路径,例如$IOTDB_HOME /data/system/schema/mlog.txt(若用户未手动设置schema元数据的路径,则该路径为默认路径),该参数默认不生效,用户有需求时进行手动设置</td>
+      <td>进行同步的存储组列表,存储组间用逗号分隔;若列表设置为空表示同步所有存储组,默认为空</td>
    </tr>
    <tr>
       <td>类型</td>
       <td>String</td>
    </tr>
    <tr>
+      <td>示例</td>
+      <td>root.sg1, root.sg2</td>
+   </tr>
+   <tr>
       <td>改后生效方式</td>
       <td>重启同步功能发送端生效</td>
    </tr>
@@ -226,15 +216,19 @@
 
 <table>
    <tr>
-      <td colspan="2">参数名: iotdb_bufferWrite_directory</td>
+      <td colspan="2">参数名: max_number_of_sync_file_retry</td>
    </tr>
    <tr>
       <td width="20%">描述</td>
-      <td>同步发送端的IoTDB 的bufferWrite数据(tsfile文件)目录的绝对路径,定位至bufferWrite目录下,例如: $IOTDB_HOME /data/data/settled(若用户未手动设置数据路径,则该路径为默认路径),该参数默认不生效,用户有需求时进行手动设置。该参数需要保证和参数iotdb_schema_directory属于同一个IoTDB</td>
+      <td>发送端同步文件到接收端失败时的最大重试次数</td>
    </tr>
    <tr>
       <td>类型</td>
-      <td>String</td>
+      <td>Int : [0,2147483647]</td>
+   </tr>
+   <tr>
+      <td>示例</td>
+      <td>5</td>
    </tr>
    <tr>
       <td>改后生效方式</td>
@@ -245,7 +239,7 @@
 # 使用方式
 ## 启动同步功能接收端
 1. 配置接收端的参数,例如:
-<img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/26211279/59494502-daaa4380-8ebf-11e9-8bce-363e2433005a.png">
+<img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/26211279/64172919-a32cb100-ce88-11e9-821c-33369bff6d34.png">
 2. 启动IoTDB引擎,同步功能接收端会同时启动,启动时LOG日志会出现`IoTDB: start SYNC ServerService successfully`字样,表示同步接收端启动成功,如图所示:
 <img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/26211279/59494513-df6ef780-8ebf-11e9-83e1-ee8ae64b76d0.png">
 
@@ -254,7 +248,7 @@
 
 ## 启动同步功能发送端
 1. 配置发送端的参数, 如图所示:
-<img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/26211279/59494559-f9a8d580-8ebf-11e9-875e-355199c1a1e9.png">
+<img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/26211279/64172668-15e95c80-ce88-11e9-9700-dff7daf06bb7.png">
 2. 启动同步功能发送端
 
 用户可以使用```$IOTDB_HOME/bin```文件夹下的脚本启动同步功能的发送端
diff --git a/docs/Documentation-CHN/UserGuide/9-System Tools/2-Memory Estimation Tool.md b/docs/Documentation-CHN/UserGuide/9-System Tools/2-Memory Estimation Tool.md
index 9d19993..925257c 100644
--- a/docs/Documentation-CHN/UserGuide/9-System Tools/2-Memory Estimation Tool.md	
+++ b/docs/Documentation-CHN/UserGuide/9-System Tools/2-Memory Estimation Tool.md	
@@ -75,7 +75,7 @@ Linux系统与MacOS系统启动命令如下:
 ```
 * 以20个存储组,共10w条时间序列,存储组中最大时间序列数为50000为例:
 ```
-  Shell >$IOTDB_HOME/bin/memory-tool.sh calmem -sg 20 -ts 100000 -tsm -50000
+  Shell >$IOTDB_HOME/bin/memory-tool.sh calmem -sg 20 -ts 100000 -mts -50000
 ```
 
 Windows系统启动命令如下:
@@ -85,6 +85,6 @@ Windows系统启动命令如下:
 ```
 * 以20个存储组,共10w条时间序列,存储组中最大时间序列数为50000为例:
 ```
-  Shell >$IOTDB_HOME\bin\memory-tool.bat calmem -sg 20 -ts 100000 -tsm -50000
+  Shell >$IOTDB_HOME\bin\memory-tool.bat calmem -sg 20 -ts 100000 -mts -50000
 ```
 
diff --git a/docs/Documentation/UserGuide/9-System Tools/1-Sync Tool.md b/docs/Documentation/UserGuide/9-System Tools/1-Sync Tool.md
index 1aa14fa..c04129d 100644
--- a/docs/Documentation/UserGuide/9-System Tools/1-Sync Tool.md	
+++ b/docs/Documentation/UserGuide/9-System Tools/1-Sync Tool.md	
@@ -23,6 +23,22 @@
 
 ## Data Import
 
+<!-- TOC -->
+
+- [Chapter 8: System Tools](#chapter-8-system-tools)
+    - [Data Import](#data-import)
+- [Introduction](#introduction)
+- [Application Scenario](#application-scenario)
+- [Configuration](#configuration)
+    - [Sync Receiver](#sync-receiver)
+    - [Sync Sender](#sync-sender)
+- [Usage](#usage)
+    - [Start Sync Receiver](#start-sync-receiver)
+    - [Stop Sync Receiver](#stop-sync-receiver)
+    - [Start Sync Sender](#start-sync-sender)
+    - [Stop Sync Sender](#stop-sync-sender)
+
+<!-- /TOC -->
 # Introduction
 The Sync Tool is an IoTDB suite tool that periodically uploads persistent tsfiles from the sender disk to the receiver and loads them.
 
@@ -33,6 +49,13 @@ On the receiver side of the sync, the sync module is embedded in the engine of I
 The sync tool has a many-to-one sender-receiver mode - that is, one sync receiver can receive data from multiple sync senders simultaneously while one sync sender can only send data to one sync receiver.
 
 > Note: Before using the sync tool, the client and server need to be configured separately. The configuration is detailed in Sections Configuration.
+# Application Scenario
+In the case of a factory application, there are usually multiple sub-factories and multiple general(main) factories. Each sub-factory uses an IoTDB instance to collect data, and then synchronize the data to the general factory for backup or analysis. A general factory can receive data from multiple sub-factories and a sub-factory can also synchronize data to multiple general factories. In this scenario, each IoTDB instance manages different devices. 
+      
+In the sync module, each sub-factory is a sender, a general factory is a receiver, and senders periodically synchronizes the data to receivers. In the above application scenario, the data of one device can only be collected by one sender, so there is no device overlap between the data synchronized by multiple senders. Otherwise, the application scenario of the sync module is not satisfied.
+
+When there is an abnormal scenario, namely, two or more senders synchronize the data of the same device (whose storage group is set as root.sg) to the same receiver, the root.sg data of the sender containing the device data received later by the receiver will be rejected. Example: the engine 1 synchronizes the storage groups root.sg1 and root.sg2 to the receiver, and the engine 2 synchronizes the storage groups root.sg2 and root.sg3 to the receiver. All of them include the time series ro [...]
+If the receiver receives the data of root.sg2.d0.s0 of the sender 1 first, the receiver will reject the data of root.sg2 of the sender 2.
 # Configuration
 ## Sync Receiver
 The parameter configuration of the sync receiver is located in the configuration file `iotdb-engine.properties` of IoTDB, and its directory is `$IOTDB_HOME/conf/iotdb-engine.properties`. In this configuration file, there are four parameters related to the sync receiver. The configuration instructions are as follows:
@@ -83,31 +106,6 @@ The parameter configuration of the sync receiver is located in the configuration
 
 <table>
    <tr>
-      <td colspan="2">parameter: update_historical_data_possibility</td>
-   </tr>
-   <tr>
-      <td width="30%">Description</td>
-      <td>The processing strategy chosen by the sync receiver when merging the sync data.<br/>
-        1. If the sync data accounts for more than 50% of the update of the historical data (compared with the latest timestamp of the local storage group data),then it is recommended this parameter be set to TRUE, which has a greater impact on the write performance and reduce CPU usage.<br/>
-        2. If the sync data accounts for less than 50% of the update of the historical data (compared with the latest timestamp of the local storage group data),then it is recommended this parameter be set to FALSE,which has little impact on the write performance and takes up a large amount of CPU power.<br/>
-</td>
-   </tr>
-   <tr>
-      <td>Type</td>
-      <td>Boolean</td>
-   </tr>
-   <tr>
-      <td>Default</td>
-      <td>false</td>
-   </tr>
-   <tr>
-      <td>Modalities for Entry into Force after Modification</td>
-      <td>Restart receiver</td>
-   </tr>
-</table>
-
-<table>
-   <tr>
       <td colspan="2">parameter: sync_server_port</td>
    </tr>
    <tr>
@@ -216,26 +214,53 @@ The parameters of the sync sender are configured in a separate configuration fil
 
 <table>
    <tr>
-      <td colspan="2">parameter: iotdb_bufferWrite_directory</td>
+      <td colspan="2">parameter: sync_storage_groups</td>
    </tr>
    <tr>
-      <td width="30%">Description</td>
-      <td>The absolute path of the buffer write data (tsfile file) directory of the IoTDB at the sender, such as: $IOTDB_HOME/data/data/settled (if the user does not set the data path manually, the path is the default path of IoTDB engine). This parameter is not valid by default, and is set manually when the user needs it. This parameter needs to be guaranteed to belong to the same IoTDB as the parameter iotdb_schema_directory.</td>
+      <td width="20%">Description</td>
+      <td>This parameter represents storage groups that participate in the synchronization task, which distinguishes each storage group by comma. If the list is empty, it means that all storage groups participate in synchronization. By default, it is an empty list.</td>
    </tr>
    <tr>
       <td>Type</td>
       <td>String</td>
    </tr>
    <tr>
+      <td>Example</td>
+      <td>root.sg1, root.sg2</td>
+   </tr>
+   <tr>
+      <td>Modalities for Entry into Force after Modification</td>
+      <td>Restart client</td>
+   </tr>
+</table>
+
+<table>
+   <tr>
+      <td colspan="2">parameter: max_number_of_sync_file_retry</td>
+   </tr>
+   <tr>
+      <td width="20%">Description</td>
+      <td>The maximum number of retry when syncing a file to receiver fails.</td>
+   </tr>
+   <tr>
+      <td>Type</td>
+      <td>Int : [0,2147483647]</td>
+   </tr>
+   <tr>
+      <td>Example</td>
+      <td>5</td>
+   </tr>
+   <tr>
       <td>Modalities for Entry into Force after Modification</td>
       <td>Restart client</td>
    </tr>
 </table>
 
+
 # Usage
 ## Start Sync Receiver
 1. Set up parameters of sync receiver. For example:
-<img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/26211279/59494502-daaa4380-8ebf-11e9-8bce-363e2433005a.png">
+<img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/26211279/64172919-a32cb100-ce88-11e9-821c-33369bff6d34.png">
 2. Start IoTDB engine, and the sync receiver will start at the same time, and the LOG log will start with the sentence `IoTDB: start SYNC ServerService successfully` indicating the successful start of the return receiver.
 <img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/26211279/59494513-df6ef780-8ebf-11e9-83e1-ee8ae64b76d0.png">
 
@@ -244,7 +269,7 @@ Stop IoTDB and the sync receiver will be closed at the same time.
 
 ## Start Sync Sender
 1. Set up parameters of sync sender. For example:
-<img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/26211279/59494559-f9a8d580-8ebf-11e9-875e-355199c1a1e9.png">
+<img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/26211279/64172668-15e95c80-ce88-11e9-9700-dff7daf06bb7.png">
 2. Start sync sender
 Users can use the scripts under the ```$IOTDB_HOME/bin``` folder to start the sync sender.
 For Linux and Mac OS X users:
diff --git a/docs/Documentation/UserGuide/9-System Tools/2-Memory Estimation Tool.md b/docs/Documentation/UserGuide/9-System Tools/2-Memory Estimation Tool.md
index 46a406c..00397ca 100644
--- a/docs/Documentation/UserGuide/9-System Tools/2-Memory Estimation Tool.md	
+++ b/docs/Documentation/UserGuide/9-System Tools/2-Memory Estimation Tool.md	
@@ -69,7 +69,7 @@ For Linux and Mac OS X users:
 ```
 * Assume that there are 20 storage groups, 10w timeseries and maximum timeseries number among storage groups is 50000:
 ```
-  Shell >$IOTDB_HOME/bin/memory-tool.sh calmem -sg 20 -ts 100000 -tsm -50000
+  Shell >$IOTDB_HOME/bin/memory-tool.sh calmem -sg 20 -ts 100000 -mts -50000
 ```
 
 For Windows users:
@@ -79,6 +79,6 @@ For Windows users:
 ```
 * Assume that there are 20 storage groups, 10w timeseries and maximum timeseries number among storage groups is 50000:
 ```
-  Shell >$IOTDB_HOME\bin\memory-tool.bat calmem -sg 20 -ts 100000 -tsm -50000
+  Shell >$IOTDB_HOME\bin\memory-tool.bat calmem -sg 20 -ts 100000 -mts -50000
 ```
 
diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties
index 6e68394..f5f6419 100644
--- a/server/src/assembly/resources/conf/iotdb-engine.properties
+++ b/server/src/assembly/resources/conf/iotdb-engine.properties
@@ -283,14 +283,6 @@ sync_server_port=5555
 # The default is to allow all IP to sync
 ip_white_list=0.0.0.0/0
 
-# The processing strategy chosen by the sync server when merging the sync data.
-# 1. If the sync data accounts for more than 50% of the update of the historical data (compared with the latest timestamp of the local storage group data),then it is recommended to select strategy 1.
-#    Setting the parameter to true, which has a greater impact on the insert performance of the IoTDB system and occupies less CPU of the machine.
-# 2. If the sync data accounts for less than 50% of the update of the historical data (compared with the latest timestamp of the local storage group data),then it is recommended to select strategy 2.
-#    Setting the parameter to false, which has little impact on the insert performance of IoTDB system and takes up a large amount of CPU power.
-update_historical_data_possibility=false
-
-
 ####################
 ### performance statistic configuration
 ####################
diff --git a/server/src/assembly/resources/conf/iotdb-sync-client.properties b/server/src/assembly/resources/conf/iotdb-sync-client.properties
index 65b3074..f975f3b 100644
--- a/server/src/assembly/resources/conf/iotdb-sync-client.properties
+++ b/server/src/assembly/resources/conf/iotdb-sync-client.properties
@@ -17,19 +17,19 @@
 # under the License.
 #
 
-# Sync server port address
+# Sync receiver server address
 server_ip=127.0.0.1
 
-# Sync client port
+# Sync receiver server port
 server_port=5555
 
 # The period time of sync process, the time unit is second.
 sync_period_in_second=600
 
-# Set bufferWrite data absolute path of IoTDB
-# It needs to be set with iotdb_schema_directory, they have to belong to the same IoTDB
-# iotdb_bufferWrite_directory = D:\\iotdb\\data\\data\\settled
+# This parameter represents storage groups that participate in the synchronization task, which distinguishes each storage group by comma.
+# If the list is empty, it means that all storage groups participate in synchronization.
+# By default, it is empty list.
+# sync_storage_groups = root.sg1, root.sg2
 
-# Set schema file absolute path of IoTDB
-# It needs to be set with iotdb_bufferWrite_directory, they have to belong to the same IoTDB
-# iotdb_schema_directory = D:\\iotdb\\data\\system\\schema\\mlog.txt
+# The maximum number of retry when syncing a file to receiver fails.
+max_number_of_sync_file_retry=5
diff --git a/server/src/assembly/resources/conf/logback.xml b/server/src/assembly/resources/conf/logback.xml
index 796dcd6..e199641 100644
--- a/server/src/assembly/resources/conf/logback.xml
+++ b/server/src/assembly/resources/conf/logback.xml
@@ -146,6 +146,23 @@
             <level>INFO</level>
         </filter>
     </appender>
+    <appender class="ch.qos.logback.core.rolling.RollingFileAppender" name="FILE_DYNAMIC_PARAMETER">
+        <file>${IOTDB_HOME}/logs/log_dynamic_adapter.log</file>
+        <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+            <fileNamePattern>${IOTDB_HOME}/logs/log-measure-%d{yyyy-MM-dd}.%i.log</fileNamePattern>
+            <timeBasedFileNamingAndTriggeringPolicy class="ch.qos.logback.core.rolling.SizeAndTimeBasedFNATP">
+                <maxFileSize>200MB</maxFileSize>
+            </timeBasedFileNamingAndTriggeringPolicy>
+        </rollingPolicy>
+        <append>true</append>
+        <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+            <pattern>%d [%t] %-5p %C:%L - %m %n</pattern>
+            <charset>utf-8</charset>
+        </encoder>
+        <filter class="ch.qos.logback.classic.filter.ThresholdFilter">
+            <level>DEBUG</level>
+        </filter>
+    </appender>
     <root level="info">
         <appender-ref ref="FILEDEBUG"/>
         <appender-ref ref="FILEWARN"/>
@@ -156,4 +173,7 @@
     <logger level="info" name="org.apache.iotdb.db.cost.statistic">
         <appender-ref ref="FILE_COST_MEASURE"/>
     </logger>
+    <logger level="debug" name="org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter">
+        <appender-ref ref="FILE_DYNAMIC_PARAMETER"/>
+    </logger>
 </configuration>
diff --git a/server/src/assembly/resources/tools/start-sync-client.bat b/server/src/assembly/resources/tools/start-sync-client.bat
index 49360ef..dc44fcd 100755
--- a/server/src/assembly/resources/tools/start-sync-client.bat
+++ b/server/src/assembly/resources/tools/start-sync-client.bat
@@ -29,7 +29,7 @@ set IOTDB_CONF=%IOTDB_HOME%\conf
 set IOTDB_LOGS=%IOTDB_HOME%\logs
 
 
-if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.db.sync.sender.SyncSenderImpl
+if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.db.sync.sender.transfer.DataTransferManager
 if NOT DEFINED JAVA_HOME goto :err
 
 @REM -----------------------------------------------------------------------------
diff --git a/server/src/assembly/resources/tools/start-sync-client.sh b/server/src/assembly/resources/tools/start-sync-client.sh
index c796197..a3ae56a 100755
--- a/server/src/assembly/resources/tools/start-sync-client.sh
+++ b/server/src/assembly/resources/tools/start-sync-client.sh
@@ -47,7 +47,7 @@ for f in ${IOTDB_HOME}/lib/*.jar; do
   CLASSPATH=${CLASSPATH}":"$f
 done
 
-MAIN_CLASS=org.apache.iotdb.db.sync.sender.SyncSenderImpl
+MAIN_CLASS=org.apache.iotdb.db.sync.sender.transfer.DataTransferManager
 
 "$JAVA" -DIOTDB_HOME=${IOTDB_HOME} -DTSFILE_HOME=${IOTDB_HOME} -DIOTDB_CONF=${IOTDB_CONF} -Dlogback.configurationFile=${IOTDB_CONF}/logback.xml $IOTDB_DERBY_OPTS $IOTDB_JMX_OPTS -Dname=SyncClient -cp "$CLASSPATH" "$MAIN_CLASS"
 
diff --git a/server/src/main/java/org/apache/iotdb/db/concurrent/ThreadName.java b/server/src/main/java/org/apache/iotdb/db/concurrent/ThreadName.java
index 28e4957..02f48ba 100644
--- a/server/src/main/java/org/apache/iotdb/db/concurrent/ThreadName.java
+++ b/server/src/main/java/org/apache/iotdb/db/concurrent/ThreadName.java
@@ -40,6 +40,7 @@ public enum ThreadName {
   SYNC_CLIENT("Sync-Client"),
   SYNC_SERVER("Sync-Server"),
   SYNC_MONITOR("Sync-Monitor"),
+  LOAD_TSFILE("Load TsFile"),
   TIME_COST_STATSTIC("TIME_COST_STATSTIC");
 
   private String name;
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index d0e9109..7f1e301 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -240,12 +240,6 @@ public class IoTDBConfig {
    * Set the language version when loading file including error information, default value is "EN"
    */
   private String languageVersion = "EN";
-  /**
-   * Choose a postBack strategy of merging historical data: 1. It's more likely to update historical
-   * data, choose "true". 2. It's more likely not to update historical data or you don't know
-   * exactly, choose "false".
-   */
-  private boolean updateHistoricalDataPossibility = false;
 
   private String ipWhiteList = "0.0.0.0/0";
   /**
@@ -702,14 +696,6 @@ public class IoTDBConfig {
     this.languageVersion = languageVersion;
   }
 
-  public boolean isUpdateHistoricalDataPossibility() {
-    return updateHistoricalDataPossibility;
-  }
-
-  void setUpdateHistoricalDataPossibility(boolean updateHistoricalDataPossibility) {
-    this.updateHistoricalDataPossibility = updateHistoricalDataPossibility;
-  }
-
   public String getBaseDir() {
     return baseDir;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
index 096452e..c624d22 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
@@ -65,6 +65,11 @@ public class IoTDBConstant {
   public static final String USER = "User";
   public static final String PRIVILEGE = "Privilege";
 
+  // data folder name
+  public static final String SEQUENCE_FLODER_NAME = "sequence";
+  public static final String UNSEQUENCE_FLODER_NAME = "unsequence";
+  public static final String TSFILE_NAME_SEPARATOR = "-";
+
   public static final int MEMTABLE_NUM_IN_EACH_STORAGE_GROUP = 4;
 
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index 499a51d..bc95f45 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -199,9 +199,6 @@ public class IoTDBDescriptor {
           .parseInt(properties.getProperty("sync_server_port",
               Integer.toString(conf.getSyncServerPort())).trim()));
 
-      conf.setUpdateHistoricalDataPossibility(Boolean.parseBoolean(
-          properties.getProperty("update_historical_data_possibility",
-              Boolean.toString(conf.isSyncEnable()))));
       conf.setIpWhiteList(properties.getProperty("ip_white_list", conf.getIpWhiteList()));
 
       conf.setConcurrentFlushThread(Integer
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/adapter/IoTDBConfigDynamicAdapter.java b/server/src/main/java/org/apache/iotdb/db/conf/adapter/IoTDBConfigDynamicAdapter.java
index bc4f144..038db4d 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/adapter/IoTDBConfigDynamicAdapter.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/adapter/IoTDBConfigDynamicAdapter.java
@@ -149,10 +149,12 @@ public class IoTDBConfigDynamicAdapter implements IDynamicAdapter {
       CONFIG.setMaxMemtableNumber(maxMemTableNum);
       CONFIG.setTsFileSizeThreshold(tsFileSizeThreshold);
       CONFIG.setMemtableSizeThreshold(memtableSizeInByte);
-      if(LOGGER.isDebugEnabled()) {
+      if (LOGGER.isDebugEnabled()) {
         LOGGER.debug(
-            "After adjusting, max memTable num is {}, tsFile threshold is {}, memtableSize is {}, memTableSizeFloorThreshold is {}",
-            maxMemTableNum, tsFileSizeThreshold, memtableSizeInByte, memTableSizeFloorThreshold);
+            "After adjusting, max memTable num is {}, tsFile threshold is {}, memtableSize is {}, memTableSizeFloorThreshold is {}, storage group = {}, total timeseries = {}, the max number of timeseries among storage groups = {}",
+            maxMemTableNum, tsFileSizeThreshold, memtableSizeInByte, memTableSizeFloorThreshold,
+            totalStorageGroup, totalTimeseries,
+            MManager.getInstance().getMaximalSeriesNumberAmongStorageGroups());
       }
       currentMemTableSize = memtableSizeInByte;
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java b/server/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
index 7767a5a..63e4742 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
@@ -22,6 +22,7 @@ import java.io.File;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.directories.strategy.DirectoryStrategy;
 import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
@@ -45,14 +46,16 @@ public class DirectoryManager {
     sequenceFileFolders =
         new ArrayList<>(Arrays.asList(IoTDBDescriptor.getInstance().getConfig().getDataDirs()));
     for (int i = 0; i < sequenceFileFolders.size(); i++) {
-      sequenceFileFolders.set(i, sequenceFileFolders.get(i) + File.separator + "sequence");
+      sequenceFileFolders
+          .set(i, sequenceFileFolders.get(i) + File.separator + IoTDBConstant.SEQUENCE_FLODER_NAME);
     }
     mkDataDirs(sequenceFileFolders);
 
     unsequenceFileFolders =
         new ArrayList<>(Arrays.asList(IoTDBDescriptor.getInstance().getConfig().getDataDirs()));
     for (int i = 0; i < unsequenceFileFolders.size(); i++) {
-      unsequenceFileFolders.set(i, unsequenceFileFolders.get(i) + File.separator + "unsequence");
+      unsequenceFileFolders.set(i,
+          unsequenceFileFolders.get(i) + File.separator + IoTDBConstant.UNSEQUENCE_FLODER_NAME);
     }
     mkDataDirs(unsequenceFileFolders);
 
@@ -121,6 +124,7 @@ public class DirectoryManager {
   }
 
   private static class DirectoriesHolder {
+
     private static final DirectoryManager INSTANCE = new DirectoryManager();
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
index f5db3fe..8a49841 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.db.engine;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
@@ -36,6 +37,7 @@ import org.apache.iotdb.db.exception.ProcessorException;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.StorageEngineFailureException;
 import org.apache.iotdb.db.exception.StorageGroupException;
+import org.apache.iotdb.db.exception.TsFileProcessorException;
 import org.apache.iotdb.db.exception.qp.QueryProcessorException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.qp.physical.crud.BatchInsertPlan;
@@ -115,8 +117,7 @@ public class StorageEngine implements IService {
     return ServiceType.STORAGE_ENGINE_SERVICE;
   }
 
-
-  private StorageGroupProcessor getProcessor(String path) throws StorageEngineException {
+  public StorageGroupProcessor getProcessor(String path) throws StorageEngineException {
     String storageGroupName = "";
     try {
       storageGroupName = MManager.getInstance().getStorageGroupNameByPath(path);
@@ -354,4 +355,13 @@ public class StorageEngine implements IService {
     }
   }
 
+  public void loadNewTsFile(TsFileResource newTsFileResource)
+      throws TsFileProcessorException, StorageEngineException {
+    getProcessor(newTsFileResource.getFile().getParentFile().getName()).loadNewTsFile(newTsFileResource);
+  }
+
+  public void deleteTsfile(File deletedTsfile) throws StorageEngineException {
+    getProcessor(deletedTsfile.getParentFile().getName()).deleteTsfile(deletedTsfile);
+  }
+
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeFileTask.java b/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeFileTask.java
index e559c8e..8569136 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeFileTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/merge/task/MergeFileTask.java
@@ -19,17 +19,22 @@
 
 package org.apache.iotdb.db.engine.merge.task;
 
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+import java.io.File;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.engine.cache.DeviceMetaDataCache;
 import org.apache.iotdb.db.engine.cache.TsFileMetaDataCache;
 import org.apache.iotdb.db.engine.merge.manage.MergeContext;
 import org.apache.iotdb.db.engine.merge.manage.MergeResource;
 import org.apache.iotdb.db.engine.merge.recover.MergeLogger;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.query.control.FileReaderManager;
 import org.apache.iotdb.tsfile.exception.write.TsFileNotCompleteException;
 import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetaData;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
@@ -115,6 +120,8 @@ class MergeFileTask {
     try {
       TsFileMetaDataCache.getInstance().remove(seqFile);
       DeviceMetaDataCache.getInstance().remove(seqFile);
+      FileReaderManager.getInstance().closeFileAndRemoveReader(seqFile);
+
       resource.removeFileReader(seqFile);
       TsFileIOWriter oldFileWriter;
       try {
@@ -148,6 +155,13 @@ class MergeFileTask {
       logger.debug("{} moved merged chunks of {} to the old file", taskName, seqFile);
 
       newFileWriter.getFile().delete();
+
+      File nextMergeVersionFile = getNextMergeVersionFile(seqFile.getFile());
+      FileUtils.moveFile(seqFile.getFile(), nextMergeVersionFile);
+      FileUtils
+          .moveFile(new File(seqFile.getFile().getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX),
+              new File(nextMergeVersionFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX));
+      seqFile.setFile(nextMergeVersionFile);
     } finally {
       seqFile.getMergeQueryLock().writeLock().unlock();
     }
@@ -208,13 +222,29 @@ class MergeFileTask {
       resource.removeFileReader(seqFile);
       TsFileMetaDataCache.getInstance().remove(seqFile);
       DeviceMetaDataCache.getInstance().remove(seqFile);
+      FileReaderManager.getInstance().closeFileAndRemoveReader(seqFile);
       seqFile.getFile().delete();
-      FileUtils.moveFile(fileWriter.getFile(), seqFile.getFile());
+
+      File nextMergeVersionFile = getNextMergeVersionFile(seqFile.getFile());
+      FileUtils.moveFile(fileWriter.getFile(), nextMergeVersionFile);
+      FileUtils
+          .moveFile(new File(seqFile.getFile().getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX),
+              new File(nextMergeVersionFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX));
+      seqFile.setFile(nextMergeVersionFile);
     } finally {
       seqFile.getMergeQueryLock().writeLock().unlock();
     }
   }
 
+  private File getNextMergeVersionFile(File seqFile) {
+    String[] splits = seqFile.getName().replace(TSFILE_SUFFIX, "")
+        .split(IoTDBConstant.TSFILE_NAME_SEPARATOR);
+    int mergeVersion = Integer.parseInt(splits[2]) + 1;
+    return new File(seqFile.getParentFile(),
+        splits[0] + IoTDBConstant.TSFILE_NAME_SEPARATOR + splits[1]
+            + IoTDBConstant.TSFILE_NAME_SEPARATOR + mergeVersion + TSFILE_SUFFIX);
+  }
+
   private long writeUnmergedChunks(List<Long> chunkStartTimes,
       List<ChunkMetaData> chunkMetaDataList, TsFileSequenceReader reader,
       RestorableTsFileIOWriter fileWriter) throws IOException {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
index def8bed..4260dd4 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
@@ -24,6 +24,7 @@ import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFF
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -37,8 +38,10 @@ import java.util.Set;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.engine.merge.manage.MergeManager;
 import org.apache.iotdb.db.engine.merge.manage.MergeResource;
 import org.apache.iotdb.db.engine.merge.selector.IMergeFileSelector;
@@ -67,20 +70,19 @@ import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.JobFileManager;
-import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.db.utils.CopyOnReadLinkedList;
 import org.apache.iotdb.db.writelog.recover.TsFileRecoverPerformer;
+import org.apache.iotdb.rpc.TSStatusCode;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
-import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.utils.Pair;
-import org.apache.iotdb.tsfile.write.schema.Schema;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.Schema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -210,7 +212,8 @@ public class StorageGroupProcessor {
 
     try {
       // collect TsFiles from sequential and unsequential data directory
-      List<TsFileResource> seqTsFiles = getAllFiles(DirectoryManager.getInstance().getAllSequenceFileFolders());
+      List<TsFileResource> seqTsFiles = getAllFiles(
+          DirectoryManager.getInstance().getAllSequenceFileFolders());
       List<TsFileResource> unseqTsFiles =
           getAllFiles(DirectoryManager.getInstance().getAllUnSequenceFileFolders());
 
@@ -218,7 +221,8 @@ public class StorageGroupProcessor {
       recoverUnseqFiles(unseqTsFiles);
 
       String taskName = storageGroupName + "-" + System.currentTimeMillis();
-      File mergingMods = SystemFileFactory.INSTANCE.getFile(storageGroupSysDir, MERGING_MODIFICAITON_FILE_NAME);
+      File mergingMods = SystemFileFactory.INSTANCE
+          .getFile(storageGroupSysDir, MERGING_MODIFICAITON_FILE_NAME);
       if (mergingMods.exists()) {
         mergingModification = new ModificationFile(mergingMods.getPath());
       }
@@ -226,7 +230,8 @@ public class StorageGroupProcessor {
           storageGroupSysDir.getPath(), this::mergeEndAction, taskName,
           IoTDBDescriptor.getInstance().getConfig().isForceFullMerge(), storageGroupName);
       logger.info("{} a RecoverMergeTask {} starts...", storageGroupName, taskName);
-      recoverMergeTask.recoverMerge(IoTDBDescriptor.getInstance().getConfig().isContinueMergeAfterReboot());
+      recoverMergeTask
+          .recoverMerge(IoTDBDescriptor.getInstance().getConfig().isContinueMergeAfterReboot());
       if (!IoTDBDescriptor.getInstance().getConfig().isContinueMergeAfterReboot()) {
         mergingMods.delete();
       }
@@ -301,10 +306,12 @@ public class StorageGroupProcessor {
   }
 
   // TsFileNameComparator compares TsFiles by the version number in its name
-  // ({systemTime}-{versionNum}.tsfile)
-  public int compareFileName(File o1, File o2) {
-    String[] items1 = o1.getName().replace(TSFILE_SUFFIX, "").split("-");
-    String[] items2 = o2.getName().replace(TSFILE_SUFFIX, "").split("-");
+  // ({systemTime}-{versionNum}-{mergeNum}.tsfile)
+  private int compareFileName(File o1, File o2) {
+    String[] items1 = o1.getName().replace(TSFILE_SUFFIX, "")
+        .split(IoTDBConstant.TSFILE_NAME_SEPARATOR);
+    String[] items2 = o2.getName().replace(TSFILE_SUFFIX, "")
+        .split(IoTDBConstant.TSFILE_NAME_SEPARATOR);
     if (Long.valueOf(items1[0]) - Long.valueOf(items2[0]) == 0) {
       return Long.compare(Long.valueOf(items1[1]), Long.valueOf(items2[1]));
     } else {
@@ -401,7 +408,8 @@ public class StorageGroupProcessor {
     boolean result = tsFileProcessor.insertBatch(batchInsertPlan, indexes, results);
 
     // try to update the latest time of the device of this tsRecord
-    if (result && latestTimeForEachDevice.get(batchInsertPlan.getDeviceId()) < batchInsertPlan.getMaxTime()) {
+    if (result && latestTimeForEachDevice.get(batchInsertPlan.getDeviceId()) < batchInsertPlan
+        .getMaxTime()) {
       latestTimeForEachDevice.put(batchInsertPlan.getDeviceId(), batchInsertPlan.getMaxTime());
     }
 
@@ -477,8 +485,9 @@ public class StorageGroupProcessor {
           e);
       IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
     } catch (IOException e) {
-      logger.error("meet IOException when creating TsFileProcessor, change system mode to read-only",
-          e);
+      logger
+          .error("meet IOException when creating TsFileProcessor, change system mode to read-only",
+              e);
       IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
     }
     return tsFileProcessor;
@@ -494,8 +503,9 @@ public class StorageGroupProcessor {
     }
     fsFactory.getFile(baseDir, storageGroupName).mkdirs();
 
-    String filePath = baseDir + File.separator + storageGroupName + File.separator +
-        System.currentTimeMillis() + "-" + versionController.nextVersion() + TSFILE_SUFFIX;
+    String filePath = Paths.get(baseDir, storageGroupName,
+        System.currentTimeMillis() + IoTDBConstant.TSFILE_NAME_SEPARATOR + versionController
+            .nextVersion()).toString() + IoTDBConstant.TSFILE_NAME_SEPARATOR + "0" + TSFILE_SUFFIX;
 
     if (sequence) {
       return new TsFileProcessor(storageGroupName, fsFactory.getFile(filePath),
@@ -576,7 +586,7 @@ public class StorageGroupProcessor {
       this.latestFlushedTimeForEachDevice.clear();
       this.latestTimeForEachDevice.clear();
     } catch (IOException e) {
-      logger.error("Cannot delete files in storage group {}, because", storageGroupName, e);
+      logger.error("Cannot delete files in storage group {}", storageGroupName, e);
     } finally {
       writeUnlock();
     }
@@ -632,7 +642,8 @@ public class StorageGroupProcessor {
           deviceId, measurementId, context);
       List<TsFileResource> unseqResources = getFileReSourceListForQuery(unSequenceFileList,
           deviceId, measurementId, context);
-      QueryDataSource dataSource =  new QueryDataSource(new Path(deviceId, measurementId), seqResources, unseqResources);
+      QueryDataSource dataSource = new QueryDataSource(new Path(deviceId, measurementId),
+          seqResources, unseqResources);
       // used files should be added before mergeLock is unlocked, or they may be deleted by
       // running merge
       // is null only in tests
@@ -879,8 +890,10 @@ public class StorageGroupProcessor {
         mergeResource.setCacheDeviceMeta(true);
 
         MergeTask mergeTask = new MergeTask(mergeResource, storageGroupSysDir.getPath(),
-            this::mergeEndAction, taskName, fullMerge, fileSelector.getConcurrentMergeNum(), storageGroupName);
-        mergingModification = new ModificationFile(storageGroupSysDir + File.separator + MERGING_MODIFICAITON_FILE_NAME);
+            this::mergeEndAction, taskName, fullMerge, fileSelector.getConcurrentMergeNum(),
+            storageGroupName);
+        mergingModification = new ModificationFile(
+            storageGroupSysDir + File.separator + MERGING_MODIFICAITON_FILE_NAME);
         MergeManager.getINSTANCE().submitMainTask(mergeTask);
         if (logger.isInfoEnabled()) {
           logger.info("{} submits a merge task {}, merging {} seqFiles, {} unseqFiles",
@@ -974,15 +987,221 @@ public class StorageGroupProcessor {
     logger.info("{} a merge task ends", storageGroupName);
   }
 
+  /**
+   * Load a new tsfile to storage group processor
+   *
+   * Firstly, determine the loading type of the file, whether it needs to be loaded in sequence list
+   * or unsequence list.
+   *
+   * Secondly, execute the loading process by the type.
+   *
+   * Finally, update the latestTimeForEachDevice and latestFlushedTimeForEachDevice.
+   *
+   * @param newTsFileResource tsfile resource
+   * @UsedBy sync module.
+   */
+  public void loadNewTsFile(TsFileResource newTsFileResource)
+      throws TsFileProcessorException {
+    File tsfileToBeInserted = newTsFileResource.getFile();
+    writeLock();
+    mergeLock.writeLock().lock();
+    try {
+      loadTsFileByType(LoadTsFileType.LOAD_SEQUENCE, tsfileToBeInserted, newTsFileResource,
+          getBinarySearchIndex(newTsFileResource));
+      updateLatestTimeMap(newTsFileResource);
+    } catch (TsFileProcessorException | DiskSpaceInsufficientException e) {
+      logger.error("Failed to append the tsfile {} to storage group processor {}.",
+          tsfileToBeInserted.getAbsolutePath(), tsfileToBeInserted.getParentFile().getName());
+      IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
+      throw new TsFileProcessorException(e);
+    } finally {
+      mergeLock.writeLock().unlock();
+      writeUnlock();
+    }
+  }
+
+  /**
+   * Get binary search index in @code{sequenceFileList}
+   *
+   * @return right index to insert
+   */
+  private int getBinarySearchIndex(TsFileResource tsFileResource) {
+    if (sequenceFileList.isEmpty()) {
+      return 0;
+    }
+    long targetTsFileTime = Long.parseLong(
+        tsFileResource.getFile().getName().split(IoTDBConstant.TSFILE_NAME_SEPARATOR)[0]);
+    int s = 0, e = sequenceFileList.size() - 1;
+    while (s <= e) {
+      int m = s + ((e - s) >> 1);
+      long currentTsFileTime = Long.parseLong(sequenceFileList.get(m).getFile().getName()
+          .split(IoTDBConstant.TSFILE_NAME_SEPARATOR)[0]);
+      if (currentTsFileTime >= targetTsFileTime) {
+        e = m - 1;
+      } else {
+        s = m + 1;
+      }
+    }
+    return s;
+  }
+
+  /**
+   * Update latest time in latestTimeForEachDevice and latestFlushedTimeForEachDevice.
+   *
+   * @UsedBy sync module
+   */
+  private void updateLatestTimeMap(TsFileResource newTsFileResource) {
+    for (Entry<String, Long> entry : newTsFileResource.getEndTimeMap().entrySet()) {
+      String device = entry.getKey();
+      long endTime = newTsFileResource.getEndTimeMap().get(device);
+      if (!latestTimeForEachDevice.containsKey(device)
+          || latestTimeForEachDevice.get(device) < endTime) {
+        latestTimeForEachDevice.put(device, endTime);
+      }
+      if (!latestFlushedTimeForEachDevice.containsKey(device)
+          || latestFlushedTimeForEachDevice.get(device) < endTime) {
+        latestFlushedTimeForEachDevice.put(device, endTime);
+      }
+    }
+  }
+
+  /**
+   * Execute the loading process by the type.
+   *
+   * @param type load type
+   * @param tsFileResource tsfile resource to be loaded
+   * @param index the index in sequenceFileList/unSequenceFileList
+   * @UsedBy sync module
+   */
+  private void loadTsFileByType(LoadTsFileType type, File syncedTsFile,
+      TsFileResource tsFileResource, int index)
+      throws TsFileProcessorException, DiskSpaceInsufficientException {
+    File targetFile;
+    switch (type) {
+      case LOAD_UNSEQUENCE:
+        targetFile =
+            new File(DirectoryManager.getInstance().getNextFolderForUnSequenceFile(),
+                syncedTsFile.getParentFile().getName() + File.separatorChar + syncedTsFile
+                    .getName());
+        tsFileResource.setFile(targetFile);
+        unSequenceFileList.add(index, tsFileResource);
+        logger
+            .info("Load tsfile in unsequence list, move file from {} to {}",
+                syncedTsFile.getAbsolutePath(),
+                targetFile.getAbsolutePath());
+        break;
+      case LOAD_SEQUENCE:
+        targetFile =
+            new File(DirectoryManager.getInstance().getNextFolderForSequenceFile(),
+                syncedTsFile.getParentFile().getName() + File.separatorChar + syncedTsFile
+                    .getName());
+        tsFileResource.setFile(targetFile);
+        sequenceFileList.add(index, tsFileResource);
+        logger
+            .info("Load tsfile in sequence list, move file from {} to {}",
+                syncedTsFile.getAbsolutePath(),
+                targetFile.getAbsolutePath());
+        break;
+      default:
+        throw new TsFileProcessorException(
+            String.format("Unsupported type of loading tsfile : %s", type));
+    }
+
+    // move file from sync dir to data dir
+    if (!targetFile.getParentFile().exists()) {
+      targetFile.getParentFile().mkdirs();
+    }
+    if (syncedTsFile.exists() && !targetFile.exists()) {
+      try {
+        FileUtils.moveFile(syncedTsFile, targetFile);
+      } catch (IOException e) {
+        throw new TsFileProcessorException(String.format(
+            "File renaming failed when loading tsfile. Origin: %s, Target: %s, because %s",
+            syncedTsFile.getAbsolutePath(), targetFile.getAbsolutePath(), e.getMessage()));
+      }
+    }
+    try {
+      FileUtils.moveFile(new File(syncedTsFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX),
+          new File(targetFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX));
+    } catch (IOException e) {
+      throw new TsFileProcessorException(String.format(
+          "File renaming failed when loading .resource file. Origin: %s, Target: %s, because %s",
+          new File(syncedTsFile + TsFileResource.RESOURCE_SUFFIX).getAbsolutePath(),
+          new File(targetFile + TsFileResource.RESOURCE_SUFFIX).getAbsolutePath(), e.getMessage()));
+    }
+  }
+
+  /**
+   * Delete tsfile if it exists.
+   *
+   * Firstly, remove the TsFileResource from sequenceFileList/unSequenceFileList.
+   *
+   * Secondly, delete the tsfile and .resource file.
+   *
+   * @param deletedTsfile tsfile to be deleted
+   * @UsedBy sync module.
+   */
+  public void deleteTsfile(File deletedTsfile) {
+    writeLock();
+    mergeLock.writeLock().lock();
+    TsFileResource deletedTsFileResource = null;
+    try {
+      Iterator<TsFileResource> sequenceIterator = sequenceFileList.iterator();
+      while (sequenceIterator.hasNext()) {
+        TsFileResource sequenceResource = sequenceIterator.next();
+        if (sequenceResource.getFile().getName().equals(deletedTsfile.getName())) {
+          deletedTsFileResource = sequenceResource;
+          sequenceIterator.remove();
+          break;
+        }
+      }
+      if (deletedTsFileResource == null) {
+        Iterator<TsFileResource> unsequenceIterator = unSequenceFileList.iterator();
+        while (unsequenceIterator.hasNext()) {
+          TsFileResource unsequenceResource = unsequenceIterator.next();
+          if (unsequenceResource.getFile().getName().equals(deletedTsfile.getName())) {
+            deletedTsFileResource = unsequenceResource;
+            unsequenceIterator.remove();
+            break;
+          }
+        }
+      }
+    } finally {
+      mergeLock.writeLock().unlock();
+      writeUnlock();
+    }
+    if (deletedTsFileResource == null) {
+      return;
+    }
+    deletedTsFileResource.getMergeQueryLock().writeLock().lock();
+    try {
+      logger.info("Delete tsfile {} in sync loading process.", deletedTsFileResource.getFile());
+      deletedTsFileResource.remove();
+    } finally {
+      deletedTsFileResource.getMergeQueryLock().writeLock().unlock();
+    }
+  }
 
   public TsFileProcessor getWorkSequenceTsFileProcessor() {
     return workSequenceTsFileProcessor;
   }
 
+  public List<TsFileResource> getSequenceFileList() {
+    return sequenceFileList;
+  }
+
+  public List<TsFileResource> getUnSequenceFileList() {
+    return unSequenceFileList;
+  }
+
   @FunctionalInterface
   public interface CloseTsFileCallBack {
 
     void call(TsFileProcessor caller) throws TsFileProcessorException, IOException;
   }
 
+  private enum LoadTsFileType {
+    LOAD_SEQUENCE, LOAD_UNSEQUENCE
+  }
+
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
index c30361f..3e8b602 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java
@@ -185,6 +185,10 @@ public class TsFileResource {
     return modFile;
   }
 
+  public void setFile(File file) {
+    this.file = file;
+  }
+
   public boolean containsDevice(String deviceId) {
     return startTimeMap.containsKey(deviceId);
   }
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/test/Utils.java b/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
similarity index 56%
rename from server/src/test/java/org/apache/iotdb/db/sync/test/Utils.java
rename to server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
index 6e2f928..90fa5c1 100644
--- a/server/src/test/java/org/apache/iotdb/db/sync/test/Utils.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
@@ -16,29 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.test;
+package org.apache.iotdb.db.exception;
 
-public class Utils {
+public class SyncDeviceOwnerConflictException extends Exception {
 
-  public static String getType(String properties) {
-    return properties.split(",")[0];
-  }
+  private static final long serialVersionUID = -5037926672199248044L;
 
-  public static String getEncode(String properties) {
-    return properties.split(",")[1];
+  public SyncDeviceOwnerConflictException(String message) {
+    super(message);
   }
 
-  private Utils() {
-    throw new IllegalStateException("Utility class");
+  public SyncDeviceOwnerConflictException(String message, Throwable cause) {
+    super(message, cause);
   }
 
-  public static String getPath(String timeseries) {
-    int lastPointIndex = timeseries.lastIndexOf('.');
-    return timeseries.substring(0, lastPointIndex);
+  public SyncDeviceOwnerConflictException(Throwable cause) {
+    super(cause);
   }
 
-  public static String getSensor(String timeseries) {
-    int lastPointIndex = timeseries.lastIndexOf('.');
-    return timeseries.substring(lastPointIndex + 1);
-  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index 8654db6..320ed8a 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@ -85,7 +85,7 @@ public class MManager {
   private MManager() {
 
     schemaDir =
-        IoTDBDescriptor.getInstance().getConfig().getSystemDir() + File.separator + "schema";
+        IoTDBDescriptor.getInstance().getConfig().getSchemaDir();
 
     File systemFolder = SystemFileFactory.INSTANCE.getFile(schemaDir);
     if (!systemFolder.exists()) {
@@ -199,7 +199,7 @@ public class MManager {
     }
   }
 
-  private void operation(String cmd)
+  public void operation(String cmd)
       throws PathErrorException, IOException, MetadataErrorException {
     //see addPathToMTree() to get the detailed format of the cmd
     String[] args = cmd.trim().split(",");
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/universal/CachedPriorityMergeReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/universal/CachedPriorityMergeReader.java
index e28d6f6..125fa0c 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/universal/CachedPriorityMergeReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/universal/CachedPriorityMergeReader.java
@@ -34,10 +34,10 @@ public class CachedPriorityMergeReader extends PriorityMergeReader {
   private TimeValuePair[] timeValuePairCache = new TimeValuePair[CACHE_SIZE];
   private int cacheLimit = 0;
   private int cacheIdx = 0;
-  private TSDataType dataType;
+
+  private Long lastTimestamp = null;
 
   public CachedPriorityMergeReader(TSDataType dataType) {
-    this.dataType = dataType;
     for (int i = 0; i < CACHE_SIZE; i++) {
       timeValuePairCache[i] = TimeValuePairUtils.getEmptyTimeValuePair(dataType);
     }
@@ -52,16 +52,14 @@ public class CachedPriorityMergeReader extends PriorityMergeReader {
     cacheLimit = 0;
     cacheIdx = 0;
     while (!heap.isEmpty() && cacheLimit < CACHE_SIZE) {
-      Element top = heap.poll();
-      if (cacheLimit == 0 || top.currTime() != timeValuePairCache[cacheLimit - 1].getTimestamp()) {
+      Element top = heap.peek();
+      if (lastTimestamp == null || top.currTime() != lastTimestamp) {
         TimeValuePairUtils.setTimeValuePair(top.timeValuePair, timeValuePairCache[cacheLimit++]);
-        if (top.hasNext()) {
-          top.next();
-          heap.add(top);
-        } else {
-          top.close();
-        }
-      } else if (top.currTime() == timeValuePairCache[cacheLimit - 1].getTimestamp()) {
+        lastTimestamp = top.currTime();
+      }
+      // remove duplicates
+      while (heap.peek() != null && heap.peek().currTime() == lastTimestamp) {
+        top = heap.poll();
         if (top.hasNext()) {
           top.next();
           heap.add(top);
@@ -72,6 +70,7 @@ public class CachedPriorityMergeReader extends PriorityMergeReader {
     }
   }
 
+
   @Override
   public TimeValuePair next() throws IOException {
     TimeValuePair ret;
@@ -85,11 +84,12 @@ public class CachedPriorityMergeReader extends PriorityMergeReader {
   }
 
   @Override
-  public TimeValuePair current() {
+  public TimeValuePair current() throws IOException {
     if (0 <= cacheIdx && cacheIdx < cacheLimit) {
       return timeValuePairCache[cacheIdx];
     } else {
-      return heap.peek().timeValuePair;
+      fetch();
+      return timeValuePairCache[cacheIdx];
     }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java
index c3d8096..1e1e5d1 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java
@@ -76,7 +76,7 @@ public class PriorityMergeReader implements IPointReader {
   }
 
   @Override
-  public TimeValuePair current() {
+  public TimeValuePair current() throws IOException {
     return heap.peek().timeValuePair;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/conf/Constans.java b/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncConstant.java
similarity index 59%
rename from server/src/main/java/org/apache/iotdb/db/sync/conf/Constans.java
rename to server/src/main/java/org/apache/iotdb/db/sync/conf/SyncConstant.java
index 7aea233..762fd03 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/conf/Constans.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncConstant.java
@@ -18,31 +18,43 @@
  */
 package org.apache.iotdb.db.sync.conf;
 
-public class Constans {
+public class SyncConstant {
 
-  private Constans() {
+  private SyncConstant() {
   }
 
   public static final String CONFIG_NAME = "iotdb-sync-client.properties";
-  public static final String SYNC_CLIENT = "sync-client";
-  public static final String SYNC_SERVER = "sync-server";
 
-  public static final String LOCK_FILE_NAME = "sync-lock";
-  public static final String UUID_FILE_NAME = "uuid.txt";
-  public static final String LAST_LOCAL_FILE_NAME = "last_local_files.txt";
-  public static final String DATA_SNAPSHOT_NAME = "data-snapshot";
+  public static final String SYNC_NAME = "sync";
+
+  public static final String SYNC_SENDER = "sync-sender";
+
+  public static final String SYNC_RECEIVER = "sync-receiver";
+
+  public static final String MESSAGE_DIGIT_NAME = "MD5";
 
-  public static final String BACK_UP_DIRECTORY_NAME = "backup";
+  public static final String SYNC_DIR_NAME_SEPARATOR = "_";
 
   /**
-   * Split data file , block size at each transmission
+   * Split data file, block size at each transmission
    **/
   public static final int DATA_CHUNK_SIZE = 64 * 1024 * 1024;
 
-  /**
-   * Max try when syncing the same file to receiver fails.
-   */
-  public static final int MAX_SYNC_FILE_TRY = 10;
+  // sender section
+
+  public static final String LOCK_FILE_NAME = "sync_lock";
+
+  public static final String UUID_FILE_NAME = "uuid.txt";
+
+  public static final String SCHEMA_POS_FILE_NAME = "sync_schema_pos";
+
+  public static final String LAST_LOCAL_FILE_NAME = "last_local_files.txt";
+
+  public static final String CURRENT_LOCAL_FILE_NAME = "current_local_files.txt";
+
+  public static final String DATA_SNAPSHOT_NAME = "snapshot";
+
+  public static final String SYNC_LOG_NAME = "sync.log";
 
   private static final SyncSenderConfig CONFIG = SyncSenderDescriptor.getInstance().getConfig();
 
@@ -54,4 +66,14 @@ public class Constans {
 
   public static final long SYNC_MONITOR_PERIOD = CONFIG.getSyncPeriodInSecond();
 
+  // receiver section
+
+  public static final String RECEIVER_DATA_FOLDER_NAME = "data";
+
+  public static final String LOAD_LOG_NAME = "load.log";
+
+  public static final String DEVICE_OWNER_FILE_NAME = "device_owner";
+
+  public static final String DEVICE_OWNER_TMP_FILE_NAME = "device_owner.tmp";
+
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderConfig.java b/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderConfig.java
index 4eba688..45d54e2 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderConfig.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -19,135 +19,130 @@
 package org.apache.iotdb.db.sync.conf;
 
 import java.io.File;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.metadata.MetadataConstant;
-import org.apache.iotdb.db.utils.FilePathUtils;
+import java.util.ArrayList;
+import java.util.List;
 
 public class SyncSenderConfig {
 
-  private String[] seqFileDirectory = IoTDBDescriptor.getInstance().getConfig()
-      .getDataDirs();
+  private String serverIp = "127.0.0.1";
 
-  private String dataDirectory = IoTDBDescriptor.getInstance().getConfig().getBaseDir();
+  private int serverPort = 5555;
 
-  private String lockFilePath;
+  private int syncPeriodInSecond = 600;
 
-  private String uuidPath;
+  private String senderFolderPath;
 
-  private String lastFileInfo;
+  private String lockFilePath;
 
-  private String[] snapshotPaths;
+  private String uuidPath;
 
-  private String schemaPath;
+  private String lastFileInfoPath;
 
-  private String serverIp = "127.0.0.1";
+  private String snapshotPath;
 
-  private int serverPort = 5555;
+  /**
+   * The maximum number of retry when syncing a file to receiver fails.
+   */
+  private int maxNumOfSyncFileRetry = 5;
 
-  private int syncPeriodInSecond = 10;
+  /**
+   * Storage groups which participate in sync process
+   */
+  private List<String> storageGroupList = new ArrayList<>();
 
   /**
-   * Init path
+   * Update paths based on data directory
    */
-  public void init() {
-    schemaPath = IoTDBDescriptor.getInstance().getConfig().getSystemDir() + File.separator + MetadataConstant.METADATA_LOG;
-    if (dataDirectory.length() > 0
-        && dataDirectory.charAt(dataDirectory.length() - 1) != File.separatorChar) {
-      dataDirectory += File.separatorChar;
-    }
-    lockFilePath =
-        dataDirectory + Constans.SYNC_CLIENT + File.separatorChar + Constans.LOCK_FILE_NAME;
-    uuidPath = dataDirectory + Constans.SYNC_CLIENT + File.separatorChar + Constans.UUID_FILE_NAME;
-    lastFileInfo =
-        dataDirectory + Constans.SYNC_CLIENT + File.separatorChar + Constans.LAST_LOCAL_FILE_NAME;
-    snapshotPaths = new String[seqFileDirectory.length];
-    for (int i = 0; i < seqFileDirectory.length; i++) {
-      seqFileDirectory[i] = new File(seqFileDirectory[i]).getAbsolutePath();
-      seqFileDirectory[i] = FilePathUtils.regularizePath(seqFileDirectory[i]);
-      snapshotPaths[i] = seqFileDirectory[i] + Constans.SYNC_CLIENT + File.separatorChar
-          + Constans.DATA_SNAPSHOT_NAME
-          + File.separatorChar;
+  public void update(String dataDirectory) {
+    senderFolderPath =
+        dataDirectory + File.separatorChar + SyncConstant.SYNC_SENDER + File.separatorChar +
+            getSyncReceiverName();
+    lockFilePath = senderFolderPath + File.separatorChar + SyncConstant.LOCK_FILE_NAME;
+    uuidPath = senderFolderPath + File.separatorChar + SyncConstant.UUID_FILE_NAME;
+    lastFileInfoPath = senderFolderPath + File.separatorChar + SyncConstant.LAST_LOCAL_FILE_NAME;
+    snapshotPath = senderFolderPath + File.separatorChar + SyncConstant.DATA_SNAPSHOT_NAME;
+    if (!new File(snapshotPath).exists()) {
+      new File(snapshotPath).mkdirs();
     }
-
   }
 
-  public String[] getSeqFileDirectory() {
-    return seqFileDirectory;
+  public String getServerIp() {
+    return serverIp;
   }
 
-  public void setSeqFileDirectory(String[] seqFileDirectory) {
-    this.seqFileDirectory = seqFileDirectory;
+  public void setServerIp(String serverIp) {
+    this.serverIp = serverIp;
   }
 
-  public String getDataDirectory() {
-    return dataDirectory;
+  public int getServerPort() {
+    return serverPort;
   }
 
-  public void setDataDirectory(String dataDirectory) {
-    this.dataDirectory = dataDirectory;
+  public void setServerPort(int serverPort) {
+    this.serverPort = serverPort;
   }
 
-  public String getUuidPath() {
-    return uuidPath;
+  public int getSyncPeriodInSecond() {
+    return syncPeriodInSecond;
   }
 
-  public void setUuidPath(String uuidPath) {
-    this.uuidPath = uuidPath;
+  public void setSyncPeriodInSecond(int syncPeriodInSecond) {
+    this.syncPeriodInSecond = syncPeriodInSecond;
   }
 
-  public String getLastFileInfo() {
-    return lastFileInfo;
+  public String getSenderFolderPath() {
+    return senderFolderPath;
   }
 
-  public void setLastFileInfo(String lastFileInfo) {
-    this.lastFileInfo = lastFileInfo;
+  public void setSenderFolderPath(String senderFolderPath) {
+    this.senderFolderPath = senderFolderPath;
   }
 
-  public String[] getSnapshotPaths() {
-    return snapshotPaths;
+  public String getLockFilePath() {
+    return lockFilePath;
   }
 
-  public void setSnapshotPaths(String[] snapshotPaths) {
-    this.snapshotPaths = snapshotPaths;
+  public void setLockFilePath(String lockFilePath) {
+    this.lockFilePath = lockFilePath;
   }
 
-  public String getSchemaPath() {
-    return schemaPath;
+  public String getLastFileInfoPath() {
+    return lastFileInfoPath;
   }
 
-  public void setSchemaPath(String schemaPath) {
-    this.schemaPath = schemaPath;
+  public void setLastFileInfoPath(String lastFileInfoPath) {
+    this.lastFileInfoPath = lastFileInfoPath;
   }
 
-  public String getServerIp() {
-    return serverIp;
+  public String getSnapshotPath() {
+    return snapshotPath;
   }
 
-  public void setServerIp(String serverIp) {
-    this.serverIp = serverIp;
+  public void setSnapshotPath(String snapshotPath) {
+    this.snapshotPath = snapshotPath;
   }
 
-  public int getServerPort() {
-    return serverPort;
+  public String getUuidPath() {
+    return uuidPath;
   }
 
-  public void setServerPort(int serverPort) {
-    this.serverPort = serverPort;
+  public String getSyncReceiverName() {
+    return serverIp + SyncConstant.SYNC_DIR_NAME_SEPARATOR + serverPort;
   }
 
-  public int getSyncPeriodInSecond() {
-    return syncPeriodInSecond;
+  public List<String> getStorageGroupList() {
+    return new ArrayList<>(storageGroupList);
   }
 
-  public void setSyncPeriodInSecond(int syncPeriodInSecond) {
-    this.syncPeriodInSecond = syncPeriodInSecond;
+  public void setStorageGroupList(List<String> storageGroupList) {
+    this.storageGroupList = storageGroupList;
   }
 
-  public String getLockFilePath() {
-    return lockFilePath;
+  public int getMaxNumOfSyncFileRetry() {
+    return maxNumOfSyncFileRetry;
   }
 
-  public void setLockFilePath(String lockFilePath) {
-    this.lockFilePath = lockFilePath;
+  public void setMaxNumOfSyncFileRetry(int maxNumOfSyncFileRetry) {
+    this.maxNumOfSyncFileRetry = maxNumOfSyncFileRetry;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderDescriptor.java b/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderDescriptor.java
index b9eff47..a5d9143 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderDescriptor.java
@@ -23,9 +23,11 @@ import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Properties;
 import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.apache.iotdb.db.utils.FilePathUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,8 +40,8 @@ public class SyncSenderDescriptor {
     loadProps();
   }
 
-  public static final SyncSenderDescriptor getInstance() {
-    return PostBackDescriptorHolder.INSTANCE;
+  public static SyncSenderDescriptor getInstance() {
+    return SyncSenderDescriptorHolder.INSTANCE;
   }
 
   public SyncSenderConfig getConfig() {
@@ -54,22 +56,21 @@ public class SyncSenderDescriptor {
    * load an properties file and set sync config variables
    */
   private void loadProps() {
-    conf.init();
     InputStream inputStream;
     String url = System.getProperty(IoTDBConstant.IOTDB_CONF, null);
     if (url == null) {
       url = System.getProperty(IoTDBConstant.IOTDB_HOME, null);
       if (url != null) {
         url = url + File.separatorChar + "conf" + File.separatorChar
-            + Constans.CONFIG_NAME;
+            + SyncConstant.CONFIG_NAME;
       } else {
         logger.warn(
             "Cannot find IOTDB_HOME or IOTDB_CONF environment variable when loading config file {}, use default configuration",
-            Constans.CONFIG_NAME);
+            SyncConstant.CONFIG_NAME);
         return;
       }
     } else {
-      url += (File.separatorChar + Constans.CONFIG_NAME);
+      url += (File.separatorChar + SyncConstant.CONFIG_NAME);
     }
 
     try {
@@ -90,42 +91,30 @@ public class SyncSenderDescriptor {
       conf.setSyncPeriodInSecond(Integer.parseInt(properties
           .getProperty("sync_period_in_second",
               Integer.toString(conf.getSyncPeriodInSecond()))));
-      conf.setSchemaPath(properties.getProperty("iotdb_schema_directory", conf.getSchemaPath()));
-      conf.setDataDirectory(
-          properties.getProperty("iotdb_bufferWrite_directory", conf.getDataDirectory()));
-      String dataDirectory = conf.getDataDirectory();
-      if (dataDirectory.length() > 0
-          && dataDirectory.charAt(dataDirectory.length() - 1) != File.separatorChar) {
-        dataDirectory += File.separatorChar;
+      String storageGroups = properties.getProperty("sync_storage_groups", null);
+      if (storageGroups != null) {
+        String[] splits = storageGroups.split(",");
+        List<String> storageGroupList = new ArrayList<>();
+        Arrays.stream(splits).forEach(sg -> storageGroupList.add(sg.trim()));
+        conf.setStorageGroupList(storageGroupList);
       }
-      conf.setUuidPath(
-          dataDirectory + Constans.SYNC_CLIENT + File.separatorChar + Constans.UUID_FILE_NAME);
-      conf.setLastFileInfo(
-          dataDirectory + Constans.SYNC_CLIENT + File.separatorChar
-              + Constans.LAST_LOCAL_FILE_NAME);
-      String[] sequenceFileDirectory = conf.getSeqFileDirectory();
-      String[] snapshots = new String[conf.getSeqFileDirectory().length];
-      for (int i = 0; i < conf.getSeqFileDirectory().length; i++) {
-        sequenceFileDirectory[i] = FilePathUtils.regularizePath(sequenceFileDirectory[i]);
-        snapshots[i] = sequenceFileDirectory[i] + Constans.SYNC_CLIENT + File.separatorChar
-            + Constans.DATA_SNAPSHOT_NAME + File.separatorChar;
-      }
-      conf.setSeqFileDirectory(sequenceFileDirectory);
-      conf.setSnapshotPaths(snapshots);
+      conf.setMaxNumOfSyncFileRetry(Integer
+          .parseInt(properties.getProperty("max_number_of_sync_file_retry",
+              Integer.toString(conf.getMaxNumOfSyncFileRetry()))));
     } catch (IOException e) {
-      logger.warn("Cannot load config file because {}, use default configuration", e);
+      logger.warn("Cannot load sync config file, use default sync configuration.", e);
     } catch (Exception e) {
-      logger.warn("Error format in config file because {}, use default configuration", e);
+      logger.warn("Error format in sync config file, use default sync configuration.", e);
     } finally {
       try {
         inputStream.close();
       } catch (IOException e) {
-        logger.error("Fail to close sync config file input stream because ", e);
+        logger.error("Fail to close sync config file input stream.", e);
       }
     }
   }
 
-  private static class PostBackDescriptorHolder {
+  private static class SyncSenderDescriptorHolder {
 
     private static final SyncSenderDescriptor INSTANCE = new SyncSenderDescriptor();
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/package-info.java b/server/src/main/java/org/apache/iotdb/db/sync/package-info.java
new file mode 100644
index 0000000..7d30f49
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/package-info.java
@@ -0,0 +1,39 @@
+/**
+ * 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.
+ */
+
+/**
+ * <p>
+ * Package Sync is a suite tool that periodically uploads persistent tsfiles from the sender disk to
+ * the receiver and loads them. With merge module, synchronous update of write, update and delete
+ * operations can be synced.
+ *
+ * On the sender side of the sync, the sync module is a separate process, independent of the IoTDB
+ * process. It can be started and closed through separate scripts.
+ *
+ * On the receiver side of the sync, the sync module is embedded in the engine of IoTDB and is in
+ * the same process with IoTDB. The receiver module listens to a separate port. Before using it, it
+ * needs to set up a whitelist at the sync receiver, which is expressed as a network segment. The
+ * receiver only accepts the data transferred from the sender located in the whitelist segment.
+ *
+ * Due to the IoTDB system supports multiple directories of data files, it will perform sub-tasks
+ * according to disks in every complete synchronization task, because hard links are needed in the
+ * execution process. Hard links can not be operated across disk partitions, and a synchronization
+ * task will be performed in turn according to disks.
+ */
+package org.apache.iotdb.db.sync;
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManager.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManager.java
index e8593fb..61d8fae 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManager.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.db.sync.receiver;
 
+import java.io.IOException;
 import java.net.InetSocketAddress;
 import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.db.concurrent.ThreadName;
@@ -27,6 +28,9 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.service.IService;
 import org.apache.iotdb.db.service.ServiceType;
+import org.apache.iotdb.db.sync.receiver.load.FileLoaderManager;
+import org.apache.iotdb.db.sync.receiver.recover.SyncReceiverLogAnalyzer;
+import org.apache.iotdb.db.sync.receiver.transfer.SyncServiceImpl;
 import org.apache.iotdb.service.sync.thrift.SyncService;
 import org.apache.iotdb.service.sync.thrift.SyncService.Processor;
 import org.apache.thrift.protocol.TBinaryProtocol;
@@ -46,13 +50,15 @@ import org.slf4j.LoggerFactory;
 public class SyncServerManager implements IService {
 
   private static final Logger logger = LoggerFactory.getLogger(SyncServerManager.class);
-  private Thread syncServerThread;
+
   private IoTDBConfig conf = IoTDBDescriptor.getInstance().getConfig();
 
+  private Thread syncServerThread;
+
   private SyncServerManager() {
   }
 
-  public static final SyncServerManager getInstance() {
+  public static SyncServerManager getInstance() {
     return ServerManagerHolder.INSTANCE;
   }
 
@@ -64,6 +70,12 @@ public class SyncServerManager implements IService {
     if (!conf.isSyncEnable()) {
       return;
     }
+    FileLoaderManager.getInstance().start();
+    try {
+      SyncReceiverLogAnalyzer.getInstance().recoverAll();
+    } catch (IOException e) {
+      logger.error("Can not recover receiver sync state", e);
+    }
     if (conf.getIpWhiteList() == null) {
       logger.error(
           "Sync server failed to start because IP white list is null, please set IP white list.");
@@ -82,6 +94,7 @@ public class SyncServerManager implements IService {
   @Override
   public void stop() {
     if (conf.isSyncEnable()) {
+      FileLoaderManager.getInstance().stop();
       ((SyncServiceThread) syncServerThread).close();
     }
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServiceImpl.java
deleted file mode 100644
index 1ddb2bb..0000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServiceImpl.java
+++ /dev/null
@@ -1,737 +0,0 @@
-/*
- * 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.sync.receiver;
-
-import java.io.*;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.security.MessageDigest;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.iotdb.db.concurrent.ThreadName;
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.directories.DirectoryManager;
-import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.MetadataErrorException;
-import org.apache.iotdb.db.exception.PathErrorException;
-import org.apache.iotdb.db.exception.ProcessorException;
-import org.apache.iotdb.db.metadata.MManager;
-import org.apache.iotdb.db.metadata.MetadataConstant;
-import org.apache.iotdb.db.metadata.MetadataOperationType;
-import org.apache.iotdb.db.qp.executor.QueryProcessExecutor;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.sync.conf.Constans;
-import org.apache.iotdb.db.utils.FilePathUtils;
-import org.apache.iotdb.db.utils.SyncUtils;
-import org.apache.iotdb.service.sync.thrift.SyncDataStatus;
-import org.apache.iotdb.service.sync.thrift.SyncService;
-import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetaData;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
-import org.apache.iotdb.tsfile.file.metadata.TsDeviceMetadata;
-import org.apache.iotdb.tsfile.file.metadata.TsDeviceMetadataIndex;
-import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.read.ReadOnlyTsFile;
-import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
-import org.apache.iotdb.tsfile.read.common.Field;
-import org.apache.iotdb.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.read.common.RowRecord;
-import org.apache.iotdb.tsfile.read.expression.QueryExpression;
-import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class SyncServiceImpl implements SyncService.Iface {
-
-  private static final Logger logger = LoggerFactory.getLogger(SyncServiceImpl.class);
-
-  private static final StorageEngine STORAGE_GROUP_MANAGER = StorageEngine.getInstance();
-  /**
-   * Metadata manager
-   **/
-  private static final MManager metadataManger = MManager.getInstance();
-
-  private static final String SYNC_SERVER = Constans.SYNC_SERVER;
-
-  private ThreadLocal<String> uuid = new ThreadLocal<>();
-  /**
-   * String means storage group,List means the set of new files(path) in local IoTDB and String
-   * means path of new Files
-   **/
-  private ThreadLocal<Map<String, List<String>>> fileNodeMap = new ThreadLocal<>();
-  /**
-   * Map String1 means timeseries String2 means path of new Files, long means startTime
-   **/
-  private ThreadLocal<Map<String, Map<String, Long>>> fileNodeStartTime = new ThreadLocal<>();
-  /**
-   * Map String1 means timeseries String2 means path of new Files, long means endTime
-   **/
-  private ThreadLocal<Map<String, Map<String, Long>>> fileNodeEndTime = new ThreadLocal<>();
-
-  /**
-   * Total num of files that needs to be loaded
-   */
-  private ThreadLocal<Integer> fileNum = new ThreadLocal<>();
-
-  /**
-   * IoTDB config
-   **/
-  private IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
-
-  /**
-   * IoTDB data directory
-   **/
-  private String baseDir = config.getBaseDir();
-
-  /**
-   * IoTDB  multiple bufferWrite directory
-   **/
-  private String[] bufferWritePaths = config.getDataDirs();
-
-  /**
-   * The path to store metadata file of sender
-   */
-  private ThreadLocal<String> schemaFromSenderPath = new ThreadLocal<>();
-
-  /**
-   * Sync folder path of server
-   **/
-  private String syncFolderPath;
-
-  /**
-   * Sync data path of server
-   */
-  private String syncDataPath;
-
-  /**
-   * Init threadLocal variable and delete old useless files.
-   */
-  @Override
-  public boolean init(String storageGroup) {
-    logger.info("Sync process starts to receive data of storage group {}", storageGroup);
-    fileNum.set(0);
-    fileNodeMap.set(new HashMap<>());
-    fileNodeStartTime.set(new HashMap<>());
-    fileNodeEndTime.set(new HashMap<>());
-    try {
-      FileUtils.deleteDirectory(new File(syncDataPath));
-    } catch (IOException e) {
-      logger.error("cannot delete directory {} ", syncFolderPath);
-      return false;
-    }
-    for (String bufferWritePath : bufferWritePaths) {
-      bufferWritePath = FilePathUtils.regularizePath(bufferWritePath);
-      String backupPath = bufferWritePath + SYNC_SERVER + File.separator;
-      File backupDirectory = new File(backupPath, this.uuid.get());
-      if (backupDirectory.exists() && backupDirectory.list().length != 0) {
-        try {
-          FileUtils.deleteDirectory(backupDirectory);
-        } catch (IOException e) {
-          logger.error("cannot delete directory {} ", syncFolderPath);
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-
-  /**
-   * Verify IP address of sender
-   */
-  @Override
-  public boolean checkIdentity(String uuid, String ipAddress) {
-    Thread.currentThread().setName(ThreadName.SYNC_SERVER.getName());
-    this.uuid.set(uuid);
-    initPath();
-    return SyncUtils.verifyIPSegment(config.getIpWhiteList(), ipAddress);
-  }
-
-  /**
-   * Init file path and clear data if last sync process failed.
-   */
-  private void initPath() {
-    baseDir = FilePathUtils.regularizePath(baseDir);
-    syncFolderPath = baseDir + SYNC_SERVER + File.separatorChar + this.uuid.get();
-    syncDataPath = syncFolderPath + File.separatorChar + Constans.DATA_SNAPSHOT_NAME;
-    schemaFromSenderPath
-        .set(syncFolderPath + File.separator + MetadataConstant.METADATA_LOG);
-  }
-
-  /**
-   * Acquire schema from sender
-   *
-   * @param status: FINIFSH_STATUS, SUCCESS_STATUS or PROCESSING_STATUS. status = FINISH_STATUS :
-   * finish receiving schema file, start to sync schema. status = PROCESSING_STATUS : the schema
-   * file has not received completely.SUCCESS_STATUS: load metadata.
-   */
-  @Override
-  public String syncSchema(String md5, ByteBuffer schema, SyncDataStatus status) {
-    String md5OfReceiver = Boolean.toString(Boolean.TRUE);
-    if (status == SyncDataStatus.SUCCESS_STATUS) {
-      /** sync metadata, include storage group and timeseries **/
-      return Boolean.toString(loadMetadata());
-    } else if (status == SyncDataStatus.PROCESSING_STATUS) {
-      File file = new File(schemaFromSenderPath.get());
-      if (!file.getParentFile().exists()) {
-        try {
-          file.getParentFile().mkdirs();
-          file.createNewFile();
-        } catch (IOException e) {
-          logger.error("Cannot make schema file {}.", file.getPath(), e);
-          md5OfReceiver = Boolean.toString(Boolean.FALSE);
-        }
-      }
-      try (FileOutputStream fos = new FileOutputStream(file, true);
-          FileChannel channel = fos.getChannel()) {
-        channel.write(schema);
-      } catch (Exception e) {
-        logger.error("Cannot insert data to file {}.", file.getPath(), e);
-        md5OfReceiver = Boolean.toString(Boolean.FALSE);
-      }
-    } else {
-      try (FileInputStream fis = new FileInputStream(schemaFromSenderPath.get())) {
-        MessageDigest md = MessageDigest.getInstance("MD5");
-        byte[] buffer = new byte[Constans.DATA_CHUNK_SIZE];
-        int n;
-        while ((n = fis.read(buffer)) != -1) {
-          md.update(buffer, 0, n);
-        }
-        md5OfReceiver = (new BigInteger(1, md.digest())).toString(16);
-        if (!md5.equals(md5OfReceiver)) {
-          FileUtils.forceDelete(new File(schemaFromSenderPath.get()));
-        }
-      } catch (Exception e) {
-        logger.error("Receiver cannot generate md5 {}", schemaFromSenderPath.get(), e);
-      }
-    }
-    return md5OfReceiver;
-  }
-
-  /**
-   * Load metadata from sender
-   */
-  private boolean loadMetadata() {
-    if (new File(schemaFromSenderPath.get()).exists()) {
-      try (BufferedReader br = new BufferedReader(
-          new java.io.FileReader(schemaFromSenderPath.get()))) {
-        String metadataOperation;
-        while ((metadataOperation = br.readLine()) != null) {
-          operation(metadataOperation);
-        }
-      } catch (FileNotFoundException e) {
-        logger.error("Cannot read the file {}.",
-            schemaFromSenderPath.get(), e);
-        return false;
-      } catch (IOException e) {
-        /** multiple insert schema, ignore it **/
-      } catch (Exception e) {
-        logger.error("Parse metadata operation failed.", e);
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /**
-   * Operate metadata operation in MManager
-   *
-   * @param cmd metadata operation
-   */
-  private void operation(String cmd)
-      throws PathErrorException, IOException, MetadataErrorException {
-    String[] args = cmd.trim().split(",");
-    switch (args[0]) {
-      case MetadataOperationType.ADD_PATH_TO_MTREE:
-        Map<String, String> props;
-        String[] kv;
-        props = new HashMap<>(args.length - 5 + 1, 1);
-        for (int k = 5; k < args.length; k++) {
-          kv = args[k].split("=");
-          props.put(kv[0], kv[1]);
-        }
-        metadataManger.addPathToMTree(new Path(args[1]), TSDataType.deserialize(Short.valueOf(args[2])),
-            TSEncoding.deserialize(Short.valueOf(args[3])),
-            CompressionType.deserialize(Short.valueOf(args[4])),
-            props);
-        break;
-      case MetadataOperationType.DELETE_PATH_FROM_MTREE:
-        metadataManger.deletePaths(Collections.singletonList(new Path(args[1])));
-        break;
-      case MetadataOperationType.SET_STORAGE_GROUP_TO_MTREE:
-        metadataManger.setStorageGroupToMTree(args[1]);
-        break;
-      case MetadataOperationType.ADD_A_PTREE:
-        metadataManger.addAPTree(args[1]);
-        break;
-      case MetadataOperationType.ADD_A_PATH_TO_PTREE:
-        metadataManger.addPathToPTree(args[1]);
-        break;
-      case MetadataOperationType.DELETE_PATH_FROM_PTREE:
-        metadataManger.deletePathFromPTree(args[1]);
-        break;
-      case MetadataOperationType.LINK_MNODE_TO_PTREE:
-        metadataManger.linkMNodeToPTree(args[1], args[2]);
-        break;
-      case MetadataOperationType.UNLINK_MNODE_FROM_PTREE:
-        metadataManger.unlinkMNodeFromPTree(args[1], args[2]);
-        break;
-      case MetadataOperationType.DELETE_STORAGE_GROUP_FROM_MTREE:
-        List<Path> storageGroups = new ArrayList<>();
-        for (int l = 1; l < args.length; l++){
-          storageGroups.add(new Path(args[l]));
-        }
-        metadataManger.deleteStorageGroupsFromMTree(storageGroups);
-      default:
-        logger.error("Unrecognizable command {}", cmd);
-    }
-  }
-
-  /**
-   * Start receiving tsfile from sender
-   *
-   * @param status status = SUCCESS_STATUS : finish receiving one tsfile status = PROCESSING_STATUS
-   * : tsfile has not received completely.
-   */
-  @Override
-  public String syncData(String md5OfSender, List<String> filePathSplit,
-      ByteBuffer dataToReceive, SyncDataStatus status) {
-    String md5OfReceiver = Boolean.toString(Boolean.TRUE);
-    FileChannel channel;
-    /** Recombination File Path **/
-    String filePath = StringUtils.join(filePathSplit, File.separatorChar);
-    syncDataPath = FilePathUtils.regularizePath(syncDataPath);
-    filePath = syncDataPath + filePath;
-    if (status == SyncDataStatus.PROCESSING_STATUS) { // there are still data stream to add
-      File file = new File(filePath);
-      if (!file.getParentFile().exists()) {
-        try {
-          file.getParentFile().mkdirs();
-          file.createNewFile();
-        } catch (IOException e) {
-          logger.error("cannot make file {}", file.getPath(), e);
-          md5OfReceiver = Boolean.toString(Boolean.FALSE);
-        }
-      }
-      try (FileOutputStream fos = new FileOutputStream(file, true)) {// append new data
-        channel = fos.getChannel();
-        channel.write(dataToReceive);
-      } catch (IOException e) {
-        logger.error("cannot insert data to file {}", file.getPath(), e);
-        md5OfReceiver = Boolean.toString(Boolean.FALSE);
-
-      }
-    } else { // all data in the same file has received successfully
-      try (FileInputStream fis = new FileInputStream(filePath)) {
-        MessageDigest md = MessageDigest.getInstance("MD5");
-        byte[] buffer = new byte[Constans.DATA_CHUNK_SIZE];
-        int n;
-        while ((n = fis.read(buffer)) != -1) {
-          md.update(buffer, 0, n);
-        }
-        md5OfReceiver = (new BigInteger(1, md.digest())).toString(16);
-        if (md5OfSender.equals(md5OfReceiver)) {
-          fileNum.set(fileNum.get() + 1);
-
-          logger.info(String.format("Receiver has received %d files from sender", fileNum.get()));
-        } else {
-          FileUtils.forceDelete(new File(filePath));
-        }
-      } catch (Exception e) {
-        logger.error("Receiver cannot generate md5 {}", filePath, e);
-      }
-    }
-    return md5OfReceiver;
-  }
-
-
-  @Override
-  public boolean load() {
-    try {
-      getFileNodeInfo();
-      loadData();
-    } catch (Exception e) {
-      logger.error("fail to load data", e);
-      return false;
-    }
-    return true;
-  }
-
-  /**
-   * Get all tsfiles' info which are sent from sender, it is preparing for merging these data
-   */
-  public void getFileNodeInfo() throws IOException {
-    File dataFileRoot = new File(syncDataPath);
-    File[] files = dataFileRoot.listFiles();
-    int processedNum = 0;
-    for (File storageGroupPB : files) {
-      List<String> filesPath = new ArrayList<>();
-      File[] filesSG = storageGroupPB.listFiles();
-      for (File fileTF : filesSG) { // fileTF means TsFiles
-        Map<String, Long> startTimeMap = new HashMap<>();
-        Map<String, Long> endTimeMap = new HashMap<>();
-        TsFileSequenceReader reader = null;
-        try {
-          reader = new TsFileSequenceReader(fileTF.getPath());
-          Map<String, TsDeviceMetadataIndex> deviceIdMap = reader.readFileMetadata().getDeviceMap();
-          Iterator<String> it = deviceIdMap.keySet().iterator();
-          while (it.hasNext()) {
-            String key = it.next();
-            TsDeviceMetadataIndex device = deviceIdMap.get(key);
-            startTimeMap.put(key, device.getStartTime());
-            endTimeMap.put(key, device.getEndTime());
-          }
-        } catch (IOException e) {
-          logger.error("Unable to read tsfile {}", fileTF.getPath());
-          throw new IOException(e);
-        } finally {
-          try {
-            if (reader != null) {
-              reader.close();
-            }
-          } catch (IOException e) {
-            logger.error("Cannot close tsfile stream {}", fileTF.getPath());
-            throw new IOException(e);
-          }
-        }
-        fileNodeStartTime.get().put(fileTF.getPath(), startTimeMap);
-        fileNodeEndTime.get().put(fileTF.getPath(), endTimeMap);
-        filesPath.add(fileTF.getPath());
-        processedNum++;
-        logger.info(String
-            .format("Get tsfile info has complete : %d/%d", processedNum, fileNum.get()));
-        fileNodeMap.get().put(storageGroupPB.getName(), filesPath);
-      }
-    }
-  }
-
-
-  /**
-   * It is to merge data. If data in the tsfile is new, append the tsfile to the storage group
-   * directly. If data in the tsfile is old, it has two strategy to merge.It depends on the
-   * possibility of updating historical data.
-   */
-  public void loadData() throws StorageEngineException {
-    syncDataPath = FilePathUtils.regularizePath(syncDataPath);
-    int processedNum = 0;
-    for (String storageGroup : fileNodeMap.get().keySet()) {
-      List<String> filesPath = fileNodeMap.get().get(storageGroup);
-      /**  before load external tsFile, it is necessary to order files in the same storage group **/
-      Collections.sort(filesPath, (o1, o2) -> {
-        Map<String, Long> startTimePath1 = fileNodeStartTime.get().get(o1);
-        Map<String, Long> endTimePath2 = fileNodeEndTime.get().get(o2);
-        for (Entry<String, Long> entry : endTimePath2.entrySet()) {
-          if (startTimePath1.containsKey(entry.getKey())) {
-            if (startTimePath1.get(entry.getKey()) > entry.getValue()) {
-              return 1;
-            } else {
-              return -1;
-            }
-          }
-        }
-        return 0;
-      });
-
-      for (String path : filesPath) {
-        // get startTimeMap and endTimeMap
-        Map<String, Long> startTimeMap = fileNodeStartTime.get().get(path);
-        Map<String, Long> endTimeMap = fileNodeEndTime.get().get(path);
-
-        // create a new fileNode
-        String header = syncDataPath;
-        String relativePath = path.substring(header.length());
-        TsFileResource fileNode = new TsFileResource(
-            new File(DirectoryManager.getInstance().getNextFolderIndexForSequenceFile() +
-                File.separator + relativePath), startTimeMap, endTimeMap
-        );
-        // call interface of load external file
-        try {
-          if (!STORAGE_GROUP_MANAGER.appendFileToStorageGroupProcessor(storageGroup, fileNode, path)) {
-            // it is a file with unsequence data
-            if (config.isUpdateHistoricalDataPossibility()) {
-              loadOldData(path);
-            } else {
-              List<String> overlapFiles = STORAGE_GROUP_MANAGER.getOverlapFiles(
-                  storageGroup,
-                  fileNode, uuid.get());
-              if (overlapFiles.isEmpty()) {
-                loadOldData(path);
-              } else {
-                loadOldData(path, overlapFiles);
-              }
-            }
-          }
-        } catch (StorageEngineException | IOException | ProcessorException e) {
-          logger.error("Can not load external file {}", path);
-          throw new StorageEngineException(e);
-        }
-        processedNum++;
-        logger.info(String
-            .format("Merging files has completed : %d/%d", processedNum, fileNum.get()));
-      }
-    }
-  }
-
-  /**
-   * Insert all data in the tsfile into IoTDB.
-   */
-  public void loadOldData(String filePath) throws IOException, ProcessorException {
-    Set<String> timeseriesSet = new HashSet<>();
-    TsFileSequenceReader reader = null;
-    QueryProcessExecutor insertExecutor = new QueryProcessExecutor();
-    try {
-      /** use tsfile reader to get data **/
-      reader = new TsFileSequenceReader(filePath);
-      Map<String, TsDeviceMetadataIndex> deviceIdMap = reader.readFileMetadata().getDeviceMap();
-      Iterator<Entry<String, TsDeviceMetadataIndex>> entryIterator = deviceIdMap.entrySet()
-          .iterator();
-      while (entryIterator.hasNext()) {
-        Entry<String, TsDeviceMetadataIndex> deviceMIEntry = entryIterator.next();
-        String deviceId = deviceMIEntry.getKey();
-        TsDeviceMetadataIndex deviceMI = deviceMIEntry.getValue();
-        TsDeviceMetadata deviceMetadata = reader.readTsDeviceMetaData(deviceMI);
-        List<ChunkGroupMetaData> rowGroupMetadataList = deviceMetadata.getChunkGroupMetaDataList();
-        timeseriesSet.clear();
-        /** firstly, get all timeseries in the same device **/
-        for (ChunkGroupMetaData chunkGroupMetaData : rowGroupMetadataList) {
-          List<ChunkMetaData> chunkMetaDataList = chunkGroupMetaData
-              .getChunkMetaDataList();
-          for (ChunkMetaData chunkMetaData : chunkMetaDataList) {
-            String measurementUID = chunkMetaData.getMeasurementUid();
-            measurementUID = deviceId + "." + measurementUID;
-            timeseriesSet.add(measurementUID);
-          }
-        }
-        /** Secondly, use tsFile Reader to form InsertPlan **/
-        ReadOnlyTsFile readTsFile = new ReadOnlyTsFile(reader);
-        List<Path> paths = new ArrayList<>();
-        paths.clear();
-        for (String timeseries : timeseriesSet) {
-          paths.add(new Path(timeseries));
-        }
-        QueryExpression queryExpression = QueryExpression.create(paths, null);
-        QueryDataSet queryDataSet = readTsFile.query(queryExpression);
-        while (queryDataSet.hasNext()) {
-          RowRecord record = queryDataSet.next();
-          List<Field> fields = record.getFields();
-          List<String> measurementList = new ArrayList<>();
-          List<String> insertValues = new ArrayList<>();
-          for (int i = 0; i < fields.size(); i++) {
-            Field field = fields.get(i);
-            if (!field.isNull()) {
-              measurementList.add(paths.get(i).getMeasurement());
-              if (fields.get(i).getDataType() == TSDataType.TEXT) {
-                insertValues.add(String.format("'%s'", field.toString()));
-              } else {
-                insertValues.add(String.format("%s", field.toString()));
-              }
-            }
-          }
-          if (insertExecutor.insert(new InsertPlan(deviceId, record.getTimestamp(),
-              measurementList.toArray(new String[0]), insertValues.toArray(new String[0])))) {
-            throw new IOException("Inserting series data to IoTDB engine has failed.");
-          }
-        }
-      }
-    } catch (IOException e) {
-      logger.error("Can not parse tsfile into SQL", e);
-      throw new IOException(e);
-    } catch (ProcessorException e) {
-      logger.error("Meet error while processing non-query.");
-      throw new ProcessorException(e);
-    } finally {
-      try {
-        if (reader != null) {
-          reader.close();
-        }
-      } catch (IOException e) {
-        logger.error("Cannot close file stream {}", filePath, e);
-      }
-    }
-  }
-
-  /**
-   * Insert those valid data in the tsfile into IoTDB
-   *
-   * @param overlapFiles:files which are conflict with the sync file
-   */
-  public void loadOldData(String filePath, List<String> overlapFiles)
-      throws IOException, ProcessorException {
-    Set<String> timeseriesList = new HashSet<>();
-    QueryProcessExecutor insertExecutor = new QueryProcessExecutor();
-    Map<String, ReadOnlyTsFile> tsfilesReaders = openReaders(filePath, overlapFiles);
-    try {
-      TsFileSequenceReader reader = new TsFileSequenceReader(filePath);
-      Map<String, TsDeviceMetadataIndex> deviceIdMap = reader.readFileMetadata().getDeviceMap();
-      Iterator<String> it = deviceIdMap.keySet().iterator();
-      while (it.hasNext()) {
-        String deviceID = it.next();
-        TsDeviceMetadataIndex deviceMI = deviceIdMap.get(deviceID);
-        TsDeviceMetadata deviceMetadata = reader.readTsDeviceMetaData(deviceMI);
-        List<ChunkGroupMetaData> chunkGroupMetaDataList = deviceMetadata
-            .getChunkGroupMetaDataList();
-        timeseriesList.clear();
-        /** firstly, get all timeseries in the same device **/
-        for (ChunkGroupMetaData chunkGroupMetaData : chunkGroupMetaDataList) {
-          List<ChunkMetaData> chunkMetaDataList = chunkGroupMetaData.getChunkMetaDataList();
-          for (ChunkMetaData timeSeriesChunkMetaData : chunkMetaDataList) {
-            String measurementUID = timeSeriesChunkMetaData.getMeasurementUid();
-            measurementUID = deviceID + "." + measurementUID;
-            timeseriesList.add(measurementUID);
-          }
-        }
-        reader.close();
-
-        /** secondly, use tsFile Reader to form SQL **/
-        ReadOnlyTsFile readOnlyTsFile = tsfilesReaders.get(filePath);
-        List<Path> paths = new ArrayList<>();
-        /** compare data with one timeseries in a round to get valid data **/
-        for (String timeseries : timeseriesList) {
-          paths.clear();
-          paths.add(new Path(timeseries));
-          Set<InsertPlan> originDataPoints = new HashSet<>();
-          QueryExpression queryExpression = QueryExpression.create(paths, null);
-          QueryDataSet queryDataSet = readOnlyTsFile.query(queryExpression);
-          Set<InsertPlan> newDataPoints = convertToInserPlans(queryDataSet, paths, deviceID);
-
-          /** get all data with the timeseries in all overlap files. **/
-          for (String overlapFile : overlapFiles) {
-            ReadOnlyTsFile readTsFileOverlap = tsfilesReaders.get(overlapFile);
-            QueryDataSet queryDataSetOverlap = readTsFileOverlap.query(queryExpression);
-            originDataPoints.addAll(convertToInserPlans(queryDataSetOverlap, paths, deviceID));
-          }
-
-          /** If there has no overlap data with the timeseries, inserting all data in the sync file **/
-          if (originDataPoints.isEmpty()) {
-            for (InsertPlan insertPlan : newDataPoints) {
-              if (insertExecutor.insert(insertPlan)) {
-                throw new IOException("Inserting series data to IoTDB engine has failed.");
-              }
-            }
-          } else {
-            /** Compare every data to get valid data **/
-            for (InsertPlan insertPlan : newDataPoints) {
-              if (!originDataPoints.contains(insertPlan)) {
-                if (insertExecutor.insert(insertPlan)) {
-                  throw new IOException("Inserting series data to IoTDB engine has failed.");
-                }
-              }
-            }
-          }
-        }
-      }
-    } catch (IOException e) {
-      logger.error("Can not parse tsfile into SQL", e);
-      throw new IOException(e);
-    } catch (ProcessorException e) {
-      logger.error("Meet error while processing non-query.", e);
-      throw new ProcessorException(e);
-    } finally {
-      try {
-        closeReaders(tsfilesReaders);
-      } catch (IOException e) {
-        logger.error("Cannot close file stream {}", filePath, e);
-      }
-    }
-  }
-
-  private Set<InsertPlan> convertToInserPlans(QueryDataSet queryDataSet, List<Path> paths, String deviceID) throws IOException {
-    Set<InsertPlan> plans = new HashSet<>();
-    while (queryDataSet.hasNext()) {
-      RowRecord record = queryDataSet.next();
-      List<Field> fields = record.getFields();
-      /** get all data with the timeseries in the sync file **/
-      for (int i = 0; i < fields.size(); i++) {
-        Field field = fields.get(i);
-        String[] measurementList = new String[1];
-        if (!field.isNull()) {
-          measurementList[0] = paths.get(i).getMeasurement();
-          InsertPlan insertPlan = new InsertPlan(deviceID, record.getTimestamp(),
-              measurementList, new String[]{field.getDataType() == TSDataType.TEXT ? String.format("'%s'", field.toString())
-              : field.toString()});
-          plans.add(insertPlan);
-        }
-      }
-    }
-    return plans;
-  }
-
-  /**
-   * Open all tsfile reader and cache
-   */
-  private Map<String, ReadOnlyTsFile> openReaders(String filePath, List<String> overlapFiles)
-      throws IOException {
-    Map<String, ReadOnlyTsFile> tsfileReaders = new HashMap<>();
-    tsfileReaders.put(filePath, new ReadOnlyTsFile(new TsFileSequenceReader(filePath)));
-    for (String overlapFile : overlapFiles) {
-      tsfileReaders.put(overlapFile, new ReadOnlyTsFile(new TsFileSequenceReader(overlapFile)));
-    }
-    return tsfileReaders;
-  }
-
-  /**
-   * Close all tsfile reader
-   */
-  private void closeReaders(Map<String, ReadOnlyTsFile> readers) throws IOException {
-    for (ReadOnlyTsFile tsfileReader : readers.values()) {
-      tsfileReader.close();
-    }
-  }
-
-  /**
-   * Release threadLocal variable resources
-   */
-  @Override
-  public void cleanUp() {
-    uuid.remove();
-    fileNum.remove();
-    fileNodeMap.remove();
-    fileNodeStartTime.remove();
-    fileNodeEndTime.remove();
-    schemaFromSenderPath.remove();
-    try {
-      FileUtils.deleteDirectory(new File(syncFolderPath));
-    } catch (IOException e) {
-      logger.error("can not delete directory {}", syncFolderPath, e);
-    }
-    logger.info("Synchronization has finished!");
-  }
-
-  public Map<String, List<String>> getFileNodeMap() {
-    return fileNodeMap.get();
-  }
-
-  public void setFileNodeMap(Map<String, List<String>> fileNodeMap) {
-    this.fileNodeMap.set(fileNodeMap);
-  }
-
-}
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/FileLoader.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/FileLoader.java
new file mode 100644
index 0000000..34fb735
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/FileLoader.java
@@ -0,0 +1,234 @@
+/**
+ * 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.sync.receiver.load;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.SyncDeviceOwnerConflictException;
+import org.apache.iotdb.db.exception.TsFileProcessorException;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetaData;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
+import org.apache.iotdb.tsfile.file.metadata.TsDeviceMetadata;
+import org.apache.iotdb.tsfile.file.metadata.TsDeviceMetadataIndex;
+import org.apache.iotdb.tsfile.file.metadata.TsFileMetaData;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileLoader implements IFileLoader {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FileLoader.class);
+
+  public static final int WAIT_TIME = 100;
+
+  private String syncFolderPath;
+
+  private String senderName;
+
+  private BlockingQueue<LoadTask> queue = new LinkedBlockingQueue<>();
+
+  private ILoadLogger loadLog;
+
+  private LoadType curType = LoadType.NONE;
+
+  private volatile boolean endSync = false;
+
+  private FileLoader(String senderName, String syncFolderPath) throws IOException {
+    this.senderName = senderName;
+    this.syncFolderPath = syncFolderPath;
+    this.loadLog = new LoadLogger(new File(syncFolderPath, SyncConstant.LOAD_LOG_NAME));
+  }
+
+  public static FileLoader createFileLoader(String senderName, String syncFolderPath)
+      throws IOException {
+    FileLoader fileLoader = new FileLoader(senderName, syncFolderPath);
+    FileLoaderManager.getInstance().addFileLoader(senderName, fileLoader);
+    FileLoaderManager.getInstance().addLoadTaskRunner(fileLoader.loadTaskRunner);
+    return fileLoader;
+  }
+
+  public static FileLoader createFileLoader(File syncFolder)
+      throws IOException {
+    return createFileLoader(syncFolder.getName(), syncFolder.getAbsolutePath());
+  }
+
+  private Runnable loadTaskRunner = () -> {
+    try {
+      while (true) {
+        if (queue.isEmpty() && endSync) {
+          cleanUp();
+          break;
+        }
+        LoadTask loadTask = queue.poll(WAIT_TIME, TimeUnit.MILLISECONDS);
+        if (loadTask != null) {
+          try {
+            handleLoadTask(loadTask);
+          } catch (Exception e) {
+            LOGGER.error("Can not load task {}", loadTask, e);
+          }
+        }
+      }
+    } catch (InterruptedException e) {
+      LOGGER.error("Can not handle load task", e);
+    }
+  };
+
+  @Override
+  public void addDeletedFileName(File deletedFile) {
+    queue.add(new LoadTask(deletedFile, LoadType.DELETE));
+  }
+
+  @Override
+  public void addTsfile(File tsfile) {
+    queue.add(new LoadTask(tsfile, LoadType.ADD));
+  }
+
+  @Override
+  public void endSync() {
+    if (!endSync && FileLoaderManager.getInstance().containsFileLoader(senderName)) {
+      this.endSync = true;
+    }
+  }
+
+  @Override
+  public void handleLoadTask(LoadTask task) throws IOException {
+    switch (task.type) {
+      case ADD:
+        loadNewTsfile(task.file);
+        break;
+      case DELETE:
+        loadDeletedFile(task.file);
+        break;
+      default:
+        LOGGER.error("Wrong load task type {}", task.type);
+    }
+  }
+
+  private void loadNewTsfile(File newTsFile) throws IOException {
+    if (curType != LoadType.ADD) {
+      loadLog.startLoadTsFiles();
+      curType = LoadType.ADD;
+    }
+    if (!newTsFile.exists()) {
+      LOGGER.info("Tsfile {} doesn't exist.", newTsFile.getAbsolutePath());
+      return;
+    }
+    TsFileResource tsFileResource = new TsFileResource(newTsFile);
+    checkTsFileResource(tsFileResource);
+    try {
+      FileLoaderManager.getInstance().checkAndUpdateDeviceOwner(tsFileResource);
+      StorageEngine.getInstance().loadNewTsFile(tsFileResource);
+    } catch (SyncDeviceOwnerConflictException e) {
+      LOGGER.error("Device owner has conflicts, so skip the loading file", e);
+    } catch (TsFileProcessorException | StorageEngineException e) {
+      LOGGER.error("Can not load new tsfile {}", newTsFile.getAbsolutePath(), e);
+      throw new IOException(e);
+    }
+    loadLog.finishLoadTsfile(newTsFile);
+  }
+
+  private void checkTsFileResource(TsFileResource tsFileResource) throws IOException {
+    if (!tsFileResource.fileExists()) {
+      // .resource file does not exist, read file metadata and recover tsfile resource
+      try (TsFileSequenceReader reader = new TsFileSequenceReader(
+          tsFileResource.getFile().getAbsolutePath())) {
+        TsFileMetaData metaData = reader.readFileMetadata();
+        for (TsDeviceMetadataIndex index : metaData.getDeviceMap().values()) {
+          TsDeviceMetadata deviceMetadata = reader.readTsDeviceMetaData(index);
+          List<ChunkGroupMetaData> chunkGroupMetaDataList = deviceMetadata
+              .getChunkGroupMetaDataList();
+          for (ChunkGroupMetaData chunkGroupMetaData : chunkGroupMetaDataList) {
+            for (ChunkMetaData chunkMetaData : chunkGroupMetaData.getChunkMetaDataList()) {
+              tsFileResource.updateStartTime(chunkGroupMetaData.getDeviceID(),
+                  chunkMetaData.getStartTime());
+              tsFileResource
+                  .updateEndTime(chunkGroupMetaData.getDeviceID(), chunkMetaData.getEndTime());
+            }
+          }
+        }
+      }
+      // write .resource file
+      tsFileResource.serialize();
+    } else {
+      tsFileResource.deSerialize();
+    }
+  }
+
+  private void loadDeletedFile(File deletedTsFile) throws IOException {
+    if (curType != LoadType.DELETE) {
+      loadLog.startLoadDeletedFiles();
+      curType = LoadType.DELETE;
+    }
+    try {
+      StorageEngine.getInstance().deleteTsfile(deletedTsFile);
+    } catch (StorageEngineException e) {
+      LOGGER.error("Can not load deleted tsfile {}", deletedTsFile.getAbsolutePath(), e);
+      throw new IOException(e);
+    }
+    loadLog.finishLoadDeletedFile(deletedTsFile);
+  }
+
+
+  @Override
+  public void cleanUp() {
+    try {
+      loadLog.close();
+      new File(syncFolderPath, SyncConstant.SYNC_LOG_NAME).delete();
+      new File(syncFolderPath, SyncConstant.LOAD_LOG_NAME).delete();
+      FileUtils.deleteDirectory(new File(syncFolderPath, SyncConstant.RECEIVER_DATA_FOLDER_NAME));
+      FileLoaderManager.getInstance().removeFileLoader(senderName);
+      LOGGER.info("Sync loading process for {} has finished.", senderName);
+    } catch (IOException e) {
+      LOGGER.error("Can not clean up sync resource.", e);
+    }
+  }
+
+  @Override
+  public void setCurType(LoadType curType) {
+    this.curType = curType;
+  }
+
+  class LoadTask {
+
+    private File file;
+    private LoadType type;
+
+    LoadTask(File file, LoadType type) {
+      this.file = file;
+      this.type = type;
+    }
+
+    @Override
+    public String toString() {
+      return "LoadTask{" +
+          "file=" + file.getAbsolutePath() +
+          ", type=" + type +
+          '}';
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderManager.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderManager.java
new file mode 100644
index 0000000..c7107f2
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderManager.java
@@ -0,0 +1,215 @@
+/**
+ * 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.sync.receiver.load;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.concurrent.ThreadName;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.SyncDeviceOwnerConflictException;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is to manage all FileLoader.
+ */
+public class FileLoaderManager {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FileLoaderManager.class);
+
+  private static final int WAIT_TIMEOUT = 2000;
+
+  private ConcurrentHashMap<String, IFileLoader> fileLoaderMap;
+
+  private ExecutorService loadTaskRunnerPool;
+
+  private Map<String, String> deviceOwnerMap = new HashMap<>();
+
+  private File deviceOwnerFile;
+
+  private File deviceOwnerTmpFile;
+
+  private FileLoaderManager() {
+    File deviceOwnerMapDir = new File(IoTDBDescriptor.getInstance().getConfig().getSystemDir(),
+        SyncConstant.SYNC_NAME);
+    deviceOwnerFile = new File(deviceOwnerMapDir, SyncConstant.DEVICE_OWNER_FILE_NAME);
+    deviceOwnerTmpFile = new File(deviceOwnerMapDir, SyncConstant.DEVICE_OWNER_TMP_FILE_NAME);
+    try {
+      recoverDeviceOwnerMap();
+    } catch (IOException | ClassNotFoundException e) {
+      LOGGER.error("Can not recover device owner map from file {}",
+          new File(deviceOwnerMapDir, SyncConstant.DEVICE_OWNER_FILE_NAME).getAbsolutePath());
+    }
+  }
+
+  public static FileLoaderManager getInstance() {
+    return FileLoaderManagerHolder.INSTANCE;
+  }
+
+  private void recoverDeviceOwnerMap() throws IOException, ClassNotFoundException {
+    if (deviceOwnerTmpFile.exists()) {
+      deviceOwnerFile.delete();
+      FileUtils.moveFile(deviceOwnerTmpFile, deviceOwnerFile);
+    }
+    if (deviceOwnerFile.exists()) {
+      deSerializeDeviceOwnerMap(deviceOwnerFile);
+    }
+  }
+
+  /**
+   * Check whether there have conflicts about the device owner. If there have conflicts, reject the
+   * sync process of the sg. Otherwise, update the device owners and deserialize.
+   *
+   * @param tsFileResource tsfile resource
+   */
+  public synchronized void checkAndUpdateDeviceOwner(TsFileResource tsFileResource)
+      throws SyncDeviceOwnerConflictException, IOException {
+    String curOwner = tsFileResource.getFile().getParentFile().getParentFile().getParentFile()
+        .getName();
+    Set<String> deviceSet = tsFileResource.getStartTimeMap().keySet();
+    checkDeviceConflict(curOwner, deviceSet);
+    updateDeviceOwner(curOwner, deviceSet);
+  }
+
+  /**
+   * Check whether there have conflicts about the device owner.
+   *
+   * @param curOwner sender name that want to be owner.
+   * @param deviceSet device set
+   */
+  private void checkDeviceConflict(String curOwner, Set<String> deviceSet)
+      throws SyncDeviceOwnerConflictException {
+    for (String device : deviceSet) {
+      if (deviceOwnerMap.containsKey(device) && !deviceOwnerMap.get(device).equals(curOwner)) {
+        throw new SyncDeviceOwnerConflictException(String
+            .format("Device: %s, correct owner: %s, conflict owner: %s", device,
+                deviceOwnerMap.get(device), curOwner));
+      }
+    }
+  }
+
+  /**
+   * Update the device owners and deserialize.
+   *
+   * @param curOwner sender name that want to be owner.
+   * @param deviceSet device set.
+   */
+  private void updateDeviceOwner(String curOwner, Set<String> deviceSet) throws IOException {
+    boolean modify = false;
+    for (String device : deviceSet) {
+      if (!deviceOwnerMap.containsKey(device)) {
+        deviceOwnerMap.put(device, curOwner);
+        modify = true;
+      }
+    }
+    if (modify) {
+      serializeDeviceOwnerMap(deviceOwnerTmpFile);
+      deviceOwnerFile.delete();
+      FileUtils.moveFile(deviceOwnerTmpFile, deviceOwnerFile);
+    }
+  }
+
+  private void deSerializeDeviceOwnerMap(File deviceOwnerFile)
+      throws IOException, ClassNotFoundException {
+    try (ObjectInputStream deviceOwnerInput = new ObjectInputStream(
+        new FileInputStream(deviceOwnerFile))) {
+      deviceOwnerMap = (Map<String, String>) deviceOwnerInput.readObject();
+    }
+  }
+
+  private void serializeDeviceOwnerMap(File deviceOwnerFile) throws IOException {
+    if (!deviceOwnerFile.getParentFile().exists()) {
+      deviceOwnerFile.getParentFile().mkdirs();
+    }
+    if (!deviceOwnerFile.exists()) {
+      deviceOwnerFile.createNewFile();
+    }
+    try (ObjectOutputStream deviceOwnerOutput = new ObjectOutputStream(
+        new FileOutputStream(deviceOwnerFile, false))) {
+      deviceOwnerOutput.writeObject(deviceOwnerMap);
+    }
+  }
+
+  public void addFileLoader(String senderName, IFileLoader fileLoader) {
+    fileLoaderMap.put(senderName, fileLoader);
+  }
+
+  public void removeFileLoader(String senderName) {
+    fileLoaderMap.remove(senderName);
+  }
+
+  public IFileLoader getFileLoader(String senderName) {
+    return fileLoaderMap.get(senderName);
+  }
+
+  public boolean containsFileLoader(String senderName) {
+    return fileLoaderMap.containsKey(senderName);
+  }
+
+  public void addLoadTaskRunner(Runnable taskRunner) {
+    loadTaskRunnerPool.submit(taskRunner);
+  }
+
+  public void start() {
+    if (fileLoaderMap == null) {
+      fileLoaderMap = new ConcurrentHashMap<>();
+    }
+    if (loadTaskRunnerPool == null) {
+      loadTaskRunnerPool = IoTDBThreadPoolFactory
+          .newCachedThreadPool(ThreadName.LOAD_TSFILE.getName());
+    }
+  }
+
+  public void stop() {
+    fileLoaderMap = null;
+    loadTaskRunnerPool.shutdownNow();
+    int totalWaitTime = WAIT_TIMEOUT;
+    while (!loadTaskRunnerPool.isTerminated()) {
+      try {
+        if (!loadTaskRunnerPool.awaitTermination(WAIT_TIMEOUT, TimeUnit.MILLISECONDS)) {
+          LOGGER.info("File load manager thread pool doesn't exit after {}ms.",
+              +totalWaitTime);
+        }
+        totalWaitTime += WAIT_TIMEOUT;
+      } catch (InterruptedException e) {
+        LOGGER.error("Interrupted while waiting file load manager thread pool to exit. ", e);
+      }
+    }
+    loadTaskRunnerPool = null;
+  }
+
+  private static class FileLoaderManagerHolder {
+
+    private static final FileLoaderManager INSTANCE = new FileLoaderManager();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/IFileLoader.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/IFileLoader.java
new file mode 100644
index 0000000..9c6b3b3
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/IFileLoader.java
@@ -0,0 +1,60 @@
+/**
+ * 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.sync.receiver.load;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.iotdb.db.sync.receiver.load.FileLoader.LoadTask;
+
+/**
+ * This interface is used to load files, including deleted files and new tsfiles. The
+ * producer-consumer model is used to load files. A background consumer thread is used to load
+ * files. There is a queue recording tasks. After receiving a file, the receiver adds a task to the
+ * queue. When all files are loaded and the synchronization task is completed, the thread is
+ * closed.
+ */
+public interface IFileLoader {
+
+  /**
+   * Add a deleted file name to be loaded.
+   */
+  void addDeletedFileName(File deletedFile);
+
+  /**
+   * Add a new tsfile to be loaded.
+   */
+  void addTsfile(File tsfile);
+
+  /**
+   * Mark sync end.
+   */
+  void endSync();
+
+  /**
+   * Handle load task by type.
+   */
+  void handleLoadTask(LoadTask task) throws IOException;
+
+  /**
+   * Set current load type
+   */
+  void setCurType(LoadType curType);
+
+  void cleanUp();
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/ILoadLogger.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/ILoadLogger.java
new file mode 100644
index 0000000..a4a6429
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/ILoadLogger.java
@@ -0,0 +1,62 @@
+/**
+ * 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.sync.receiver.load;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * This interface is used to log progress in the process of loading deleted files and new tsfiles.
+ * If the loading tasks are completed normally and there are no exceptions, the log records will be
+ * deleted; otherwise, the status can be restored according to the log at the start of each task. It
+ * ensures the correctness of synchronization module when system crashed or network abnormality
+ * occurred.
+ */
+public interface ILoadLogger {
+
+  String LOAD_DELETED_FILE_NAME_START = "load deleted files start";
+  String LOAD_TSFILE_START = "load tsfile start";
+
+  /**
+   * Start to load deleted files.
+   */
+  void startLoadDeletedFiles() throws IOException;
+
+  /**
+   * After a deleted file is loaded, record it in load log.
+   *
+   * @param file deleted file to be loaded
+   */
+  void finishLoadDeletedFile(File file) throws IOException;
+
+  /**
+   * Start to load tsfiles
+   */
+  void startLoadTsFiles() throws IOException;
+
+  /**
+   * After a new tsfile is loaded, record it in load log.
+   *
+   * @param file new tsfile to be loaded
+   */
+  void finishLoadTsfile(File file) throws IOException;
+
+  void close() throws IOException;
+
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadLogger.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadLogger.java
new file mode 100644
index 0000000..f0e1860
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadLogger.java
@@ -0,0 +1,72 @@
+/**
+ * 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.sync.receiver.load;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+
+public class LoadLogger implements ILoadLogger {
+
+  private BufferedWriter bw;
+
+  public LoadLogger(File logFile) throws IOException {
+    if (!logFile.getParentFile().exists()) {
+      logFile.getParentFile().mkdirs();
+    }
+    bw = new BufferedWriter(new FileWriter(logFile));
+  }
+
+  @Override
+  public void startLoadDeletedFiles() throws IOException {
+    bw.write(LOAD_DELETED_FILE_NAME_START);
+    bw.newLine();
+    bw.flush();
+  }
+
+  @Override
+  public void finishLoadDeletedFile(File file) throws IOException {
+    bw.write(file.getAbsolutePath());
+    bw.newLine();
+    bw.flush();
+  }
+
+  @Override
+  public void startLoadTsFiles() throws IOException {
+    bw.write(LOAD_TSFILE_START);
+    bw.newLine();
+    bw.flush();
+  }
+
+  @Override
+  public void finishLoadTsfile(File file) throws IOException {
+    bw.write(file.getAbsolutePath());
+    bw.newLine();
+    bw.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if(bw != null) {
+      bw.close();
+      bw = null;
+    }
+  }
+}
diff --git a/service-rpc/src/main/thrift/sync.thrift b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
old mode 100755
new mode 100644
similarity index 60%
copy from service-rpc/src/main/thrift/sync.thrift
copy to server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
index fec6079..4742093
--- a/service-rpc/src/main/thrift/sync.thrift
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
@@ -1,38 +1,23 @@
-/*
- * 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.
- */
-namespace java org.apache.iotdb.service.sync.thrift
-
-typedef i32 int 
-typedef i16 short
-typedef i64 long
-
-enum SyncDataStatus {
-  SUCCESS_STATUS,
-  FINISH_STATUS,
-  PROCESSING_STATUS
-}
-
-service SyncService{
-	bool checkIdentity(1:string uuid, 2:string address)
-	string syncSchema(1:string md5, 2:binary buff, 3:SyncDataStatus status)
-	string syncData(1:string md5, 2:list<string> filename, 3:binary buff, 4:SyncDataStatus status)
-	bool load()
-	void cleanUp()
-	bool init(1:string storageGroupName)
-}
\ No newline at end of file
+/**
+ * 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.sync.receiver.load;
+
+public enum LoadType {
+    DELETE, ADD, NONE
+}
diff --git a/service-rpc/src/main/thrift/sync.thrift b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
old mode 100755
new mode 100644
similarity index 60%
copy from service-rpc/src/main/thrift/sync.thrift
copy to server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
index fec6079..ffa0e8d
--- a/service-rpc/src/main/thrift/sync.thrift
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
@@ -1,38 +1,34 @@
-/*
- * 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.
- */
-namespace java org.apache.iotdb.service.sync.thrift
-
-typedef i32 int 
-typedef i16 short
-typedef i64 long
-
-enum SyncDataStatus {
-  SUCCESS_STATUS,
-  FINISH_STATUS,
-  PROCESSING_STATUS
-}
-
-service SyncService{
-	bool checkIdentity(1:string uuid, 2:string address)
-	string syncSchema(1:string md5, 2:binary buff, 3:SyncDataStatus status)
-	string syncData(1:string md5, 2:list<string> filename, 3:binary buff, 4:SyncDataStatus status)
-	bool load()
-	void cleanUp()
-	bool init(1:string storageGroupName)
-}
\ No newline at end of file
+/*
+ * 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.sync.receiver.recover;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.iotdb.db.sync.receiver.load.FileLoader;
+import org.apache.iotdb.db.sync.receiver.load.IFileLoader;
+
+public interface ISyncReceiverLogAnalyzer {
+
+  void recoverAll() throws IOException;
+
+  boolean recover(String senderName) throws IOException;
+
+  void scanLogger(IFileLoader loader, File syncLog, File loadLog);
+
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogger.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogger.java
new file mode 100644
index 0000000..200afa3
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogger.java
@@ -0,0 +1,54 @@
+/**
+ * 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.sync.receiver.recover;
+
+import java.io.File;
+import java.io.IOException;
+
+public interface ISyncReceiverLogger {
+
+  String SYNC_DELETED_FILE_NAME_START = "sync deleted file names start";
+  String SYNC_TSFILE_START = "sync tsfile start";
+
+  /**
+   * Start to sync deleted files name
+   */
+  void startSyncDeletedFilesName() throws IOException;
+
+  /**
+   * After a deleted file name is synced to the receiver end, record it in sync log.
+   *
+   * @param file the deleted tsfile
+   */
+  void finishSyncDeletedFileName(File file) throws IOException;
+
+  /**
+   * Start to sync new tsfiles
+   */
+  void startSyncTsFiles() throws IOException;
+
+  /**
+   * After a new tsfile is synced to the receiver end, record it in sync log.
+   *
+   * @param file new tsfile
+   */
+  void finishSyncTsfile(File file) throws IOException;
+
+  void close() throws IOException;
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogAnalyzer.java
new file mode 100644
index 0000000..912d4b4
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogAnalyzer.java
@@ -0,0 +1,153 @@
+/**
+ * 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.sync.receiver.recover;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.sync.receiver.load.FileLoader;
+import org.apache.iotdb.db.sync.receiver.load.FileLoaderManager;
+import org.apache.iotdb.db.sync.receiver.load.IFileLoader;
+import org.apache.iotdb.db.sync.receiver.load.LoadLogger;
+import org.apache.iotdb.db.sync.receiver.load.LoadType;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.db.utils.FilePathUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SyncReceiverLogAnalyzer implements ISyncReceiverLogAnalyzer {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(SyncReceiverLogAnalyzer.class);
+
+  private SyncReceiverLogAnalyzer() {
+
+  }
+
+  public static SyncReceiverLogAnalyzer getInstance() {
+    return SyncReceiverLogAnalyzerHolder.INSTANCE;
+  }
+
+  @Override
+  public void recoverAll() throws IOException {
+    String[] dataDirs = IoTDBDescriptor.getInstance().getConfig().getDataDirs();
+    LOGGER.info("Start to recover all sync state for sync receiver.");
+    for (String dataDir : dataDirs) {
+      if (!new File(FilePathUtils.regularizePath(dataDir) + SyncConstant.SYNC_RECEIVER).exists()) {
+        continue;
+      }
+      for (File syncFolder : new File(
+          FilePathUtils.regularizePath(dataDir) + SyncConstant.SYNC_RECEIVER)
+          .listFiles()) {
+        recover(syncFolder);
+      }
+    }
+    LOGGER.info("Finish to recover all sync states for sync receiver.");
+  }
+
+  private boolean recover(File senderFolder) throws IOException {
+    // check the state
+    if (!new File(senderFolder, SyncConstant.SYNC_LOG_NAME).exists()) {
+      new File(senderFolder, SyncConstant.LOAD_LOG_NAME).delete();
+      FileUtils.deleteDirectory(new File(senderFolder, SyncConstant.RECEIVER_DATA_FOLDER_NAME));
+      return true;
+    }
+    if (FileLoaderManager.getInstance().containsFileLoader(senderFolder.getName())) {
+      FileLoaderManager.getInstance().getFileLoader(senderFolder.getName()).endSync();
+      try {
+        Thread.sleep(FileLoader.WAIT_TIME << 1);
+      } catch (InterruptedException e) {
+        LOGGER.error("Thread is interrupted from waiting for ending sync in recovery.");
+      }
+    } else {
+      scanLogger(FileLoader.createFileLoader(senderFolder),
+          new File(senderFolder, SyncConstant.SYNC_LOG_NAME),
+          new File(senderFolder, SyncConstant.LOAD_LOG_NAME));
+    }
+    return !FileLoaderManager.getInstance().containsFileLoader(senderFolder.getName());
+  }
+
+  @Override
+  public boolean recover(String senderName) throws IOException {
+    String[] dataDirs = IoTDBDescriptor.getInstance().getConfig().getDataDirs();
+    boolean recoverComplete = true;
+    for (String dataDir : dataDirs) {
+      if (!new File(FilePathUtils.regularizePath(dataDir) + SyncConstant.SYNC_RECEIVER).exists()) {
+        continue;
+      }
+      for (File syncFolder : new File(
+          FilePathUtils.regularizePath(dataDir) + SyncConstant.SYNC_RECEIVER)
+          .listFiles()) {
+        if (syncFolder.getName().equals(senderName)) {
+          recoverComplete &= recover(syncFolder);
+        }
+      }
+    }
+    return recoverComplete;
+  }
+
+  @Override
+  public void scanLogger(IFileLoader loader, File syncLog, File loadLog) {
+    LoadType loadType = LoadType.NONE;
+    try (BufferedReader syncReader = new BufferedReader(new FileReader(syncLog))) {
+      String line;
+      try (BufferedReader loadReader = new BufferedReader(new FileReader(loadLog))) {
+        while ((line = loadReader.readLine()) != null) {
+          if (line.equals(LoadLogger.LOAD_DELETED_FILE_NAME_START)) {
+            loadType = LoadType.DELETE;
+          } else if (line.equals(LoadLogger.LOAD_TSFILE_START)) {
+            loadType = LoadType.ADD;
+          } else {
+            while (!syncReader.readLine().equals(line)) {
+            }
+          }
+        }
+      }
+      loader.setCurType(loadType);
+      while ((line = syncReader.readLine()) != null) {
+        if (line.equals(SyncReceiverLogger.SYNC_DELETED_FILE_NAME_START)) {
+          loadType = LoadType.DELETE;
+        } else if (line.equals(SyncReceiverLogger.SYNC_TSFILE_START)) {
+          loadType = LoadType.ADD;
+        } else {
+          switch (loadType) {
+            case ADD:
+              loader.addTsfile(new File(line));
+              break;
+            case DELETE:
+              loader.addDeletedFileName(new File(line));
+              break;
+            default:
+              LOGGER.error("Wrong load type {}", loadType);
+          }
+        }
+      }
+      loader.endSync();
+    } catch (IOException e) {
+      LOGGER.error("Can not scan log for recovery", e);
+    }
+  }
+
+  private static class SyncReceiverLogAnalyzerHolder {
+
+    private static final SyncReceiverLogAnalyzer INSTANCE = new SyncReceiverLogAnalyzer();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogger.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogger.java
new file mode 100644
index 0000000..00b1175
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogger.java
@@ -0,0 +1,72 @@
+/**
+ * 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.sync.receiver.recover;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+
+public class SyncReceiverLogger implements ISyncReceiverLogger {
+
+  private BufferedWriter bw;
+
+  public SyncReceiverLogger(File logFile) throws IOException {
+    if (!logFile.getParentFile().exists()) {
+      logFile.getParentFile().mkdirs();
+    }
+    bw = new BufferedWriter(new FileWriter(logFile));
+  }
+
+  @Override
+  public void startSyncDeletedFilesName() throws IOException {
+    bw.write(SYNC_DELETED_FILE_NAME_START);
+    bw.newLine();
+    bw.flush();
+  }
+
+  @Override
+  public void finishSyncDeletedFileName(File file) throws IOException {
+    bw.write(file.getAbsolutePath());
+    bw.newLine();
+    bw.flush();
+  }
+
+  @Override
+  public void startSyncTsFiles() throws IOException {
+    bw.write(SYNC_TSFILE_START);
+    bw.newLine();
+    bw.flush();
+  }
+
+  @Override
+  public void finishSyncTsfile(File file) throws IOException {
+    bw.write(file.getAbsolutePath());
+    bw.newLine();
+    bw.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if(bw != null) {
+      bw.close();
+      bw = null;
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/transfer/SyncServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/transfer/SyncServiceImpl.java
new file mode 100644
index 0000000..9c9c1bd
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/transfer/SyncServiceImpl.java
@@ -0,0 +1,304 @@
+/**
+ * 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.sync.receiver.transfer;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import org.apache.iotdb.db.concurrent.ThreadName;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.db.exception.MetadataErrorException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.SyncDeviceOwnerConflictException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.db.sync.receiver.load.FileLoader;
+import org.apache.iotdb.db.sync.receiver.load.FileLoaderManager;
+import org.apache.iotdb.db.sync.receiver.load.IFileLoader;
+import org.apache.iotdb.db.sync.receiver.recover.SyncReceiverLogAnalyzer;
+import org.apache.iotdb.db.sync.receiver.recover.SyncReceiverLogger;
+import org.apache.iotdb.db.utils.FilePathUtils;
+import org.apache.iotdb.db.utils.SyncUtils;
+import org.apache.iotdb.service.sync.thrift.ResultStatus;
+import org.apache.iotdb.service.sync.thrift.SyncService;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SyncServiceImpl implements SyncService.Iface {
+
+  private static final Logger logger = LoggerFactory.getLogger(SyncServiceImpl.class);
+
+  private IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  private ThreadLocal<String> syncFolderPath = new ThreadLocal<>();
+
+  private ThreadLocal<String> currentSG = new ThreadLocal<>();
+
+  private ThreadLocal<SyncReceiverLogger> syncLog = new ThreadLocal<>();
+
+  private ThreadLocal<String> senderName = new ThreadLocal<>();
+
+  private ThreadLocal<File> currentFile = new ThreadLocal<>();
+
+  private ThreadLocal<FileChannel> currentFileWriter = new ThreadLocal<>();
+
+  private ThreadLocal<MessageDigest> messageDigest = new ThreadLocal<>();
+
+  /**
+   * Verify IP address of sender
+   */
+  @Override
+  public ResultStatus check(String ipAddress, String uuid) {
+    Thread.currentThread().setName(ThreadName.SYNC_SERVER.getName());
+    if (SyncUtils.verifyIPSegment(config.getIpWhiteList(), ipAddress)) {
+      senderName.set(ipAddress + SyncConstant.SYNC_DIR_NAME_SEPARATOR + uuid);
+      if (checkRecovery()) {
+        logger.info("Start to sync with sender {}", senderName.get());
+        return getSuccessResult();
+      } else {
+        return getErrorResult("Receiver is processing data from previous sync tasks");
+      }
+    } else {
+      return getErrorResult(
+          "Sender IP is not in the white list of receiver IP and synchronization tasks are not allowed.");
+    }
+  }
+
+  private boolean checkRecovery() {
+    try {
+      if (currentFileWriter.get() != null && currentFileWriter.get().isOpen()) {
+        currentFileWriter.get().close();
+      }
+      if (syncLog.get() != null) {
+        syncLog.get().close();
+      }
+      return SyncReceiverLogAnalyzer.getInstance().recover(senderName.get());
+    } catch (IOException e) {
+      logger.error("Check recovery state fail", e);
+      return false;
+    }
+  }
+
+  @Override
+  public ResultStatus startSync() {
+    try {
+      initPath();
+      currentSG.remove();
+      FileLoader.createFileLoader(senderName.get(), syncFolderPath.get());
+      syncLog
+          .set(new SyncReceiverLogger(new File(syncFolderPath.get(), SyncConstant.SYNC_LOG_NAME)));
+      return getSuccessResult();
+    } catch (DiskSpaceInsufficientException | IOException e) {
+      logger.error("Can not receiver data from sender", e);
+      return getErrorResult(e.getMessage());
+    }
+  }
+
+  /**
+   * Init file path.
+   */
+  private void initPath() throws DiskSpaceInsufficientException {
+    String dataDir = new File(DirectoryManager.getInstance().getNextFolderForSequenceFile())
+        .getParentFile().getAbsolutePath();
+    syncFolderPath
+        .set(FilePathUtils.regularizePath(dataDir) + SyncConstant.SYNC_RECEIVER + File.separatorChar
+            + senderName.get());
+  }
+
+  /**
+   * Init threadLocal variable.
+   */
+  @Override
+  public ResultStatus init(String storageGroup) {
+    logger.info("Sync process started to receive data of storage group {}", storageGroup);
+    currentSG.set(storageGroup);
+    try {
+      syncLog.get().startSyncDeletedFilesName();
+    } catch (IOException e) {
+      logger.error("Can not init sync process", e);
+      return getErrorResult(e.getMessage());
+    }
+    return getSuccessResult();
+  }
+
+  @Override
+  public ResultStatus syncDeletedFileName(String fileName) throws TException {
+    try {
+      syncLog.get().finishSyncDeletedFileName(
+          new File(getSyncDataPath(), currentSG.get() + File.separatorChar + fileName));
+      FileLoaderManager.getInstance().getFileLoader(senderName.get()).addDeletedFileName(
+          new File(getSyncDataPath(), currentSG.get() + File.separatorChar + fileName));
+    } catch (IOException e) {
+      logger.error("Can not sync deleted file", e);
+      return getErrorResult(
+          String.format("Can not sync deleted file %s because %s", fileName, e.getMessage()));
+    }
+    return getSuccessResult();
+  }
+
+  @Override
+  public ResultStatus initSyncData(String filename) throws TException {
+    try {
+      File file;
+      if (currentSG.get() == null) { // schema mlog.txt file
+        file = new File(getSyncDataPath(), filename);
+      } else {
+        file = new File(getSyncDataPath(), currentSG.get() + File.separatorChar + filename);
+      }
+      file.delete();
+      currentFile.set(file);
+      if (!file.getParentFile().exists()) {
+        file.getParentFile().mkdirs();
+      }
+      if (currentFileWriter.get() != null && currentFileWriter.get().isOpen()) {
+        currentFileWriter.get().close();
+      }
+      currentFileWriter.set(new FileOutputStream(file).getChannel());
+      syncLog.get().startSyncTsFiles();
+      messageDigest.set(MessageDigest.getInstance(SyncConstant.MESSAGE_DIGIT_NAME));
+    } catch (IOException | NoSuchAlgorithmException e) {
+      logger.error("Can not init sync resource for file {}", filename, e);
+      return getErrorResult(
+          String.format("Can not init sync resource for file %s because %s", filename,
+              e.getMessage()));
+    }
+    return getSuccessResult();
+  }
+
+  @Override
+  public ResultStatus syncData(ByteBuffer buff) {
+    try {
+      currentFileWriter.get().write(buff);
+      buff.flip();
+      messageDigest.get().update(buff);
+    } catch (IOException e) {
+      logger.error("Can not sync data for file {}", currentFile.get().getAbsoluteFile(), e);
+      return getErrorResult(String
+          .format("Can not sync data for file %s because %s", currentFile.get().getName(),
+              e.getMessage()));
+    }
+    return getSuccessResult();
+  }
+
+  @Override
+  public ResultStatus checkDataMD5(String md5OfSender) throws TException {
+    String md5OfReceiver = (new BigInteger(1, messageDigest.get().digest())).toString(16);
+    try {
+      if (currentFileWriter.get() != null && currentFileWriter.get().isOpen()) {
+        currentFileWriter.get().close();
+      }
+      if (!md5OfSender.equals(md5OfReceiver)) {
+        currentFile.get().delete();
+        currentFileWriter.set(new FileOutputStream(currentFile.get()).getChannel());
+        return getErrorResult(String
+            .format("MD5 of the sender is differ from MD5 of the receiver of the file %s.",
+                currentFile.get().getAbsolutePath()));
+      } else {
+        if (currentFile.get().getName().endsWith(MetadataConstant.METADATA_LOG)) {
+          loadMetadata();
+        } else {
+          if (!currentFile.get().getName().endsWith(TsFileResource.RESOURCE_SUFFIX)) {
+            logger.info("Receiver has received {} successfully.", currentFile.get());
+            FileLoaderManager.getInstance().checkAndUpdateDeviceOwner(
+                new TsFileResource(new File(currentFile.get() + TsFileResource.RESOURCE_SUFFIX)));
+            syncLog.get().finishSyncTsfile(currentFile.get());
+            FileLoaderManager.getInstance().getFileLoader(senderName.get())
+                .addTsfile(currentFile.get());
+          }
+        }
+      }
+    } catch (IOException e) {
+      logger.error("Can not check data MD5 for file {}", currentFile.get().getAbsoluteFile(), e);
+      return getErrorResult(String
+          .format("Can not check data MD5 for file %s because %s", currentFile.get().getName(),
+              e.getMessage()));
+    } catch (SyncDeviceOwnerConflictException e) {
+      logger.error("Device owner has conflicts, skip all other tsfiles in the sg {}.", currentSG.get());
+      return new ResultStatus(false, String
+          .format("Device owner has conflicts, skip all other tsfiles in the same sg %s because %s",
+              currentSG.get(), e.getMessage()), -2);
+    }
+    return new ResultStatus(true, md5OfReceiver, 1);
+  }
+
+  private void loadMetadata() {
+    logger.info("Start to load metadata in sync process.");
+    if (currentFile.get().exists()) {
+      try (BufferedReader br = new BufferedReader(
+          new java.io.FileReader(currentFile.get()))) {
+        String metadataOperation;
+        while ((metadataOperation = br.readLine()) != null) {
+          try {
+            MManager.getInstance().operation(metadataOperation);
+          } catch (IOException | MetadataErrorException | PathErrorException e) {
+            logger.error("Can not operate metadata operation {} ", metadataOperation, e);
+          }
+        }
+      } catch (IOException e) {
+        logger.error("Cannot read the file {}.", currentFile.get().getAbsoluteFile(), e);
+      }
+    }
+  }
+
+  @Override
+  public ResultStatus endSync() throws TException {
+    try {
+      if (syncLog.get() != null) {
+        syncLog.get().close();
+      }
+      IFileLoader loader = FileLoaderManager.getInstance().getFileLoader(senderName.get());
+      if (loader != null) {
+        loader.endSync();
+      } else {
+        return getErrorResult(
+            String.format("File Loader of the storage group %s is null", currentSG.get()));
+      }
+      logger.info("Sync process with sender {} finished.", senderName.get());
+    } catch (IOException e) {
+      logger.error("Can not end sync", e);
+      return getErrorResult(String.format("Can not end sync because %s", e.getMessage()));
+    }
+    return getSuccessResult();
+  }
+
+  private String getSyncDataPath() {
+    return syncFolderPath.get() + File.separatorChar + SyncConstant.RECEIVER_DATA_FOLDER_NAME;
+  }
+
+  private ResultStatus getSuccessResult() {
+    return new ResultStatus(true, null, 1);
+  }
+
+  private ResultStatus getErrorResult(String errorMsg) {
+    return new ResultStatus(false, errorMsg, -1);
+  }
+
+}
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncFileManager.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncFileManager.java
deleted file mode 100644
index 7419cfd..0000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncFileManager.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/*
- * 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.sync.sender;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.sync.conf.Constans;
-import org.apache.iotdb.db.sync.conf.SyncSenderConfig;
-import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * SyncFileManager is used to pick up those tsfiles need to sync.
- */
-public class SyncFileManager {
-
-  private static final Logger logger = LoggerFactory.getLogger(SyncFileManager.class);
-
-  /**
-   * Files that need to be synchronized
-   **/
-  private Map<String, Set<String>> validAllFiles = new HashMap<>();
-
-  /**
-   * All tsfiles in last synchronization process
-   **/
-  private Set<String> lastLocalFiles = new HashSet<>();
-
-  /**
-   * All tsfiles in data directory
-   **/
-  private Map<String, Set<String>> currentLocalFiles = new HashMap<>();
-
-  private SyncSenderConfig syncConfig = SyncSenderDescriptor.getInstance().getConfig();
-
-  private IoTDBConfig systemConfig = IoTDBDescriptor.getInstance().getConfig();
-
-  private static final String RESTORE_SUFFIX = ".restore";
-
-  private SyncFileManager() {
-  }
-
-  public static final SyncFileManager getInstance() {
-    return FileManagerHolder.INSTANCE;
-  }
-
-  /**
-   * Initialize SyncFileManager.
-   */
-  public void init() throws IOException {
-    validAllFiles.clear();
-    lastLocalFiles.clear();
-    currentLocalFiles.clear();
-    getLastLocalFileList(syncConfig.getLastFileInfo());
-    getCurrentLocalFileList(systemConfig.getDataDirs());
-    getValidFileList();
-  }
-
-  /**
-   * get files that needs to be synchronized
-   */
-  public void getValidFileList() {
-    for (Entry<String, Set<String>> entry : currentLocalFiles.entrySet()) {
-      for (String path : entry.getValue()) {
-        if (!lastLocalFiles.contains(path)) {
-          validAllFiles.get(entry.getKey()).add(path);
-        }
-      }
-    }
-    logger.info("Acquire list of valid files.");
-    for (Entry<String, Set<String>> entry : validAllFiles.entrySet()) {
-      for (String path : entry.getValue()) {
-        currentLocalFiles.get(entry.getKey()).remove(path);
-      }
-    }
-  }
-
-  /**
-   * get last local file list.
-   *
-   * @param path path
-   */
-  public void getLastLocalFileList(String path) throws IOException {
-    Set<String> fileList = new HashSet<>();
-    File file = new File(path);
-    if (!file.exists()) {
-      try {
-        file.createNewFile();
-      } catch (IOException e) {
-        throw new IOException("Cannot get last local file list", e);
-      }
-    } else {
-      try (BufferedReader bf = new BufferedReader(new FileReader(file))) {
-        String fileName;
-        while ((fileName = bf.readLine()) != null) {
-          fileList.add(fileName);
-        }
-      } catch (IOException e) {
-        logger.error("Cannot get last local file list when reading file {}.",
-            syncConfig.getLastFileInfo());
-        throw new IOException(e);
-      }
-    }
-    lastLocalFiles = fileList;
-  }
-
-  /**
-   * get current local file list.
-   *
-   * @param paths paths in String[] structure
-   */
-  public void getCurrentLocalFileList(String[] paths) {
-    for (String path : paths) {
-      if (!new File(path).exists()) {
-        continue;
-      }
-      File[] listFiles = new File(path).listFiles();
-      for (File storageGroup : listFiles) {
-        if (!storageGroup.isDirectory() || storageGroup.getName().equals(Constans.SYNC_CLIENT)) {
-          continue;
-        }
-        getStorageGroupFiles(storageGroup);
-      }
-    }
-  }
-
-  private void getStorageGroupFiles(File storageGroup) {
-    if (!currentLocalFiles.containsKey(storageGroup.getName())) {
-      currentLocalFiles.put(storageGroup.getName(), new HashSet<>());
-    }
-    if (!validAllFiles.containsKey(storageGroup.getName())) {
-      validAllFiles.put(storageGroup.getName(), new HashSet<>());
-    }
-    File[] files = storageGroup.listFiles();
-    for (File file : files) {
-      if (!file.getPath().endsWith(RESTORE_SUFFIX) && !new File(
-          file.getPath() + RESTORE_SUFFIX).exists()) {
-        currentLocalFiles.get(storageGroup.getName()).add(file.getPath());
-      }
-    }
-  }
-
-  /**
-   * backup current local file information.
-   *
-   * @param backupFile backup file path
-   */
-  public void backupNowLocalFileInfo(String backupFile) {
-    try (BufferedWriter bufferedWriter = new BufferedWriter(new FileWriter(backupFile))) {
-      for (Entry<String, Set<String>> entry : currentLocalFiles.entrySet()) {
-        for (String file : entry.getValue()) {
-          bufferedWriter.write(file + "\n");
-        }
-      }
-    } catch (IOException e) {
-      logger.error("Cannot back up current local file info", e);
-    }
-  }
-
-  public Map<String, Set<String>> getValidAllFiles() {
-    return validAllFiles;
-  }
-
-  public Set<String> getLastLocalFiles() {
-    return lastLocalFiles;
-  }
-
-  public Map<String, Set<String>> getCurrentLocalFiles() {
-    return currentLocalFiles;
-  }
-
-  public void setCurrentLocalFiles(Map<String, Set<String>> newNowLocalFiles) {
-    currentLocalFiles = newNowLocalFiles;
-  }
-
-  private static class FileManagerHolder {
-
-    private static final SyncFileManager INSTANCE = new SyncFileManager();
-  }
-}
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncSender.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncSender.java
deleted file mode 100644
index 23d9c80..0000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncSender.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.sync.sender;
-
-import java.io.IOException;
-import java.util.Set;
-import org.apache.iotdb.db.exception.SyncConnectionException;
-
-/**
- * SyncSender defines the methods of a sender in sync module.
- */
-public interface SyncSender {
-
-  /**
-   * Init
-   */
-  void init();
-
-  /**
-   * Connect to server.
-   */
-  void establishConnection(String serverIp, int serverPort) throws SyncConnectionException;
-
-  /**
-   * Transfer UUID to receiver.
-   */
-  boolean confirmIdentity(String uuidPath) throws SyncConnectionException, IOException;
-
-  /**
-   * Make file snapshots before sending files.
-   */
-  Set<String> makeFileSnapshot(Set<String> validFiles) throws IOException;
-
-  /**
-   * Send schema file to receiver.
-   */
-  void syncSchema() throws SyncConnectionException;
-
-  /**
-   * For all valid files, send it to receiver side and load these data in receiver.
-   */
-  void syncAllData() throws SyncConnectionException;
-
-  /**
-   * Close the socket after sending files.
-   */
-  boolean afterSynchronization() throws SyncConnectionException;
-
-  /**
-   * Execute a sync task.
-   */
-  void sync() throws SyncConnectionException, IOException;
-
-  /**
-   * Stop sync process
-   */
-  void stop();
-
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncSenderImpl.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncSenderImpl.java
deleted file mode 100644
index a340730..0000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncSenderImpl.java
+++ /dev/null
@@ -1,545 +0,0 @@
-/**
- * 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,
- * "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.sync.sender;
-
-import java.io.BufferedReader;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.math.BigInteger;
-import java.net.InetAddress;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileLock;
-import java.nio.file.FileSystems;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.security.MessageDigest;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.apache.commons.io.FileUtils;
-import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
-import org.apache.iotdb.db.concurrent.ThreadName;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.SyncConnectionException;
-import org.apache.iotdb.db.sync.conf.Constans;
-import org.apache.iotdb.db.sync.conf.SyncSenderConfig;
-import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
-import org.apache.iotdb.db.utils.SyncUtils;
-import org.apache.iotdb.service.sync.thrift.SyncDataStatus;
-import org.apache.iotdb.service.sync.thrift.SyncService;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.transport.TSocket;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * SyncSenderImpl is used to transfer tsfiles that needs to sync to receiver.
- */
-public class SyncSenderImpl implements SyncSender {
-
-  private static final Logger logger = LoggerFactory.getLogger(SyncSenderImpl.class);
-
-  private TTransport transport;
-
-  private SyncService.Client serviceClient;
-
-  private List<String> schema = new ArrayList<>();
-
-  private static SyncSenderConfig config = SyncSenderDescriptor.getInstance().getConfig();
-
-  /**
-   * Files that need to be synchronized
-   */
-  private Map<String, Set<String>> validAllFiles;
-
-  /**
-   * All tsfiles in data directory
-   **/
-  private Map<String, Set<String>> currentLocalFiles;
-
-  /**
-   * If true, sync is in execution.
-   **/
-  private volatile boolean syncStatus = false;
-
-  /**
-   * Key means storage group, Set means corresponding tsfiles
-   **/
-  private Map<String, Set<String>> validFileSnapshot = new HashMap<>();
-
-  private SyncFileManager syncFileManager = SyncFileManager.getInstance();
-
-  private ScheduledExecutorService executorService;
-
-  private SyncSenderImpl() {
-    init();
-  }
-
-  public static final SyncSenderImpl getInstance() {
-    return InstanceHolder.INSTANCE;
-  }
-
-  /**
-   * Create a sender and sync files to the receiver.
-   *
-   * @param args not used
-   */
-  public static void main(String[] args) throws IOException {
-    Thread.currentThread().setName(ThreadName.SYNC_CLIENT.getName());
-    SyncSenderImpl fileSenderImpl = new SyncSenderImpl();
-    fileSenderImpl.verifySingleton();
-    fileSenderImpl.startMonitor();
-    fileSenderImpl.startTimedTask();
-  }
-
-  @Override
-  public void init() {
-    if (executorService == null) {
-      executorService = IoTDBThreadPoolFactory.newScheduledThreadPool(2,
-          "sync-client-timer");
-    }
-  }
-
-  /**
-   * Start Monitor Thread, monitor sync status
-   */
-  private void startMonitor() {
-    executorService.scheduleWithFixedDelay(() -> {
-      if (syncStatus) {
-        logger.info("Sync process is in execution!");
-      }
-    }, Constans.SYNC_MONITOR_DELAY, Constans.SYNC_MONITOR_PERIOD, TimeUnit.SECONDS);
-  }
-
-  /**
-   * Start sync task in a certain time.
-   */
-  private void startTimedTask() {
-    executorService.scheduleWithFixedDelay(() -> {
-      try {
-        sync();
-      } catch (SyncConnectionException | IOException e) {
-        logger.error("Sync failed", e);
-        stop();
-      }
-    }, Constans.SYNC_PROCESS_DELAY, Constans.SYNC_PROCESS_PERIOD, TimeUnit.SECONDS);
-  }
-
-  @Override
-  public void stop() {
-    executorService.shutdownNow();
-    executorService = null;
-  }
-
-  /**
-   * Execute a sync task.
-   */
-  @Override
-  public void sync() throws SyncConnectionException, IOException {
-
-    //1. Clear old snapshots if necessary
-    for (String snapshotPath : config.getSnapshotPaths()) {
-      if (new File(snapshotPath).exists() && new File(snapshotPath).list().length != 0) {
-        // It means that the last task of sync does not succeed! Clear the files and start to sync again
-        FileUtils.deleteDirectory(new File(snapshotPath));
-      }
-    }
-
-    // 2. Acquire valid files and check
-    syncFileManager.init();
-    validAllFiles = syncFileManager.getValidAllFiles();
-    currentLocalFiles = syncFileManager.getCurrentLocalFiles();
-    if (SyncUtils.isEmpty(validAllFiles)) {
-      logger.info("There has no file to sync !");
-      return;
-    }
-
-    // 3. Connect to sync server and Confirm Identity
-    establishConnection(config.getServerIp(), config.getServerPort());
-    if (!confirmIdentity(config.getUuidPath())) {
-      logger.error("Sorry, you do not have the permission to connect to sync receiver.");
-      System.exit(1);
-    }
-
-    // 4. Create snapshot
-    for (Entry<String, Set<String>> entry : validAllFiles.entrySet()) {
-      validFileSnapshot.put(entry.getKey(), makeFileSnapshot(entry.getValue()));
-    }
-
-    syncStatus = true;
-
-    try {
-      // 5. Sync schema
-      syncSchema();
-
-      // 6. Sync data
-      syncAllData();
-    } catch (SyncConnectionException e) {
-      logger.error("cannot finish sync process", e);
-      syncStatus = false;
-      return;
-    }
-
-    // 7. clear snapshot
-    for (String snapshotPath : config.getSnapshotPaths()) {
-      FileUtils.deleteDirectory(new File(snapshotPath));
-    }
-
-    // 8. notify receiver that synchronization finish
-    // At this point the synchronization has finished even if connection fails
-    try {
-      serviceClient.cleanUp();
-    } catch (TException e) {
-      logger.error("Unable to connect to receiver.", e);
-    }
-    transport.close();
-    logger.info("Sync process has finished.");
-    syncStatus = false;
-  }
-
-  @Override
-  public void syncAllData() throws SyncConnectionException {
-    for (Entry<String, Set<String>> entry : validAllFiles.entrySet()) {
-      Set<String> validFiles = entry.getValue();
-      Set<String> validSnapshot = validFileSnapshot.get(entry.getKey());
-      if (validSnapshot.isEmpty()) {
-        continue;
-      }
-      logger.info("Sync process starts to transfer data of storage group {}", entry.getKey());
-      try {
-        if (!serviceClient.init(entry.getKey())) {
-          throw new SyncConnectionException("unable init receiver");
-        }
-      } catch (TException e) {
-        throw new SyncConnectionException("Unable to connect to receiver", e);
-      }
-      syncData(validSnapshot);
-      if (afterSynchronization()) {
-        currentLocalFiles.get(entry.getKey()).addAll(validFiles);
-        syncFileManager.setCurrentLocalFiles(currentLocalFiles);
-        syncFileManager.backupNowLocalFileInfo(config.getLastFileInfo());
-        logger.info("Sync process has finished storage group {}.", entry.getKey());
-      } else {
-        logger.error("Receiver cannot sync data, abandon this synchronization of storage group {}",
-            entry.getKey());
-      }
-    }
-  }
-
-  /**
-   * Establish a connection between the sender and the receiver.
-   *
-   * @param serverIp the ip address of the receiver
-   * @param serverPort must be same with port receiver set.
-   */
-  @Override
-  public void establishConnection(String serverIp, int serverPort) throws SyncConnectionException {
-    transport = new TSocket(serverIp, serverPort);
-    TProtocol protocol;
-    if(IoTDBDescriptor.getInstance().getConfig().isRpcThriftCompressionEnable()) {
-      protocol = new TCompactProtocol(transport);
-    }
-    else {
-      protocol = new TBinaryProtocol(transport);
-    }
-    serviceClient = new SyncService.Client(protocol);
-    try {
-      transport.open();
-    } catch (TTransportException e) {
-      syncStatus = false;
-      logger.error("Cannot connect to server");
-      throw new SyncConnectionException(e);
-    }
-  }
-
-  /**
-   * UUID marks the identity of sender for receiver.
-   */
-  @Override
-  public boolean confirmIdentity(String uuidPath) throws SyncConnectionException, IOException {
-    File file = new File(uuidPath);
-    /** Mark the identity of sender **/
-    String uuid;
-    if (!file.getParentFile().exists()) {
-      file.getParentFile().mkdirs();
-    }
-    if (!file.exists()) {
-      try (FileOutputStream out = new FileOutputStream(file)) {
-        file.createNewFile();
-        uuid = generateUUID();
-        out.write(uuid.getBytes());
-      } catch (IOException e) {
-        logger.error("Cannot insert UUID to file {}", file.getPath());
-        throw new IOException(e);
-      }
-    } else {
-      try (BufferedReader bf = new BufferedReader((new FileReader(uuidPath)))) {
-        uuid = bf.readLine();
-      } catch (IOException e) {
-        logger.error("Cannot read UUID from file{}", file.getPath());
-        throw new IOException(e);
-      }
-    }
-    boolean legalConnection;
-    try {
-      legalConnection = serviceClient.checkIdentity(uuid,
-          InetAddress.getLocalHost().getHostAddress());
-    } catch (Exception e) {
-      logger.error("Cannot confirm identity with receiver");
-      throw new SyncConnectionException(e);
-    }
-    return legalConnection;
-  }
-
-  private String generateUUID() {
-    return Constans.SYNC_CLIENT + UUID.randomUUID().toString().replaceAll("-", "");
-  }
-
-  /**
-   * Create snapshots for valid files.
-   */
-  @Override
-  public Set<String> makeFileSnapshot(Set<String> validFiles) throws IOException {
-    Set<String> validFilesSnapshot = new HashSet<>();
-    try {
-      for (String filePath : validFiles) {
-        String snapshotFilePath = SyncUtils.getSnapshotFilePath(filePath);
-        validFilesSnapshot.add(snapshotFilePath);
-        File newFile = new File(snapshotFilePath);
-        if (!newFile.getParentFile().exists()) {
-          newFile.getParentFile().mkdirs();
-        }
-        Path link = FileSystems.getDefault().getPath(snapshotFilePath);
-        Path target = FileSystems.getDefault().getPath(filePath);
-        Files.createLink(link, target);
-      }
-    } catch (IOException e) {
-      logger.error("Can not make fileSnapshot");
-      throw new IOException(e);
-    }
-    return validFilesSnapshot;
-  }
-
-  /**
-   * Transfer data of a storage group to receiver.
-   *
-   * @param fileSnapshotList list of sending snapshot files in a storage group.
-   */
-  public void syncData(Set<String> fileSnapshotList) throws SyncConnectionException {
-    try {
-      int successNum = 0;
-      for (String snapshotFilePath : fileSnapshotList) {
-        successNum++;
-        File file = new File(snapshotFilePath);
-        List<String> filePathSplit = new ArrayList<>();
-        String os = System.getProperty("os.name");
-        if (os.toLowerCase().startsWith("windows")) {
-          String[] name = snapshotFilePath.split(File.separator + File.separator);
-          filePathSplit.add(name[name.length - 2]);
-          filePathSplit.add(name[name.length - 1]);
-        } else {
-          String[] name = snapshotFilePath.split(File.separator);
-          filePathSplit.add(name[name.length - 2]);
-          filePathSplit.add(name[name.length - 1]);
-        }
-        int retryCount = 0;
-        // Get md5 of the file.
-        MessageDigest md = MessageDigest.getInstance("MD5");
-        outer:
-        while (true) {
-          retryCount++;
-          // Sync all data to receiver
-          if (retryCount > Constans.MAX_SYNC_FILE_TRY) {
-            throw new SyncConnectionException(String
-                .format("can not sync file %s after %s tries.", snapshotFilePath,
-                    Constans.MAX_SYNC_FILE_TRY));
-          }
-          md.reset();
-          byte[] buffer = new byte[Constans.DATA_CHUNK_SIZE];
-          int dataLength;
-          try (FileInputStream fis = new FileInputStream(file);
-              ByteArrayOutputStream bos = new ByteArrayOutputStream(Constans.DATA_CHUNK_SIZE)) {
-            while ((dataLength = fis.read(buffer)) != -1) { // cut the file into pieces to send
-              bos.write(buffer, 0, dataLength);
-              md.update(buffer, 0, dataLength);
-              ByteBuffer buffToSend = ByteBuffer.wrap(bos.toByteArray());
-              bos.reset();
-              if (!Boolean.parseBoolean(serviceClient
-                  .syncData(null, filePathSplit, buffToSend, SyncDataStatus.PROCESSING_STATUS))) {
-                logger.info("Receiver failed to receive data from {}, retry.", snapshotFilePath);
-                continue outer;
-              }
-            }
-          }
-
-          // the file is sent successfully
-          String md5OfSender = (new BigInteger(1, md.digest())).toString(16);
-          String md5OfReceiver = serviceClient.syncData(md5OfSender, filePathSplit,
-              null, SyncDataStatus.FINISH_STATUS);
-          if (md5OfSender.equals(md5OfReceiver)) {
-            logger.info("Receiver has received {} successfully.", snapshotFilePath);
-            break;
-          }
-        }
-        logger.info(String.format("Task of synchronization has completed %d/%d.", successNum,
-            fileSnapshotList.size()));
-      }
-    } catch (Exception e) {
-      throw new SyncConnectionException("Cannot sync data with receiver.", e);
-    }
-  }
-
-  /**
-   * Sync schema with receiver.
-   */
-  @Override
-  public void syncSchema() throws SyncConnectionException {
-    int retryCount = 0;
-    outer:
-    while (true) {
-      retryCount++;
-      if (retryCount > Constans.MAX_SYNC_FILE_TRY) {
-        throw new SyncConnectionException(String
-            .format("can not sync schema after %s tries.", Constans.MAX_SYNC_FILE_TRY));
-      }
-      byte[] buffer = new byte[Constans.DATA_CHUNK_SIZE];
-      try (FileInputStream fis = new FileInputStream(new File(config.getSchemaPath()));
-          ByteArrayOutputStream bos = new ByteArrayOutputStream(Constans.DATA_CHUNK_SIZE)) {
-        // Get md5 of the file.
-        MessageDigest md = MessageDigest.getInstance("MD5");
-        int dataLength;
-        while ((dataLength = fis.read(buffer)) != -1) { // cut the file into pieces to send
-          bos.write(buffer, 0, dataLength);
-          md.update(buffer, 0, dataLength);
-          ByteBuffer buffToSend = ByteBuffer.wrap(bos.toByteArray());
-          bos.reset();
-          // PROCESSING_STATUS represents there is still schema buffer to send.
-          if (!Boolean.parseBoolean(
-              serviceClient.syncSchema(null, buffToSend, SyncDataStatus.PROCESSING_STATUS))) {
-            logger.error("Receiver failed to receive metadata, retry.");
-            continue outer;
-          }
-        }
-        bos.close();
-        String md5OfSender = (new BigInteger(1, md.digest())).toString(16);
-        String md5OfReceiver = serviceClient
-            .syncSchema(md5OfSender, null, SyncDataStatus.FINISH_STATUS);
-        if (md5OfSender.equals(md5OfReceiver)) {
-          logger.info("Receiver has received schema successfully.");
-          /** receiver start to load metadata **/
-          if (Boolean
-              .parseBoolean(serviceClient.syncSchema(null, null, SyncDataStatus.SUCCESS_STATUS))) {
-            throw new SyncConnectionException("Receiver failed to load metadata");
-          }
-          break;
-        }
-      } catch (Exception e) {
-        logger.error("Cannot sync schema ", e);
-        throw new SyncConnectionException(e);
-      }
-    }
-  }
-
-  @Override
-  public boolean afterSynchronization() throws SyncConnectionException {
-    boolean successOrNot;
-    try {
-      successOrNot = serviceClient.load();
-    } catch (TException e) {
-      throw new SyncConnectionException(
-          "Can not finish sync process because sync receiver has broken down.", e);
-    }
-    return successOrNot;
-  }
-
-  /**
-   * The method is to verify whether the client lock file is locked or not, ensuring that only one
-   * client is running.
-   */
-  private void verifySingleton() throws IOException {
-    File lockFile = new File(config.getLockFilePath());
-    if (!lockFile.getParentFile().exists()) {
-      lockFile.getParentFile().mkdirs();
-    }
-    if (!lockFile.exists()) {
-      lockFile.createNewFile();
-    }
-    if (!lockInstance(config.getLockFilePath())) {
-      logger.error("Sync client is running.");
-      System.exit(1);
-    }
-  }
-
-  /**
-   * Try to lock lockfile. if failed, it means that sync client has benn started.
-   *
-   * @param lockFile path of lockfile
-   */
-  private static boolean lockInstance(final String lockFile) {
-    try {
-      final File file = new File(lockFile);
-      final RandomAccessFile randomAccessFile = new RandomAccessFile(file, "rw");
-      final FileLock fileLock = randomAccessFile.getChannel().tryLock();
-      if (fileLock != null) {
-        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
-          try {
-            fileLock.release();
-            randomAccessFile.close();
-            FileUtils.forceDelete(file);
-          } catch (Exception e) {
-            logger.error("Unable to remove lock file: {}", lockFile, e);
-          }
-        }));
-        return true;
-      }
-    } catch (Exception e) {
-      logger.error("Unable to create and/or lock file: {}", lockFile, e);
-    }
-    return false;
-  }
-
-  private static class InstanceHolder {
-
-    private static final SyncSenderImpl INSTANCE = new SyncSenderImpl();
-  }
-
-  public void setConfig(SyncSenderConfig config) {
-    this.config = config;
-  }
-
-  public List<String> getSchema() {
-    return schema;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/ISyncFileManager.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/ISyncFileManager.java
new file mode 100644
index 0000000..803c3d1
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/ISyncFileManager.java
@@ -0,0 +1,67 @@
+/**
+ * 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.sync.sender.manage;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This interface is used to manage deleted files and new closed files that need to be synchronized
+ * in each sync task.
+ */
+public interface ISyncFileManager {
+
+  /**
+   * Find out all closed and unmodified files, which means there has a .resource file and doesn't
+   * have a .mod file and .merge file. For these files, they will eventually generate a new tsfile
+   * file as the merge operation is executed and executed in subsequent synchronization tasks.
+   *
+   * @param dataDir data directory
+   */
+  void getCurrentLocalFiles(String dataDir);
+
+  /**
+   * Load last local files from file<lastLocalFile> which does not contain those tsfiles which are
+   * not synced successfully in previous sync tasks.
+   *
+   * @param lastLocalFile last local file, which may not exist in first sync task.
+   */
+  void getLastLocalFiles(File lastLocalFile) throws IOException;
+
+  /**
+   * Based on current local files and last local files, we can distinguish two kinds of files
+   * between them, one is deleted files, the other is new files. These two kinds of files are valid
+   * files that need to be synchronized to the receiving end.
+   *
+   * @param dataDir data directory
+   */
+  void getValidFiles(String dataDir) throws IOException;
+
+  Map<String, Set<File>> getCurrentSealedLocalFilesMap();
+
+  Map<String, Set<File>> getLastLocalFilesMap();
+
+  Map<String, Set<File>> getDeletedFilesMap();
+
+  Map<String, Set<File>> getToBeSyncedFilesMap();
+
+  Set<String> getAllSGs();
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManager.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManager.java
new file mode 100644
index 0000000..aaedca1d
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManager.java
@@ -0,0 +1,198 @@
+/**
+ * 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.sync.sender.manage;
+
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.engine.merge.task.MergeTask;
+import org.apache.iotdb.db.engine.modification.ModificationFile;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SyncFileManager implements ISyncFileManager {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(SyncFileManager.class);
+
+  /**
+   * All storage groups on the disk where the current sync task is executed
+   */
+  private Set<String> allSGs;
+
+  /**
+   * Key is storage group, value is the set of current sealed tsfiles in the storage group.
+   */
+  private Map<String, Set<File>> currentSealedLocalFilesMap;
+
+  /**
+   * Key is storage group, value is the set of last local tsfiles in the storage group, which doesn't
+   * contains those tsfiles which are not synced successfully.
+   */
+  private Map<String, Set<File>> lastLocalFilesMap;
+
+  /**
+   * Key is storage group, value is the valid set of deleted tsfiles which need to be synced to
+   * receiver end in the storage group.
+   */
+  private Map<String, Set<File>> deletedFilesMap;
+
+  /**
+   * Key is storage group, value is the valid set of new tsfiles which need to be synced to receiver
+   * end in the storage group.
+   */
+  private Map<String, Set<File>> toBeSyncedFilesMap;
+
+  private SyncFileManager() {
+
+  }
+
+  public static final SyncFileManager getInstance() {
+    return SyncFileManagerHolder.INSTANCE;
+  }
+
+  @Override
+  public void getCurrentLocalFiles(String dataDir) {
+    LOGGER.info("Start to get current local files in data folder {}", dataDir);
+
+    currentSealedLocalFilesMap = new HashMap<>();
+    // get all files in data dir sequence folder
+    Map<String, Set<File>> currentAllLocalFiles = new HashMap<>();
+    if (!new File(dataDir + File.separatorChar + IoTDBConstant.SEQUENCE_FLODER_NAME).exists()) {
+      return;
+    }
+    File[] allSGFolders = new File(
+        dataDir + File.separatorChar + IoTDBConstant.SEQUENCE_FLODER_NAME)
+        .listFiles();
+    for (File sgFolder : allSGFolders) {
+      allSGs.add(sgFolder.getName());
+      currentAllLocalFiles.putIfAbsent(sgFolder.getName(), new HashSet<>());
+      File[] files = sgFolder.listFiles();
+      if (files != null) {
+        Arrays.stream(files).forEach(file -> currentAllLocalFiles.get(sgFolder.getName())
+            .add(new File(sgFolder.getAbsolutePath(), file.getName())));
+      }
+    }
+
+    // get sealed tsfiles
+    for (Entry<String, Set<File>> entry : currentAllLocalFiles.entrySet()) {
+      String sgName = entry.getKey();
+      currentSealedLocalFilesMap.putIfAbsent(sgName, new HashSet<>());
+      for (File file : entry.getValue()) {
+        if (!file.getName().endsWith(TSFILE_SUFFIX)) {
+          continue;
+        }
+        if (new File(file.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).exists() && !new File(
+            file.getAbsolutePath() + ModificationFile.FILE_SUFFIX).exists() && !new File(
+            file.getAbsolutePath() + MergeTask.MERGE_SUFFIX).exists()) {
+          currentSealedLocalFilesMap.get(sgName).add(file);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void getLastLocalFiles(File lastLocalFileInfo) throws IOException {
+    LOGGER.info("Start to get last local files from last local file info {}",
+        lastLocalFileInfo.getAbsoluteFile());
+    lastLocalFilesMap = new HashMap<>();
+    if (!lastLocalFileInfo.exists()) {
+      return;
+    }
+    try (BufferedReader reader = new BufferedReader(new FileReader(lastLocalFileInfo))) {
+      String fileName;
+      while ((fileName = reader.readLine()) != null) {
+        String sgName = new File(fileName).getParentFile().getName();
+        allSGs.add(sgName);
+        lastLocalFilesMap.putIfAbsent(sgName, new HashSet<>());
+        lastLocalFilesMap.get(sgName).add(new File(fileName));
+      }
+    }
+  }
+
+  @Override
+  public void getValidFiles(String dataDir) throws IOException {
+    allSGs = new HashSet<>();
+    getCurrentLocalFiles(dataDir);
+    getLastLocalFiles(
+        new File(SyncSenderDescriptor.getInstance().getConfig().getLastFileInfoPath()));
+    toBeSyncedFilesMap = new HashMap<>();
+    deletedFilesMap = new HashMap<>();
+    for (String sgName : allSGs) {
+      toBeSyncedFilesMap.putIfAbsent(sgName, new HashSet<>());
+      deletedFilesMap.putIfAbsent(sgName, new HashSet<>());
+      for (File newFile : currentSealedLocalFilesMap.getOrDefault(sgName, Collections.emptySet())) {
+        if (!lastLocalFilesMap.getOrDefault(sgName, new HashSet<>()).contains(newFile)) {
+          toBeSyncedFilesMap.get(sgName).add(newFile);
+        }
+      }
+      for (File oldFile : lastLocalFilesMap.getOrDefault(sgName, new HashSet<>())) {
+        if (!currentSealedLocalFilesMap.getOrDefault(sgName, new HashSet<>()).contains(oldFile)) {
+          deletedFilesMap.get(sgName).add(oldFile);
+        }
+      }
+    }
+  }
+
+  @Override
+  public Map<String, Set<File>> getCurrentSealedLocalFilesMap() {
+    return currentSealedLocalFilesMap;
+  }
+
+  @Override
+  public Map<String, Set<File>> getLastLocalFilesMap() {
+    return lastLocalFilesMap;
+  }
+
+  @Override
+  public Map<String, Set<File>> getDeletedFilesMap() {
+    return deletedFilesMap;
+  }
+
+  @Override
+  public Map<String, Set<File>> getToBeSyncedFilesMap() {
+    return toBeSyncedFilesMap;
+  }
+
+  @Override
+  public Set<String> getAllSGs() {
+    return allSGs;
+  }
+
+  private static class SyncFileManagerHolder {
+
+    private static final SyncFileManager INSTANCE = new SyncFileManager();
+
+    private SyncFileManagerHolder() {
+
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/ISyncSenderLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/ISyncSenderLogAnalyzer.java
new file mode 100644
index 0000000..084a8d4
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/ISyncSenderLogAnalyzer.java
@@ -0,0 +1,52 @@
+/**
+ * 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.sync.sender.recover;
+
+import java.io.IOException;
+import java.util.Set;
+
+/**
+ * This interface is used to restore and clean up the status of the historical synchronization task
+ * with abnormal termination. Through the analysis of the synchronization task log, the completed
+ * progress is merged to prepare for the next synchronization task.
+ */
+public interface ISyncSenderLogAnalyzer {
+
+  /**
+   * Recover sync tasks that were not completed properly last time, and clean up the environment.
+   */
+  void recover() throws IOException;
+
+  /**
+   * Load last local files from last local info file.
+   */
+  void loadLastLocalFiles(Set<String> lastLocalFiles);
+
+  /**
+   * Load the sync log, which indicates the progress of the last synchronization task.
+   * Deleted files and new tsfiles can be obtained by log analysis.
+   */
+  void loadLogger(Set<String> deletedFiles, Set<String> newFiles);
+
+  /**
+   * Update the last local info file based on the log information of the last task
+   */
+  void updateLastLocalFile(Set<String> currentLocalFiles) throws IOException;
+
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/ISyncSenderLogger.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/ISyncSenderLogger.java
new file mode 100644
index 0000000..c615f26
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/ISyncSenderLogger.java
@@ -0,0 +1,65 @@
+/**
+ * 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.sync.sender.recover;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * This interface is used to log progress in the process of synchronization tasks. If the
+ * synchronization tasks are completed normally and there are no exceptions, the log records will be
+ * deleted; otherwise, the status can be restored according to the log at the start of each task. It
+ * ensures the correctness of synchronization module when system crashed or network abnormality
+ * occurred.
+ */
+public interface ISyncSenderLogger {
+
+  String SYNC_DELETED_FILE_NAME_START = "sync deleted file names start";
+  String SYNC_TSFILE_START = "sync tsfile start";
+
+  /**
+   * Start to sync deleted files name
+   * @throws IOException
+   */
+  void startSyncDeletedFilesName() throws IOException;
+
+  /**
+   * After a deleted file name is synced to the receiver end, record it in sync log.
+   * @param file the deleted tsfile
+   * @throws IOException
+   */
+  void finishSyncDeletedFileName(File file) throws IOException;
+
+  /**
+   * Start to sync new tsfiles
+   * @throws IOException
+   */
+  void startSyncTsFiles() throws IOException;
+
+  /**
+   *
+   * After a new tsfile is synced to the receiver end, record it in sync log.
+   * @param file new tsfile
+   * @throws IOException
+   */
+  void finishSyncTsfile(File file) throws IOException;
+
+  void close() throws IOException;
+
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/SyncSenderLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/SyncSenderLogAnalyzer.java
new file mode 100644
index 0000000..0558580
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/SyncSenderLogAnalyzer.java
@@ -0,0 +1,128 @@
+/**
+ * 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.sync.sender.recover;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SyncSenderLogAnalyzer implements ISyncSenderLogAnalyzer {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(SyncSenderLogAnalyzer.class);
+  private String senderPath;
+  private File currentLocalFile;
+  private File lastLocalFile;
+  private File syncLogFile;
+
+  public SyncSenderLogAnalyzer(String senderPath) {
+    this.senderPath = senderPath;
+    this.currentLocalFile = new File(senderPath, SyncConstant.CURRENT_LOCAL_FILE_NAME);
+    this.lastLocalFile = new File(senderPath, SyncConstant.LAST_LOCAL_FILE_NAME);
+    this.syncLogFile = new File(senderPath, SyncConstant.SYNC_LOG_NAME);
+  }
+
+  @Override
+  public void recover() throws IOException {
+    if (currentLocalFile.exists() && !lastLocalFile.exists()) {
+      FileUtils.moveFile(currentLocalFile, lastLocalFile);
+    } else {
+      Set<String> lastLocalFiles = new HashSet<>();
+      Set<String> deletedFiles = new HashSet<>();
+      Set<String> newFiles = new HashSet<>();
+      loadLastLocalFiles(lastLocalFiles);
+      loadLogger(deletedFiles, newFiles);
+      lastLocalFiles.removeAll(deletedFiles);
+      lastLocalFiles.addAll(newFiles);
+      updateLastLocalFile(lastLocalFiles);
+    }
+    FileUtils.deleteDirectory(new File(senderPath, SyncConstant.DATA_SNAPSHOT_NAME));
+    syncLogFile.delete();
+  }
+
+  @Override
+  public void loadLastLocalFiles(Set<String> lastLocalFiles) {
+    if (!lastLocalFile.exists()) {
+      LOGGER.info("last local file {} doesn't exist.", lastLocalFile.getAbsolutePath());
+      return;
+    }
+    try (BufferedReader br = new BufferedReader(new FileReader(lastLocalFile))) {
+      String line;
+      while ((line = br.readLine()) != null) {
+        lastLocalFiles.add(line);
+      }
+    } catch (IOException e) {
+      LOGGER
+          .error("Can not load last local file list from file {}", lastLocalFile.getAbsoluteFile(),
+              e);
+    }
+  }
+
+  @Override
+  public void loadLogger(Set<String> deletedFiles, Set<String> newFiles) {
+    if (!syncLogFile.exists()) {
+      LOGGER.info("log file {} doesn't exist.", syncLogFile.getAbsolutePath());
+      return;
+    }
+    try (BufferedReader br = new BufferedReader(new FileReader(syncLogFile))) {
+      String line;
+      int mode = 0;
+      while ((line = br.readLine()) != null) {
+        if (line.equals(SyncSenderLogger.SYNC_DELETED_FILE_NAME_START)) {
+          mode = -1;
+        } else if (line.equals(SyncSenderLogger.SYNC_TSFILE_START)) {
+          mode = 1;
+        } else {
+          if (mode == -1) {
+            deletedFiles.add(line);
+          } else if (mode == 1) {
+            newFiles.add(line);
+          }
+        }
+      }
+    } catch (IOException e) {
+      LOGGER
+          .error("Can not load last local file list from file {}", lastLocalFile.getAbsoluteFile(),
+              e);
+    }
+  }
+
+  @Override
+  public void updateLastLocalFile(Set<String> currentLocalFiles) throws IOException {
+    try (BufferedWriter bw = new BufferedWriter(new FileWriter(currentLocalFile))) {
+      for (String line : currentLocalFiles) {
+        bw.write(line);
+        bw.newLine();
+      }
+      bw.flush();
+    } catch (IOException e) {
+      LOGGER.error("Can not clear sync log {}", syncLogFile.getAbsoluteFile(), e);
+    }
+    lastLocalFile.delete();
+    FileUtils.moveFile(currentLocalFile, lastLocalFile);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/SyncSenderLogger.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/SyncSenderLogger.java
new file mode 100644
index 0000000..d7d0e03
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/SyncSenderLogger.java
@@ -0,0 +1,72 @@
+/**
+ * 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.sync.sender.recover;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+
+public class SyncSenderLogger implements ISyncSenderLogger {
+
+  private BufferedWriter bw;
+
+  public SyncSenderLogger(File file) throws IOException {
+    if (!file.getParentFile().exists()) {
+      file.getParentFile().mkdirs();
+    }
+    this.bw = new BufferedWriter(new FileWriter(file.getAbsolutePath()));
+  }
+
+  @Override
+  public void startSyncDeletedFilesName() throws IOException {
+    bw.write(SYNC_DELETED_FILE_NAME_START);
+    bw.newLine();
+    bw.flush();
+  }
+
+  @Override
+  public void finishSyncDeletedFileName(File file) throws IOException {
+    bw.write(file.getAbsolutePath());
+    bw.newLine();
+    bw.flush();
+  }
+
+  @Override
+  public void startSyncTsFiles() throws IOException {
+    bw.write(SYNC_TSFILE_START);
+    bw.newLine();
+    bw.flush();
+  }
+
+  @Override
+  public void finishSyncTsfile(File file) throws IOException {
+    bw.write(file.getAbsolutePath());
+    bw.newLine();
+    bw.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if(bw != null) {
+      bw.close();
+      bw = null;
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/DataTransferManager.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/DataTransferManager.java
new file mode 100644
index 0000000..2354305
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/DataTransferManager.java
@@ -0,0 +1,667 @@
+/**
+ * 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.sync.sender.transfer;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.math.BigInteger;
+import java.net.Socket;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileLock;
+import java.nio.file.FileSystems;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.db.concurrent.ThreadName;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.SyncConnectionException;
+import org.apache.iotdb.db.exception.SyncDeviceOwnerConflictException;
+import org.apache.iotdb.db.metadata.MetadataConstant;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.db.sync.conf.SyncSenderConfig;
+import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
+import org.apache.iotdb.db.sync.sender.manage.ISyncFileManager;
+import org.apache.iotdb.db.sync.sender.manage.SyncFileManager;
+import org.apache.iotdb.db.sync.sender.recover.ISyncSenderLogger;
+import org.apache.iotdb.db.sync.sender.recover.SyncSenderLogAnalyzer;
+import org.apache.iotdb.db.sync.sender.recover.SyncSenderLogger;
+import org.apache.iotdb.db.utils.SyncUtils;
+import org.apache.iotdb.service.sync.thrift.ResultStatus;
+import org.apache.iotdb.service.sync.thrift.SyncService;
+import org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DataTransferManager implements IDataTransferManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(DataTransferManager.class);
+
+  private static SyncSenderConfig config = SyncSenderDescriptor.getInstance().getConfig();
+
+  private static final int BATCH_LINE = 1000;
+
+  /**
+   * When transferring schema information, it is a better choice to transfer only new schema
+   * information, avoiding duplicate data transmission. The schema log is self-increasing, so the
+   * location is recorded once after each synchronization task for the next synchronization task to
+   * use.
+   */
+  private int schemaFileLinePos;
+
+  private TTransport transport;
+
+  private SyncService.Client serviceClient;
+
+  private Set<String> allSG;
+
+  private Map<String, Set<File>> toBeSyncedFilesMap;
+
+  private Map<String, Set<File>> deletedFilesMap;
+
+  private Map<String, Set<File>> lastLocalFilesMap;
+
+  /**
+   * If true, sync is in execution.
+   **/
+  private volatile boolean syncStatus = false;
+
+  /**
+   * Record sync progress in log.
+   */
+  private ISyncSenderLogger syncLog;
+
+  private ISyncFileManager syncFileManager = SyncFileManager.getInstance();
+
+  private ScheduledExecutorService executorService;
+
+  private DataTransferManager() {
+    init();
+  }
+
+  public static DataTransferManager getInstance() {
+    return InstanceHolder.INSTANCE;
+  }
+
+  /**
+   * Create a sender and sync files to the receiver periodically.
+   */
+  public static void main(String[] args) throws IOException {
+    Thread.currentThread().setName(ThreadName.SYNC_CLIENT.getName());
+    IDataTransferManager fileSenderImpl = new DataTransferManager();
+    fileSenderImpl.verifySingleton();
+    fileSenderImpl.startMonitor();
+    fileSenderImpl.startTimedTask();
+  }
+
+  @Override
+  public void verifySingleton() throws IOException {
+    String[] dataDirs = IoTDBDescriptor.getInstance().getConfig().getDataDirs();
+    for (String dataDir : dataDirs) {
+      config.update(dataDir);
+      File lockFile = new File(config.getLockFilePath());
+      if (!lockFile.getParentFile().exists()) {
+        lockFile.getParentFile().mkdirs();
+      }
+      if (!lockFile.exists()) {
+        lockFile.createNewFile();
+      }
+      if (!lockInstance(config.getLockFilePath())) {
+        logger.error("Sync client is already running.");
+        System.exit(1);
+      }
+    }
+  }
+
+  /**
+   * Try to lock lockfile. if failed, it means that sync client has benn started.
+   *
+   * @param lockFile path of lock file
+   */
+  private boolean lockInstance(final String lockFile) {
+    try {
+      final File file = new File(lockFile);
+      final RandomAccessFile randomAccessFile = new RandomAccessFile(file, "rw");
+      final FileLock fileLock = randomAccessFile.getChannel().tryLock();
+      if (fileLock != null) {
+        Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+          try {
+            fileLock.release();
+            randomAccessFile.close();
+          } catch (Exception e) {
+            logger.error("Unable to remove lock file: {}", lockFile, e);
+          }
+        }));
+        return true;
+      }
+    } catch (Exception e) {
+      logger.error("Unable to create and/or lock file: {}", lockFile, e);
+    }
+    return false;
+  }
+
+
+  @Override
+  public void init() {
+    if (executorService == null) {
+      executorService = IoTDBThreadPoolFactory.newScheduledThreadPool(2,
+          "sync-client-timer");
+    }
+  }
+
+  @Override
+  public void startMonitor() {
+    executorService.scheduleWithFixedDelay(() -> {
+      if (syncStatus) {
+        logger.info("Sync process for receiver {} is in execution!", config.getSyncReceiverName());
+      }
+    }, SyncConstant.SYNC_MONITOR_DELAY, SyncConstant.SYNC_MONITOR_PERIOD, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void startTimedTask() {
+    executorService.scheduleWithFixedDelay(() -> {
+      try {
+        syncAll();
+      } catch (SyncConnectionException | IOException | TException e) {
+        logger.error("Sync failed", e);
+        stop();
+      }
+    }, SyncConstant.SYNC_PROCESS_DELAY, SyncConstant.SYNC_PROCESS_PERIOD, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void stop() {
+    executorService.shutdownNow();
+    executorService = null;
+  }
+
+  @Override
+  public void syncAll() throws SyncConnectionException, IOException, TException {
+
+    // 1. Connect to sync receiver and confirm identity
+    establishConnection(config.getServerIp(), config.getServerPort());
+    confirmIdentity();
+    serviceClient.startSync();
+
+    // 2. Sync Schema
+    syncSchema();
+
+    // 3. Sync all data
+    String[] dataDirs = IoTDBDescriptor.getInstance().getConfig().getDataDirs();
+    logger.info("There are {} data dirs to be synced.", dataDirs.length);
+    for (int i = 0 ; i < dataDirs.length; i++) {
+      String dataDir = dataDirs[i];
+      logger.info("Start to sync data in data dir {}, the process is {}/{}", dataDir, i + 1,
+          dataDirs.length);
+
+      config.update(dataDir);
+      syncFileManager.getValidFiles(dataDir);
+      allSG = syncFileManager.getAllSGs();
+      lastLocalFilesMap = syncFileManager.getLastLocalFilesMap();
+      deletedFilesMap = syncFileManager.getDeletedFilesMap();
+      toBeSyncedFilesMap = syncFileManager.getToBeSyncedFilesMap();
+      checkRecovery();
+      if (SyncUtils.isEmpty(deletedFilesMap) && SyncUtils.isEmpty(toBeSyncedFilesMap)) {
+        logger.info("There has no data to sync in data dir {}", dataDir);
+        continue;
+      }
+      sync();
+      endSync();
+      logger.info("Finish to sync data in data dir {}, the process is {}/{}", dataDir, i + 1,
+          dataDirs.length);
+    }
+
+    // 4. notify receiver that synchronization finish
+    // At this point the synchronization has finished even if connection fails
+    try {
+      serviceClient.endSync();
+      transport.close();
+      logger.info("Sync process has finished.");
+    } catch (TException e) {
+      logger.error("Unable to connect to receiver.", e);
+    }
+  }
+
+  private void checkRecovery() throws IOException {
+    new SyncSenderLogAnalyzer(config.getSenderFolderPath()).recover();
+  }
+
+  @Override
+  public void establishConnection(String serverIp, int serverPort) throws SyncConnectionException {
+    transport = new TSocket(serverIp, serverPort);
+    TProtocol protocol = new TBinaryProtocol(transport);
+    serviceClient = new SyncService.Client(protocol);
+    try {
+      transport.open();
+    } catch (TTransportException e) {
+      logger.error("Cannot connect to the receiver.");
+      throw new SyncConnectionException(e);
+    }
+  }
+
+  @Override
+  public void confirmIdentity() throws SyncConnectionException {
+    try (Socket socket = new Socket(config.getServerIp(), config.getServerPort())){
+      ResultStatus status = serviceClient
+          .check(socket.getLocalAddress().getHostAddress(), getOrCreateUUID(config.getUuidPath()));
+      if (!status.success) {
+        throw new SyncConnectionException(
+            "The receiver rejected the synchronization task because " + status.msg);
+      }
+    } catch (Exception e) {
+      logger.error("Cannot confirm identity with the receiver.");
+      throw new SyncConnectionException(e);
+    }
+  }
+
+  /**
+   * UUID marks the identity of sender for receiver.
+   */
+  private String getOrCreateUUID(String uuidPath) throws IOException {
+    File file = new File(uuidPath);
+    String uuid;
+    if (!file.getParentFile().exists()) {
+      file.getParentFile().mkdirs();
+    }
+    if (!file.exists()) {
+      try (FileOutputStream out = new FileOutputStream(file)) {
+        uuid = generateUUID();
+        out.write(uuid.getBytes());
+      } catch (IOException e) {
+        logger.error("Cannot insert UUID to file {}", file.getPath());
+        throw new IOException(e);
+      }
+    } else {
+      try (BufferedReader bf = new BufferedReader((new FileReader(uuidPath)))) {
+        uuid = bf.readLine();
+      } catch (IOException e) {
+        logger.error("Cannot read UUID from file{}", file.getPath());
+        throw new IOException(e);
+      }
+    }
+    return uuid;
+  }
+
+  private String generateUUID() {
+    return UUID.randomUUID().toString().replaceAll("-", "");
+  }
+
+  @Override
+  public void syncSchema() throws SyncConnectionException, TException {
+    if (!getSchemaLogFile().exists()) {
+      logger.info("Schema file {} doesn't exist.", getSchemaLogFile().getName());
+      return;
+    }
+    int retryCount = 0;
+    serviceClient.initSyncData(MetadataConstant.METADATA_LOG);
+    while (true) {
+      if (retryCount > config.getMaxNumOfSyncFileRetry()) {
+        throw new SyncConnectionException(String
+            .format("Can not sync schema after %s retries.", config.getMaxNumOfSyncFileRetry()));
+      }
+      if (tryToSyncSchema()) {
+        writeSyncSchemaPos(getSchemaPosFile());
+        break;
+      }
+      retryCount++;
+    }
+  }
+
+  private boolean tryToSyncSchema() {
+    int schemaPos = readSyncSchemaPos(getSchemaPosFile());
+
+    // start to sync file data and get md5 of this file.
+    try (BufferedReader br = new BufferedReader(new FileReader(getSchemaLogFile()));
+        ByteArrayOutputStream bos = new ByteArrayOutputStream(SyncConstant.DATA_CHUNK_SIZE)) {
+      schemaFileLinePos = 0;
+      while (schemaFileLinePos < schemaPos) {
+        br.readLine();
+        schemaFileLinePos++;
+      }
+      MessageDigest md = MessageDigest.getInstance(SyncConstant.MESSAGE_DIGIT_NAME);
+      String line;
+      int cntLine = 0;
+      while ((line = br.readLine()) != null) {
+        schemaFileLinePos++;
+        byte[] singleLineData = BytesUtils.stringToBytes(line);
+        bos.write(singleLineData);
+        bos.write("\r\n".getBytes());
+        if (cntLine++ == BATCH_LINE) {
+          md.update(bos.toByteArray());
+          ByteBuffer buffToSend = ByteBuffer.wrap(bos.toByteArray());
+          bos.reset();
+          ResultStatus status = serviceClient.syncData(buffToSend);
+          if (!status.success) {
+            logger.error("Receiver failed to receive metadata because {}, retry.", status.msg);
+            return false;
+          }
+          cntLine = 0;
+        }
+      }
+      if (bos.size() != 0) {
+        md.update(bos.toByteArray());
+        ByteBuffer buffToSend = ByteBuffer.wrap(bos.toByteArray());
+        bos.reset();
+        ResultStatus status = serviceClient.syncData(buffToSend);
+        if (!status.success) {
+          logger.error("Receiver failed to receive metadata because {}, retry.", status.msg);
+          return false;
+        }
+      }
+
+      // check md5
+      return checkMD5ForSchema(new BigInteger(1, md.digest()).toString(16));
+    } catch (NoSuchAlgorithmException | IOException | TException e) {
+      logger.error("Can not finish transfer schema to receiver", e);
+      return false;
+    }
+  }
+
+  /**
+   * Check MD5 of schema to make sure that the receiver receives the schema correctly
+   */
+  private boolean checkMD5ForSchema(String md5OfSender) throws TException {
+    ResultStatus status = serviceClient.checkDataMD5(md5OfSender);
+    if (status.success && md5OfSender.equals(status.msg)) {
+      logger.info("Receiver has received schema successfully.");
+      return true;
+    } else {
+      logger
+          .error("MD5 check of schema file {} failed, retry", getSchemaLogFile().getAbsoluteFile());
+      return false;
+    }
+  }
+
+  private int readSyncSchemaPos(File syncSchemaLogFile) {
+    try {
+      if (syncSchemaLogFile.exists()) {
+        try (BufferedReader br = new BufferedReader(new FileReader(syncSchemaLogFile))) {
+          return Integer.parseInt(br.readLine());
+        }
+      }
+    } catch (IOException e) {
+      logger.error("Can not find file {}", syncSchemaLogFile.getAbsoluteFile(), e);
+    }
+    return 0;
+  }
+
+  private void writeSyncSchemaPos(File syncSchemaLogFile) {
+    try {
+      if (!syncSchemaLogFile.exists()) {
+        syncSchemaLogFile.createNewFile();
+      }
+      try (BufferedWriter br = new BufferedWriter(new FileWriter(syncSchemaLogFile))) {
+        br.write(Integer.toString(schemaFileLinePos));
+      }
+    } catch (IOException e) {
+      logger.error("Can not find file {}", syncSchemaLogFile.getAbsoluteFile(), e);
+    }
+  }
+
+  @Override
+  public void sync() throws IOException {
+    try {
+      syncStatus = true;
+
+      List<String> storageGroups = config.getStorageGroupList();
+      for (String sgName : allSG) {
+        if (!storageGroups.isEmpty() && !storageGroups.contains(sgName)) {
+          continue;
+        }
+        lastLocalFilesMap.putIfAbsent(sgName, new HashSet<>());
+        syncLog = new SyncSenderLogger(getSyncLogFile());
+        try {
+          ResultStatus status = serviceClient.init(sgName);
+          if (!status.success) {
+            throw new SyncConnectionException("Unable init receiver because " + status.msg);
+          }
+        } catch (TException | SyncConnectionException e) {
+          throw new SyncConnectionException("Unable to connect to receiver", e);
+        }
+        logger.info("Sync process starts to transfer data of storage group {}", sgName);
+        syncDeletedFilesNameInOneGroup(sgName,
+            deletedFilesMap.getOrDefault(sgName, new HashSet<>()));
+        try {
+          syncDataFilesInOneGroup(sgName, toBeSyncedFilesMap.getOrDefault(sgName, new HashSet<>()));
+        } catch (SyncDeviceOwnerConflictException e) {
+          deletedFilesMap.remove(sgName);
+          toBeSyncedFilesMap.remove(sgName);
+          storageGroups.remove(sgName);
+          config.setStorageGroupList(storageGroups);
+          logger.error("Skip the data files of the storage group {}", sgName, e);
+        }
+      }
+
+    } catch (SyncConnectionException e) {
+      logger.error("cannot finish sync process", e);
+    } finally {
+      if (syncLog != null) {
+        syncLog.close();
+      }
+      syncStatus = false;
+    }
+  }
+
+  @Override
+  public void syncDeletedFilesNameInOneGroup(String sgName, Set<File> deletedFilesName)
+      throws IOException {
+    if (deletedFilesName.isEmpty()) {
+      logger.info("There has no deleted files to be synced in storage group {}", sgName);
+      return;
+    }
+    syncLog.startSyncDeletedFilesName();
+    logger.info("Start to sync names of deleted files in storage group {}", sgName);
+    for (File file : deletedFilesName) {
+      try {
+        if (serviceClient.syncDeletedFileName(file.getName()).success) {
+          logger.info("Receiver has received deleted file name {} successfully.", file.getName());
+          lastLocalFilesMap.get(sgName).remove(file);
+          syncLog.finishSyncDeletedFileName(file);
+        }
+      } catch (TException e) {
+        logger.error("Can not sync deleted file name {}, skip it.", file);
+      }
+    }
+    logger.info("Finish to sync names of deleted files in storage group {}", sgName);
+  }
+
+  @Override
+  public void syncDataFilesInOneGroup(String sgName, Set<File> toBeSyncFiles)
+      throws SyncConnectionException, IOException, SyncDeviceOwnerConflictException {
+    if (toBeSyncFiles.isEmpty()) {
+      logger.info("There has no new tsfiles to be synced in storage group {}", sgName);
+      return;
+    }
+    syncLog.startSyncTsFiles();
+    logger.info("Sync process starts to transfer data of storage group {}", sgName);
+    int cnt = 0;
+    for (File tsfile : toBeSyncFiles) {
+      cnt++;
+      try {
+        File snapshotFile = makeFileSnapshot(tsfile);
+        // firstly sync .resource file, then sync tsfile
+        syncSingleFile(new File(snapshotFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX));
+        syncSingleFile(snapshotFile);
+        lastLocalFilesMap.get(sgName).add(tsfile);
+        syncLog.finishSyncTsfile(tsfile);
+        logger.info("Task of synchronization has completed {}/{}.", cnt, toBeSyncFiles.size());
+      } catch (IOException e) {
+        logger.info(
+            "Tsfile {} can not make snapshot, so skip the tsfile and continue to sync other tsfiles",
+            tsfile, e);
+      }
+    }
+    logger.info("Sync process has finished storage group {}.", sgName);
+  }
+
+  /**
+   * Make snapshot<hard link> for new tsfile and its .restore file.
+   *
+   * @param file new tsfile to be synced
+   */
+  File makeFileSnapshot(File file) throws IOException {
+    File snapshotFile = SyncUtils.getSnapshotFile(file);
+    if (!snapshotFile.getParentFile().exists()) {
+      snapshotFile.getParentFile().mkdirs();
+    }
+    Path link = FileSystems.getDefault().getPath(snapshotFile.getAbsolutePath());
+    Path target = FileSystems.getDefault().getPath(file.getAbsolutePath());
+    Files.createLink(link, target);
+    link = FileSystems.getDefault()
+        .getPath(snapshotFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);
+    target = FileSystems.getDefault()
+        .getPath(file.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);
+    Files.createLink(link, target);
+    return snapshotFile;
+  }
+
+  /**
+   * Transfer data of a tsfile to the receiver.
+   */
+  private void syncSingleFile(File snapshotFile)
+      throws SyncConnectionException, SyncDeviceOwnerConflictException {
+    try {
+      int retryCount = 0;
+      MessageDigest md = MessageDigest.getInstance(SyncConstant.MESSAGE_DIGIT_NAME);
+      serviceClient.initSyncData(snapshotFile.getName());
+      outer:
+      while (true) {
+        retryCount++;
+        if (retryCount > config.getMaxNumOfSyncFileRetry()) {
+          throw new SyncConnectionException(String
+              .format("Can not sync file %s after %s tries.", snapshotFile.getAbsoluteFile(),
+                  config.getMaxNumOfSyncFileRetry()));
+        }
+        md.reset();
+        byte[] buffer = new byte[SyncConstant.DATA_CHUNK_SIZE];
+        int dataLength;
+        try (FileInputStream fis = new FileInputStream(snapshotFile);
+            ByteArrayOutputStream bos = new ByteArrayOutputStream(SyncConstant.DATA_CHUNK_SIZE)) {
+          while ((dataLength = fis.read(buffer)) != -1) { // cut the file into pieces to send
+            bos.write(buffer, 0, dataLength);
+            md.update(buffer, 0, dataLength);
+            ByteBuffer buffToSend = ByteBuffer.wrap(bos.toByteArray());
+            bos.reset();
+            ResultStatus status = serviceClient.syncData(buffToSend);
+            if(status.errorCode == -2){
+              throw new SyncDeviceOwnerConflictException(status.msg);
+            }
+            if (!status.success) {
+              logger.info("Receiver failed to receive data from {} because {}, retry.",
+                  status.msg, snapshotFile.getAbsoluteFile());
+              continue outer;
+            }
+          }
+        }
+
+        // the file is sent successfully
+        String md5OfSender = (new BigInteger(1, md.digest())).toString(16);
+        ResultStatus status = serviceClient.checkDataMD5(md5OfSender);
+        if (status.success && md5OfSender.equals(status.msg)) {
+          logger.info("Receiver has received {} successfully.", snapshotFile.getAbsoluteFile());
+          break;
+        } else {
+          logger.error("MD5 check of tsfile {} failed, retry", snapshotFile.getAbsoluteFile());
+        }
+      }
+    } catch (IOException | TException | NoSuchAlgorithmException e) {
+      throw new SyncConnectionException("Cannot sync data with receiver.", e);
+    }
+  }
+
+  private void endSync() throws IOException {
+    File currentLocalFile = getCurrentLogFile();
+    File lastLocalFile = new File(config.getLastFileInfoPath());
+
+    // 1. Write file list to currentLocalFile
+    try (BufferedWriter bw = new BufferedWriter(new FileWriter(currentLocalFile))) {
+      for (Set<File> currentLocalFiles : lastLocalFilesMap.values()) {
+        for (File file : currentLocalFiles) {
+          bw.write(file.getAbsolutePath());
+          bw.newLine();
+        }
+        bw.flush();
+      }
+    } catch (IOException e) {
+      logger.error("Can not clear sync log {}", lastLocalFile.getAbsoluteFile(), e);
+    }
+
+    // 2. Rename currentLocalFile to lastLocalFile
+    lastLocalFile.delete();
+    FileUtils.moveFile(currentLocalFile, lastLocalFile);
+
+    // 3. delete snapshot directory
+    try {
+      FileUtils.deleteDirectory(new File(config.getSnapshotPath()));
+    } catch (IOException e) {
+      logger.error("Can not clear snapshot directory {}", config.getSnapshotPath(), e);
+    }
+
+    // 4. delete sync log file
+    getSyncLogFile().delete();
+  }
+
+
+  private File getSchemaPosFile() {
+    return new File(config.getSenderFolderPath(), SyncConstant.SCHEMA_POS_FILE_NAME);
+  }
+
+  private File getSchemaLogFile() {
+    return new File(IoTDBDescriptor.getInstance().getConfig().getSchemaDir(),
+        MetadataConstant.METADATA_LOG);
+  }
+
+  private static class InstanceHolder {
+
+    private static final DataTransferManager INSTANCE = new DataTransferManager();
+  }
+
+  private File getSyncLogFile() {
+    return new File(config.getSenderFolderPath(), SyncConstant.SYNC_LOG_NAME);
+  }
+
+  private File getCurrentLogFile() {
+    return new File(config.getSenderFolderPath(), SyncConstant.CURRENT_LOCAL_FILE_NAME);
+  }
+
+  public void setConfig(SyncSenderConfig config) {
+    DataTransferManager.config = config;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/IDataTransferManager.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/IDataTransferManager.java
new file mode 100644
index 0000000..f9d1f46
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/IDataTransferManager.java
@@ -0,0 +1,104 @@
+/**
+ * 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.sync.sender.transfer;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Set;
+import org.apache.iotdb.db.exception.SyncConnectionException;
+import org.apache.iotdb.db.exception.SyncDeviceOwnerConflictException;
+import org.apache.thrift.TException;
+
+/**
+ * This interface is used to realize the data transmission part of synchronization task, and is also
+ * the most important part of synchronization task. By screening out all transmission files to be
+ * synchronized in <class>SyncFileManager</class>, these files are synchronized to the receiving end
+ * to complete the synchronization task.
+ */
+public interface IDataTransferManager {
+
+  void init();
+
+  /**
+   * Verify whether the client lock file is locked or not, ensuring that only one client is
+   * running.
+   */
+  void verifySingleton() throws IOException;
+
+  /**
+   * Start monitor thread, which monitor sync status.
+   */
+  void startMonitor();
+
+  /**
+   * Start sync task in a certain time.
+   */
+  void startTimedTask();
+
+  /**
+   * Establish a connection to receiver end.
+   */
+  void establishConnection(String serverIp, int serverPort) throws SyncConnectionException;
+
+  /**
+   * Confirm identity, the receiver will check whether the sender has synchronization privileges.
+   */
+  void confirmIdentity() throws SyncConnectionException, IOException;
+
+  /**
+   * Sync schema file to receiver before all data to be synced.
+   */
+  void syncSchema() throws SyncConnectionException, TException;
+
+  /**
+   * For deleted files in a storage group, sync them to receiver side and load these files in
+   * receiver.
+   *
+   * @param sgName storage group name
+   * @param deletedFilesName list of deleted file names
+   */
+  void syncDeletedFilesNameInOneGroup(String sgName, Set<File> deletedFilesName)
+      throws SyncConnectionException, IOException;
+
+  /**
+   * Execute a sync task for all data directory.
+   */
+  void syncAll() throws SyncConnectionException, IOException, TException;
+
+  /**
+   * Execute a sync task for a data directory.
+   */
+  void sync() throws SyncConnectionException, IOException;
+
+  /**
+   * For new valid files in a storage group, sync them to receiver side and load these data in
+   * receiver.
+   *
+   * @param sgName storage group name
+   * @param toBeSyncFiles list of new tsfile names
+   */
+  void syncDataFilesInOneGroup(String sgName, Set<File> toBeSyncFiles)
+      throws SyncConnectionException, IOException, SyncDeviceOwnerConflictException;
+
+  /**
+   * Stop sync process
+   */
+  void stop();
+
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/SyncUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/SyncUtils.java
index 0ed8d76..0ec6b24 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/SyncUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/SyncUtils.java
@@ -29,9 +29,6 @@ public class SyncUtils {
 
   private static final String IP_SEPARATOR = "\\.";
 
-  private static String[] snapshotPaths = SyncSenderDescriptor.getInstance()
-      .getConfig().getSnapshotPaths();
-
   private SyncUtils() {
   }
 
@@ -40,41 +37,20 @@ public class SyncUtils {
    * multiple directories, it's necessary to make a snapshot in the same disk. It's used by sync
    * sender.
    */
-  public static String getSnapshotFilePath(String filePath) {
-    String[] name;
-    String relativeFilePath;
-    String os = System.getProperty("os.name");
-    if (os.toLowerCase().startsWith("windows")) {
-      name = filePath.split(File.separator + File.separator);
-      relativeFilePath = name[name.length - 2] + File.separator + name[name.length - 1];
-    } else {
-      name = filePath.split(File.separator);
-      relativeFilePath = name[name.length - 2] + File.separator + name[name.length - 1];
-    }
-    String bufferWritePath = name[0];
-    for (int i = 1; i < name.length - 2; i++) {
-      bufferWritePath = bufferWritePath + File.separatorChar + name[i];
-    }
-    for (String snapshotPath : snapshotPaths) {
-      if (snapshotPath.startsWith(bufferWritePath)) {
-        if (!new File(snapshotPath).exists()) {
-          new File(snapshotPath).mkdir();
-        }
-        if (snapshotPath.length() > 0
-            && snapshotPath.charAt(snapshotPath.length() - 1) != File.separatorChar) {
-          snapshotPath = snapshotPath + File.separatorChar;
-        }
-        return snapshotPath + relativeFilePath;
-      }
+  public static File getSnapshotFile(File file) {
+    String relativeFilePath = file.getParentFile().getName() + File.separator + file.getName();
+    String snapshotDir = SyncSenderDescriptor.getInstance().getConfig().getSnapshotPath();
+    if (!new File(snapshotDir).exists()) {
+      new File(snapshotDir).mkdirs();
     }
-    return null;
+    return new File(snapshotDir, relativeFilePath);
   }
 
   /**
    * Verify sending list is empty or not It's used by sync sender.
    */
-  public static boolean isEmpty(Map<String, Set<String>> sendingFileList) {
-    for (Entry<String, Set<String>> entry : sendingFileList.entrySet()) {
+  public static boolean isEmpty(Map<String, Set<File>> sendingFileList) {
+    for (Entry<String, Set<File>> entry : sendingFileList.entrySet()) {
       if (!entry.getValue().isEmpty()) {
         return false;
       }
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java
index 6fe1c3e..61b54da 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java
@@ -71,7 +71,6 @@ public class MergeLogTest extends MergeTest {
       String line;
       while ((line = bufferedReader.readLine()) != null) {
         lineCnt ++;
-        System.out.println(line);
       }
     } catch (IOException e) {
       e.printStackTrace();
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java
new file mode 100644
index 0000000..0dfcd51
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java
@@ -0,0 +1,151 @@
+/*
+ * 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.merge;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.engine.merge.manage.MergeResource;
+import org.apache.iotdb.db.engine.merge.task.MergeTask;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.MetadataErrorException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.resourceRelated.SeqResourceIterateReader;
+import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.write.TsFileWriter;
+import org.apache.iotdb.tsfile.write.record.TSRecord;
+import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class MergeOverLapTest extends MergeTest {
+
+  private File tempSGDir;
+
+  @Before
+  public void setUp()
+      throws IOException, WriteProcessException, MetadataErrorException, PathErrorException {
+    ptNum = 1000;
+    super.setUp();
+    tempSGDir = new File("tempSG");
+    tempSGDir.mkdirs();
+  }
+
+  @After
+  public void tearDown() throws IOException, StorageEngineException {
+    super.tearDown();
+    FileUtils.deleteDirectory(tempSGDir);
+  }
+
+  void prepareFiles(int seqFileNum, int unseqFileNum) throws IOException, WriteProcessException {
+    for (int i = 0; i < seqFileNum; i++) {
+      File file = SystemFileFactory.INSTANCE.getFile(i + "seq.tsfile");
+      TsFileResource tsFileResource = new TsFileResource(file);
+      seqResources.add(tsFileResource);
+      prepareFile(tsFileResource, i * ptNum, ptNum, 0);
+    }
+    for (int i = 0; i < unseqFileNum; i++) {
+      File file = SystemFileFactory.INSTANCE.getFile(i + "unseq.tsfile");
+      TsFileResource tsFileResource = new TsFileResource(file);
+      unseqResources.add(tsFileResource);
+      prepareFile(tsFileResource, i * ptNum, ptNum * (i + 1) / unseqFileNum, 10000);
+    }
+    File file = SystemFileFactory.INSTANCE.getFile(unseqFileNum + "unseq.tsfile");
+    TsFileResource tsFileResource = new TsFileResource(file);
+    unseqResources.add(tsFileResource);
+    prepareUnseqFile(tsFileResource, 0, ptNum * unseqFileNum, 20000);
+  }
+
+  void prepareUnseqFile(TsFileResource tsFileResource, long timeOffset, long ptNum,
+      long valueOffset)
+      throws IOException, WriteProcessException {
+    TsFileWriter fileWriter = new TsFileWriter(tsFileResource.getFile());
+    for (MeasurementSchema measurementSchema : measurementSchemas) {
+      fileWriter.addMeasurement(measurementSchema);
+    }
+    for (long i = timeOffset; i < timeOffset + ptNum; i++) {
+      for (int j = 0; j < deviceNum; j++) {
+        TSRecord record = new TSRecord(i, deviceIds[j]);
+        for (int k = 0; k < measurementNum; k++) {
+          record.addTuple(DataPoint.getDataPoint(measurementSchemas[k].getType(),
+              measurementSchemas[k].getMeasurementId(), String.valueOf(i + valueOffset)));
+        }
+        fileWriter.write(record);
+        tsFileResource.updateStartTime(deviceIds[j], i);
+        tsFileResource.updateEndTime(deviceIds[j], i);
+      }
+      // insert overlapping tuples
+      if ((i + 1) % 100 == 0) {
+        for (int j = 0; j < deviceNum; j++) {
+          TSRecord record = new TSRecord(i, deviceIds[j]);
+          for (int k = 0; k < measurementNum; k++) {
+            record.addTuple(DataPoint.getDataPoint(measurementSchemas[k].getType(),
+                measurementSchemas[k].getMeasurementId(), String.valueOf(i + valueOffset)));
+          }
+          fileWriter.write(record);
+          tsFileResource.updateStartTime(deviceIds[j], i);
+          tsFileResource.updateEndTime(deviceIds[j], i);
+        }
+      }
+      if ((i + 1) % flushInterval == 0) {
+        fileWriter.flushForTest();
+      }
+    }
+    fileWriter.close();
+  }
+
+  @Test
+  public void testFullMerge() throws Exception {
+    MergeTask mergeTask =
+        new MergeTask(new MergeResource(seqResources, unseqResources), tempSGDir.getPath(), (k, v, l) -> {}, "test",
+            true, 1, MERGE_TEST_SG);
+    mergeTask.call();
+
+    QueryContext context = new QueryContext();
+    Path path = new Path(deviceIds[0], measurementSchemas[0].getMeasurementId());
+    SeqResourceIterateReader tsFilesReader = new SeqResourceIterateReader(path,
+        Collections.singletonList(seqResources.get(0)),
+        null, context);
+    int cnt = 0;
+    try {
+      while (tsFilesReader.hasNext()) {
+        BatchData batchData = tsFilesReader.nextBatch();
+        for (int i = 0; i < batchData.length(); i++) {
+          cnt ++;
+          assertEquals(batchData.getTimeByIndex(i) + 20000.0, batchData.getDoubleByIndex(i), 0.001);
+        }
+      }
+      assertEquals(1000, cnt);
+    } finally {
+      tsFilesReader.close();
+    }
+  }
+}
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java
index 007060d..b9e809e 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java
@@ -26,6 +26,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.cache.DeviceMetaDataCache;
 import org.apache.iotdb.db.engine.cache.TsFileMetaDataCache;
@@ -41,7 +42,6 @@ import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
 import org.apache.iotdb.tsfile.write.TsFileWriter;
 import org.apache.iotdb.tsfile.write.record.TSRecord;
 import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
@@ -114,20 +114,29 @@ abstract class MergeTest {
     }
   }
 
-  private void prepareFiles(int seqFileNum, int unseqFileNum) throws IOException, WriteProcessException {
+  private void prepareFiles(int seqFileNum, int unseqFileNum)
+      throws IOException, WriteProcessException {
     for (int i = 0; i < seqFileNum; i++) {
-      File file = SystemFileFactory.INSTANCE.getFile(i + "seq.tsfile");
+      File file = new File(
+          i + "seq" + IoTDBConstant.TSFILE_NAME_SEPARATOR + i + IoTDBConstant.TSFILE_NAME_SEPARATOR
+              + i
+              + ".tsfile");
       TsFileResource tsFileResource = new TsFileResource(file);
       seqResources.add(tsFileResource);
       prepareFile(tsFileResource, i * ptNum, ptNum, 0);
     }
     for (int i = 0; i < unseqFileNum; i++) {
-      File file = SystemFileFactory.INSTANCE.getFile(i + "unseq.tsfile");
+      File file = new File(
+          i + "unseq" + IoTDBConstant.TSFILE_NAME_SEPARATOR
+              + i + IoTDBConstant.TSFILE_NAME_SEPARATOR
+              + i
+              + ".tsfile");
       TsFileResource tsFileResource = new TsFileResource(file);
       unseqResources.add(tsFileResource);
       prepareFile(tsFileResource, i * ptNum, ptNum * (i + 1) / unseqFileNum, 10000);
     }
-    File file = SystemFileFactory.INSTANCE.getFile(unseqFileNum + "unseq.tsfile");
+    File file = new File(unseqFileNum + "unseq" + IoTDBConstant.TSFILE_NAME_SEPARATOR + unseqFileNum
+        + IoTDBConstant.TSFILE_NAME_SEPARATOR + unseqFileNum + ".tsfile");
     TsFileResource tsFileResource = new TsFileResource(file);
     unseqResources.add(tsFileResource);
     prepareFile(tsFileResource, 0, ptNum * unseqFileNum, 20000);
@@ -144,7 +153,7 @@ abstract class MergeTest {
     FileReaderManager.getInstance().stop();
   }
 
-  private void prepareFile(TsFileResource tsFileResource, long timeOffset, long ptNum,
+  void prepareFile(TsFileResource tsFileResource, long timeOffset, long ptNum,
       long valueOffset)
       throws IOException, WriteProcessException {
     TsFileWriter fileWriter = new TsFileWriter(tsFileResource.getFile());
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java
index ad76ef2..093eb6a 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java
@@ -28,7 +28,7 @@ import org.apache.iotdb.db.exception.PathErrorException;
 import org.apache.iotdb.db.exception.StorageGroupException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.sync.test.RandomNum;
+import org.apache.iotdb.db.utils.RandomNum;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderTest.java b/server/src/test/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderTest.java
new file mode 100644
index 0000000..5803642
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderTest.java
@@ -0,0 +1,330 @@
+/**
+ * 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.sync.receiver.load;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.db.exception.MetadataErrorException;
+import org.apache.iotdb.db.exception.StartupException;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileLoaderTest {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FileLoaderTest.class);
+  private static final String SG_NAME = "root.sg";
+  private static IoTDB daemon;
+  private String dataDir;
+  private IFileLoader fileLoader;
+
+  @Before
+  public void setUp()
+      throws IOException, InterruptedException, StartupException, DiskSpaceInsufficientException, MetadataErrorException {
+    EnvironmentUtils.closeStatMonitor();
+    daemon = IoTDB.getInstance();
+    daemon.active();
+    EnvironmentUtils.envSetUp();
+    dataDir = new File(DirectoryManager.getInstance().getNextFolderForSequenceFile())
+        .getParentFile().getAbsolutePath();
+    initMetadata();
+  }
+
+  private void initMetadata() throws MetadataErrorException {
+    MManager mmanager = MManager.getInstance();
+    mmanager.init();
+    mmanager.clear();
+    mmanager.setStorageGroupToMTree("root.sg0");
+    mmanager.setStorageGroupToMTree("root.sg1");
+    mmanager.setStorageGroupToMTree("root.sg2");
+  }
+
+  @After
+  public void tearDown() throws InterruptedException, IOException, StorageEngineException {
+    daemon.stop();
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void loadNewTsfiles() throws IOException, StorageEngineException {
+    fileLoader = FileLoader.createFileLoader(getReceiverFolderFile());
+    Map<String, List<File>> allFileList = new HashMap<>();
+    Map<String, Set<String>> correctSequenceLoadedFileMap = new HashMap<>();
+
+    // add some new tsfiles
+    Random r = new Random(0);
+    long time = System.currentTimeMillis();
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 10; j++) {
+        allFileList.putIfAbsent(SG_NAME + i, new ArrayList<>());
+        correctSequenceLoadedFileMap.putIfAbsent(SG_NAME + i, new HashSet<>());
+        String rand = String.valueOf(r.nextInt(10000));
+        String fileName =
+            getSnapshotFolder() + File.separator + SG_NAME + i + File.separator + (time + i * 100
+                + j) + IoTDBConstant.TSFILE_NAME_SEPARATOR + rand
+                + IoTDBConstant.TSFILE_NAME_SEPARATOR + "0.tsfile";
+        File syncFile = new File(fileName);
+        File dataFile = new File(
+            syncFile.getParentFile().getParentFile().getParentFile().getParentFile()
+                .getParentFile(), IoTDBConstant.SEQUENCE_FLODER_NAME
+            + File.separatorChar + syncFile.getParentFile().getName() + File.separatorChar
+            + syncFile.getName());
+        correctSequenceLoadedFileMap.get(SG_NAME + i).add(dataFile.getAbsolutePath());
+        allFileList.get(SG_NAME + i).add(syncFile);
+        if (!syncFile.getParentFile().exists()) {
+          syncFile.getParentFile().mkdirs();
+        }
+        if (!syncFile.exists() && !syncFile.createNewFile()) {
+          LOGGER.error("Can not create new file {}", syncFile.getPath());
+        }
+        if (!new File(syncFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).exists()
+            && !new File(syncFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX)
+            .createNewFile()) {
+          LOGGER.error("Can not create new file {}", syncFile.getPath());
+        }
+        TsFileResource tsFileResource = new TsFileResource(syncFile);
+        tsFileResource.getStartTimeMap().put(String.valueOf(i), (long) j * 10);
+        tsFileResource.getEndTimeMap().put(String.valueOf(i), (long) j * 10 + 5);
+        tsFileResource.serialize();
+      }
+    }
+
+    for (int i = 0; i < 3; i++) {
+      StorageGroupProcessor processor = StorageEngine.getInstance().getProcessor(SG_NAME + i);
+      assertTrue(processor.getSequenceFileList().isEmpty());
+      assertTrue(processor.getUnSequenceFileList().isEmpty());
+    }
+
+    assertTrue(getReceiverFolderFile().exists());
+    for (List<File> set : allFileList.values()) {
+      for (File newTsFile : set) {
+        if (!newTsFile.getName().endsWith(TsFileResource.RESOURCE_SUFFIX)) {
+          fileLoader.addTsfile(newTsFile);
+        }
+      }
+    }
+    fileLoader.endSync();
+
+    try {
+      long waitTime = 0;
+      while (FileLoaderManager.getInstance()
+          .containsFileLoader(getReceiverFolderFile().getName())) {
+        Thread.sleep(100);
+        waitTime += 100;
+        LOGGER.info("Has waited for loading new tsfiles {}ms", waitTime);
+      }
+    } catch (InterruptedException e) {
+      LOGGER.error("Fail to wait for loading new tsfiles", e);
+    }
+
+    assertFalse(new File(getReceiverFolderFile(), SyncConstant.RECEIVER_DATA_FOLDER_NAME).exists());
+    Map<String, Set<String>> sequenceLoadedFileMap = new HashMap<>();
+    for (int i = 0; i < 3; i++) {
+      StorageGroupProcessor processor = StorageEngine.getInstance().getProcessor(SG_NAME + i);
+      sequenceLoadedFileMap.putIfAbsent(SG_NAME + i, new HashSet<>());
+      assertEquals(10, processor.getSequenceFileList().size());
+      for (TsFileResource tsFileResource : processor.getSequenceFileList()) {
+        sequenceLoadedFileMap.get(SG_NAME + i).add(tsFileResource.getFile().getAbsolutePath());
+      }
+      assertTrue(processor.getUnSequenceFileList().isEmpty());
+    }
+
+    assertEquals(sequenceLoadedFileMap.size(), correctSequenceLoadedFileMap.size());
+    for (Entry<String, Set<String>> entry : correctSequenceLoadedFileMap.entrySet()) {
+      String sg = entry.getKey();
+      assertEquals(entry.getValue().size(), sequenceLoadedFileMap.get(sg).size());
+      assertTrue(entry.getValue().containsAll(sequenceLoadedFileMap.get(sg)));
+    }
+  }
+
+  @Test
+  public void loadDeletedFileName()
+      throws IOException, StorageEngineException, InterruptedException {
+    fileLoader = FileLoader.createFileLoader(getReceiverFolderFile());
+    Map<String, List<File>> allFileList = new HashMap<>();
+    Map<String, Set<String>> correctLoadedFileMap = new HashMap<>();
+
+    // add some tsfiles
+    Random r = new Random(0);
+    long time = System.currentTimeMillis();
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 25; j++) {
+        allFileList.putIfAbsent(SG_NAME + i, new ArrayList<>());
+        correctLoadedFileMap.putIfAbsent(SG_NAME + i, new HashSet<>());
+        String rand = String.valueOf(r.nextInt(10000));
+        String fileName =
+            getSnapshotFolder() + File.separator + SG_NAME + i + File.separator + (time + i * 100
+                + j) + IoTDBConstant.TSFILE_NAME_SEPARATOR + rand
+                + IoTDBConstant.TSFILE_NAME_SEPARATOR + "0.tsfile";
+        File syncFile = new File(fileName);
+        File dataFile = new File(
+            DirectoryManager.getInstance().getNextFolderForSequenceFile(),
+            syncFile.getParentFile().getName() + File.separatorChar + syncFile.getName());
+        correctLoadedFileMap.get(SG_NAME + i).add(dataFile.getAbsolutePath());
+        allFileList.get(SG_NAME + i).add(syncFile);
+        if (!syncFile.getParentFile().exists()) {
+          syncFile.getParentFile().mkdirs();
+        }
+        if (!syncFile.exists() && !syncFile.createNewFile()) {
+          LOGGER.error("Can not create new file {}", syncFile.getPath());
+        }
+        if (!new File(syncFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).exists()
+            && !new File(syncFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX)
+            .createNewFile()) {
+          LOGGER.error("Can not create new file {}", syncFile.getPath());
+        }
+        TsFileResource tsFileResource = new TsFileResource(syncFile);
+        tsFileResource.serialize();
+      }
+    }
+
+    for (int i = 0; i < 3; i++) {
+      StorageGroupProcessor processor = StorageEngine.getInstance().getProcessor(SG_NAME + i);
+      assertTrue(processor.getSequenceFileList().isEmpty());
+      assertTrue(processor.getUnSequenceFileList().isEmpty());
+    }
+
+    assertTrue(getReceiverFolderFile().exists());
+    for (List<File> set : allFileList.values()) {
+      for (File newTsFile : set) {
+        if (!newTsFile.getName().endsWith(TsFileResource.RESOURCE_SUFFIX)) {
+          fileLoader.addTsfile(newTsFile);
+        }
+      }
+    }
+    fileLoader.endSync();
+
+    try {
+      long waitTime = 0;
+      while (FileLoaderManager.getInstance()
+          .containsFileLoader(getReceiverFolderFile().getName())) {
+        Thread.sleep(100);
+        waitTime += 100;
+        LOGGER.info("Has waited for loading new tsfiles {}ms", waitTime);
+      }
+    } catch (InterruptedException e) {
+      LOGGER.error("Fail to wait for loading new tsfiles", e);
+    }
+
+    assertFalse(new File(getReceiverFolderFile(), SyncConstant.RECEIVER_DATA_FOLDER_NAME).exists());
+    Map<String, Set<String>> loadedFileMap = new HashMap<>();
+    for (int i = 0; i < 3; i++) {
+      StorageGroupProcessor processor = StorageEngine.getInstance().getProcessor(SG_NAME + i);
+      loadedFileMap.putIfAbsent(SG_NAME + i, new HashSet<>());
+      assertEquals(25, processor.getSequenceFileList().size());
+      for (TsFileResource tsFileResource : processor.getSequenceFileList()) {
+        loadedFileMap.get(SG_NAME + i).add(tsFileResource.getFile().getAbsolutePath());
+      }
+      assertTrue(processor.getUnSequenceFileList().isEmpty());
+    }
+
+    assertEquals(loadedFileMap.size(), correctLoadedFileMap.size());
+    for (Entry<String, Set<String>> entry : correctLoadedFileMap.entrySet()) {
+      String sg = entry.getKey();
+      assertEquals(entry.getValue().size(), loadedFileMap.get(sg).size());
+      assertTrue(entry.getValue().containsAll(loadedFileMap.get(sg)));
+    }
+
+    // delete some tsfiles
+    fileLoader = FileLoader.createFileLoader(getReceiverFolderFile());
+    for (Entry<String, List<File>> entry : allFileList.entrySet()) {
+      String sg = entry.getKey();
+      List<File> files = entry.getValue();
+      int cnt = 0;
+      for (File snapFile : files) {
+        if (!snapFile.getName().endsWith(TsFileResource.RESOURCE_SUFFIX)) {
+          File dataFile = new File(
+              DirectoryManager.getInstance().getNextFolderForSequenceFile(),
+              snapFile.getParentFile().getName() + File.separatorChar + snapFile.getName());
+          correctLoadedFileMap.get(sg).remove(dataFile.getAbsolutePath());
+          snapFile.delete();
+          fileLoader.addDeletedFileName(snapFile);
+          new File(snapFile + TsFileResource.RESOURCE_SUFFIX).delete();
+          if (++cnt == 15) {
+            break;
+          }
+        }
+      }
+    }
+    fileLoader.endSync();
+
+    try {
+      long waitTime = 0;
+      while (FileLoaderManager.getInstance()
+          .containsFileLoader(getReceiverFolderFile().getName())) {
+        Thread.sleep(100);
+        waitTime += 100;
+        LOGGER.info("Has waited for loading new tsfiles {}ms", waitTime);
+      }
+    } catch (InterruptedException e) {
+      LOGGER.error("Fail to wait for loading new tsfiles", e);
+    }
+
+    loadedFileMap.clear();
+    for (int i = 0; i < 3; i++) {
+      StorageGroupProcessor processor = StorageEngine.getInstance().getProcessor(SG_NAME + i);
+      loadedFileMap.putIfAbsent(SG_NAME + i, new HashSet<>());
+      for (TsFileResource tsFileResource : processor.getSequenceFileList()) {
+        loadedFileMap.get(SG_NAME + i).add(tsFileResource.getFile().getAbsolutePath());
+      }
+      assertTrue(processor.getUnSequenceFileList().isEmpty());
+    }
+
+    assertEquals(loadedFileMap.size(), correctLoadedFileMap.size());
+    for (Entry<String, Set<String>> entry : correctLoadedFileMap.entrySet()) {
+      String sg = entry.getKey();
+      assertEquals(entry.getValue().size(), loadedFileMap.get(sg).size());
+      assertTrue(entry.getValue().containsAll(loadedFileMap.get(sg)));
+    }
+  }
+
+  private File getReceiverFolderFile() {
+    return new File(dataDir + File.separatorChar + SyncConstant.SYNC_RECEIVER + File.separatorChar
+        + "127.0.0.1_5555");
+  }
+
+  private File getSnapshotFolder() {
+    return new File(getReceiverFolderFile(), SyncConstant.RECEIVER_DATA_FOLDER_NAME);
+  }
+}
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogAnalyzerTest.java b/server/src/test/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogAnalyzerTest.java
new file mode 100644
index 0000000..8369c36
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogAnalyzerTest.java
@@ -0,0 +1,212 @@
+/**
+ * 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.sync.receiver.recover;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.db.exception.MetadataErrorException;
+import org.apache.iotdb.db.exception.StartupException;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.sync.receiver.load.FileLoader;
+import org.apache.iotdb.db.sync.receiver.load.FileLoaderManager;
+import org.apache.iotdb.db.sync.receiver.load.FileLoaderTest;
+import org.apache.iotdb.db.sync.receiver.load.IFileLoader;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SyncReceiverLogAnalyzerTest {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(FileLoaderTest.class);
+  private static final String SG_NAME = "root.sg";
+  private static IoTDB daemon;
+  private String dataDir;
+  private IFileLoader fileLoader;
+  private ISyncReceiverLogAnalyzer logAnalyze;
+  private ISyncReceiverLogger receiverLogger;
+
+  @Before
+  public void setUp()
+      throws IOException, InterruptedException, StartupException, DiskSpaceInsufficientException, MetadataErrorException {
+    EnvironmentUtils.closeStatMonitor();
+    daemon = IoTDB.getInstance();
+    daemon.active();
+    EnvironmentUtils.envSetUp();
+    dataDir = new File(DirectoryManager.getInstance().getNextFolderForSequenceFile())
+        .getParentFile().getAbsolutePath();
+    logAnalyze = SyncReceiverLogAnalyzer.getInstance();
+    initMetadata();
+  }
+
+  private void initMetadata() throws MetadataErrorException {
+    MManager mmanager = MManager.getInstance();
+    mmanager.init();
+    mmanager.clear();
+    mmanager.setStorageGroupToMTree("root.sg0");
+    mmanager.setStorageGroupToMTree("root.sg1");
+    mmanager.setStorageGroupToMTree("root.sg2");
+  }
+
+  @After
+  public void tearDown() throws InterruptedException, IOException, StorageEngineException {
+    daemon.stop();
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void recover() throws IOException, StorageEngineException, InterruptedException {
+    receiverLogger = new SyncReceiverLogger(
+        new File(getReceiverFolderFile(), SyncConstant.SYNC_LOG_NAME));
+    fileLoader = FileLoader.createFileLoader(getReceiverFolderFile());
+    Map<String, Set<File>> allFileList = new HashMap<>();
+    Map<String, Set<File>> correctSequenceLoadedFileMap = new HashMap<>();
+
+    // add some new tsfiles
+    Random r = new Random(0);
+    receiverLogger.startSyncTsFiles();
+    Set<String> toBeSyncedFiles = new HashSet<>();
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 10; j++) {
+        allFileList.putIfAbsent(SG_NAME + i, new HashSet<>());
+        correctSequenceLoadedFileMap.putIfAbsent(SG_NAME + i, new HashSet<>());
+        String rand = String.valueOf(r.nextInt(10000) + i * j);
+        String fileName =
+            getSnapshotFolder() + File.separator + SG_NAME + i + File.separator + System
+                .currentTimeMillis() + IoTDBConstant.TSFILE_NAME_SEPARATOR + rand
+                + IoTDBConstant.TSFILE_NAME_SEPARATOR + "0.tsfile";
+        Thread.sleep(1);
+        File syncFile = new File(fileName);
+        receiverLogger
+            .finishSyncTsfile(syncFile);
+        toBeSyncedFiles.add(syncFile.getAbsolutePath());
+        File dataFile = new File(
+            DirectoryManager.getInstance().getNextFolderForSequenceFile(),
+            syncFile.getParentFile().getName() + File.separatorChar
+                + syncFile.getName());
+        correctSequenceLoadedFileMap.get(SG_NAME + i).add(dataFile);
+        allFileList.get(SG_NAME + i).add(syncFile);
+        if (!syncFile.getParentFile().exists()) {
+          syncFile.getParentFile().mkdirs();
+        }
+        if (!syncFile.exists() && !syncFile.createNewFile()) {
+          LOGGER.error("Can not create new file {}", syncFile.getPath());
+        }
+        if (!new File(syncFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).exists()
+            && !new File(syncFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX)
+            .createNewFile()) {
+          LOGGER.error("Can not create new file {}", syncFile.getPath());
+        }
+        TsFileResource tsFileResource = new TsFileResource(syncFile);
+        tsFileResource.getStartTimeMap().put(String.valueOf(i), (long) j * 10);
+        tsFileResource.getEndTimeMap().put(String.valueOf(i), (long) j * 10 + 5);
+        tsFileResource.serialize();
+      }
+    }
+
+    for (int i = 0; i < 3; i++) {
+      StorageGroupProcessor processor = StorageEngine.getInstance().getProcessor(SG_NAME + i);
+      assertTrue(processor.getSequenceFileList().isEmpty());
+      assertTrue(processor.getUnSequenceFileList().isEmpty());
+    }
+
+    assertTrue(getReceiverFolderFile().exists());
+    for (Set<File> set : allFileList.values()) {
+      for (File newTsFile : set) {
+        if (!newTsFile.getName().endsWith(TsFileResource.RESOURCE_SUFFIX)) {
+          fileLoader.addTsfile(newTsFile);
+        }
+      }
+    }
+
+    receiverLogger.close();
+    assertTrue(new File(getReceiverFolderFile(), SyncConstant.LOAD_LOG_NAME).exists());
+    assertTrue(new File(getReceiverFolderFile(), SyncConstant.SYNC_LOG_NAME).exists());
+    assertTrue(
+        FileLoaderManager.getInstance().containsFileLoader(getReceiverFolderFile().getName()));
+    int mode = 0;
+    Set<String> toBeSyncedFilesTest = new HashSet<>();
+    try (BufferedReader br = new BufferedReader(
+        new FileReader(new File(getReceiverFolderFile(), SyncConstant.SYNC_LOG_NAME)))) {
+      String line;
+      while ((line = br.readLine()) != null) {
+        if (line.equals(SyncReceiverLogger.SYNC_DELETED_FILE_NAME_START)) {
+          mode = -1;
+        } else if (line.equals(SyncReceiverLogger.SYNC_TSFILE_START)) {
+          mode = 1;
+        } else {
+          if (mode == 1) {
+            toBeSyncedFilesTest.add(line);
+          }
+        }
+      }
+    }
+    assertEquals(toBeSyncedFilesTest.size(), toBeSyncedFiles.size());
+    assertTrue(toBeSyncedFilesTest.containsAll(toBeSyncedFiles));
+
+    logAnalyze.recover(getReceiverFolderFile().getName());
+
+    try {
+      long waitTime = 0;
+      while (FileLoaderManager.getInstance()
+          .containsFileLoader(getReceiverFolderFile().getName())) {
+        Thread.sleep(100);
+        waitTime += 100;
+        LOGGER.info("Has waited for loading new tsfiles {}ms", waitTime);
+      }
+    } catch (InterruptedException e) {
+      LOGGER.error("Fail to wait for loading new tsfiles", e);
+    }
+
+    assertFalse(new File(getReceiverFolderFile(), SyncConstant.LOAD_LOG_NAME).exists());
+    assertFalse(new File(getReceiverFolderFile(), SyncConstant.SYNC_LOG_NAME).exists());
+  }
+
+
+  private File getReceiverFolderFile() {
+    return new File(dataDir + File.separatorChar + SyncConstant.SYNC_RECEIVER + File.separatorChar
+        + "127.0.0.1_5555");
+  }
+
+  private File getSnapshotFolder() {
+    return new File(getReceiverFolderFile(), SyncConstant.RECEIVER_DATA_FOLDER_NAME);
+  }
+}
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLoggerTest.java b/server/src/test/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLoggerTest.java
new file mode 100644
index 0000000..b5bf967
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLoggerTest.java
@@ -0,0 +1,112 @@
+/**
+ * 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.sync.receiver.recover;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.db.exception.StartupException;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SyncReceiverLoggerTest {
+
+  private ISyncReceiverLogger receiverLogger;
+  private String dataDir;
+
+  @Before
+  public void setUp()
+      throws IOException, InterruptedException, StartupException, DiskSpaceInsufficientException {
+    EnvironmentUtils.envSetUp();
+    dataDir = new File(DirectoryManager.getInstance().getNextFolderForSequenceFile())
+        .getParentFile().getAbsolutePath();
+  }
+
+  @After
+  public void tearDown() throws InterruptedException, IOException, StorageEngineException {
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void testSyncReceiverLogger() throws IOException {
+    receiverLogger = new SyncReceiverLogger(
+        new File(getReceiverFolderFile(), SyncConstant.SYNC_LOG_NAME));
+    Set<String> deletedFileNames = new HashSet<>();
+    Set<String> deletedFileNamesTest = new HashSet<>();
+    receiverLogger.startSyncDeletedFilesName();
+    for (int i = 0; i < 200; i++) {
+      receiverLogger
+          .finishSyncDeletedFileName(new File(getReceiverFolderFile(), "deleted" + i));
+      deletedFileNames
+          .add(new File(getReceiverFolderFile(), "deleted" + i).getAbsolutePath());
+    }
+    Set<String> toBeSyncedFiles = new HashSet<>();
+    Set<String> toBeSyncedFilesTest = new HashSet<>();
+    receiverLogger.startSyncTsFiles();
+    for (int i = 0; i < 200; i++) {
+      receiverLogger
+          .finishSyncTsfile(new File(getReceiverFolderFile(), "new" + i));
+      toBeSyncedFiles
+          .add(new File(getReceiverFolderFile(), "new" + i).getAbsolutePath());
+    }
+    receiverLogger.close();
+    int count = 0;
+    int mode = 0;
+    try (BufferedReader br = new BufferedReader(
+        new FileReader(new File(getReceiverFolderFile(), SyncConstant.SYNC_LOG_NAME)))) {
+      String line;
+      while ((line = br.readLine()) != null) {
+        count++;
+        if (line.equals(SyncReceiverLogger.SYNC_DELETED_FILE_NAME_START)) {
+          mode = -1;
+        } else if (line.equals(SyncReceiverLogger.SYNC_TSFILE_START)) {
+          mode = 1;
+        } else {
+          if (mode == -1) {
+            deletedFileNamesTest.add(line);
+          } else if (mode == 1) {
+            toBeSyncedFilesTest.add(line);
+          }
+        }
+      }
+    }
+    assertEquals(402, count);
+    assertEquals(deletedFileNames.size(), deletedFileNamesTest.size());
+    assertEquals(toBeSyncedFiles.size(), toBeSyncedFilesTest.size());
+    assertTrue(deletedFileNames.containsAll(deletedFileNamesTest));
+    assertTrue(toBeSyncedFiles.containsAll(toBeSyncedFilesTest));
+  }
+
+  private File getReceiverFolderFile() {
+    return new File(dataDir + File.separatorChar + SyncConstant.SYNC_RECEIVER + File.separatorChar
+        + "127.0.0.1_5555");
+  }
+}
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/sender/MultipleClientSyncTest.java b/server/src/test/java/org/apache/iotdb/db/sync/sender/MultipleClientSyncTest.java
deleted file mode 100644
index c17ba00..0000000
--- a/server/src/test/java/org/apache/iotdb/db/sync/sender/MultipleClientSyncTest.java
+++ /dev/null
@@ -1,226 +0,0 @@
-/*
- * 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.sync.sender;
-
-import static org.junit.Assert.fail;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import org.apache.iotdb.jdbc.Config;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class MultipleClientSyncTest {
-
-  Map<String, ArrayList<String>> timeseriesList = new HashMap();
-  Map<String, ArrayList<String>> timeseriesList1 = new HashMap();
-  private static final Logger logger = LoggerFactory.getLogger(MultipleClientSyncTest.class);
-  private Set<String> dataSender = new HashSet<>();
-  private Set<String> dataReceiver = new HashSet<>();
-
-  public static void main(String[] args) throws IOException {
-    MultipleClientSyncTest multipleClientSyncTest = new MultipleClientSyncTest();
-    multipleClientSyncTest.testPostback();
-  }
-
-  public void testPostback() throws IOException {
-
-    timeseriesList1.put("root.vehicle_history1", new ArrayList<>());
-    timeseriesList1.put("root.vehicle_alarm1", new ArrayList<>());
-    timeseriesList1.put("root.vehicle_temp1", new ArrayList<>());
-    timeseriesList1.put("root.range_event1", new ArrayList<>());
-    timeseriesList.put("root.vehicle_history", new ArrayList<>());
-    timeseriesList.put("root.vehicle_alarm", new ArrayList<>());
-    timeseriesList.put("root.vehicle_temp", new ArrayList<>());
-    timeseriesList.put("root.range_event", new ArrayList<>());
-
-    File file = new File("CreateTimeseries1.txt");
-    BufferedReader reader = new BufferedReader(new FileReader(file));
-    String line;
-    while ((line = reader.readLine()) != null) {
-      String timeseries = line.split(" ")[2];
-      for (String storageGroup : timeseriesList.keySet()) {
-        if (timeseries.startsWith(storageGroup + ".")) {
-          String timesery = timeseries.substring((storageGroup + ".").length());
-          timeseriesList.get(storageGroup).add(timesery);
-          break;
-        }
-      }
-    }
-
-    file = new File("CreateTimeseries2.txt");
-    reader = new BufferedReader(new FileReader(file));
-    while ((line = reader.readLine()) != null) {
-      String timeseries = line.split(" ")[2];
-      for (String storageGroup : timeseriesList1.keySet()) {
-        if (timeseries.startsWith(storageGroup + ".")) {
-          String timesery = timeseries.substring((storageGroup + ".").length());
-          timeseriesList1.get(storageGroup).add(timesery);
-          break;
-        }
-      }
-    }
-
-    for (String storageGroup : timeseriesList.keySet()) {
-      String sqlFormat = "select %s from %s";
-      logger.debug(String.format("%s:", storageGroup));
-      int count = 0;
-      int count1 = 0;
-      int count2 = 0;
-      for (String timesery : timeseriesList.get(storageGroup)) {
-        count++;
-        count1 = 0;
-        count2 = 0;
-        dataSender.clear();
-        dataReceiver.clear();
-        try {
-          Class.forName(Config.JDBC_DRIVER_NAME);
-          Connection connection = null;
-          Connection connection1 = null;
-          try {
-            connection = DriverManager
-                .getConnection("jdbc:iotdb://192.168.130.14:6667/", "root", "root");
-            connection1 = DriverManager
-                .getConnection("jdbc:iotdb://192.168.130.16:6667/", "root", "root");
-            Statement statement = connection.createStatement();
-            Statement statement1 = connection1.createStatement();
-            String sql = String.format(sqlFormat, timesery, storageGroup);
-            boolean hasResultSet = statement.execute(sql);
-            boolean hasResultSet1 = statement1.execute(sql);
-            if (hasResultSet) {
-              ResultSet res = statement.getResultSet();
-              while (res.next()) {
-                count1++;
-                dataSender
-                    .add(res.getString("Time") + res.getString(storageGroup + "." + timesery));
-              }
-            }
-            if (hasResultSet1) {
-              ResultSet res = statement1.getResultSet();
-              while (res.next()) {
-                count2++;
-                dataReceiver
-                    .add(res.getString("Time") + res.getString(storageGroup + "." + timesery));
-              }
-            }
-            assert ((dataSender.size() == dataReceiver.size()) && dataSender
-                .containsAll(dataReceiver));
-            statement.close();
-            statement1.close();
-          } catch (Exception e) {
-            logger.error("", e);
-          } finally {
-            if (connection != null) {
-              connection.close();
-            }
-            if (connection1 != null) {
-              connection1.close();
-            }
-          }
-        } catch (ClassNotFoundException | SQLException e) {
-          fail(e.getMessage());
-        }
-        if (count > 20) {
-          break;
-        }
-        logger.debug(String.valueOf(count1));
-        logger.debug(String.valueOf(count2));
-      }
-    }
-
-    for (String storageGroup : timeseriesList1.keySet()) {
-      String sqlFormat = "select %s from %s";
-      logger.debug(String.format("%s:", storageGroup));
-      int count = 0;
-      int count1;
-      int count2;
-      for (String timesery : timeseriesList1.get(storageGroup)) {
-        count++;
-        count1 = 0;
-        count2 = 0;
-        dataSender.clear();
-        dataReceiver.clear();
-        try {
-          Class.forName(Config.JDBC_DRIVER_NAME);
-          Connection connection = null;
-          Connection connection1 = null;
-          try {
-            connection = DriverManager
-                .getConnection("jdbc:iotdb://192.168.130.15:6667/", "root", "root");
-            connection1 = DriverManager
-                .getConnection("jdbc:iotdb://192.168.130.16:6667/", "root", "root");
-            Statement statement = connection.createStatement();
-            Statement statement1 = connection1.createStatement();
-            String sql = String.format(sqlFormat, timesery, storageGroup);
-            boolean hasResultSet = statement.execute(sql);
-            boolean hasResultSet1 = statement1.execute(sql);
-            if (hasResultSet) {
-              ResultSet res = statement.getResultSet();
-              while (res.next()) {
-                count1++;
-                dataSender
-                    .add(res.getString("Time") + res.getString(storageGroup + "." + timesery));
-              }
-            }
-            if (hasResultSet1) {
-              ResultSet res = statement1.getResultSet();
-              while (res.next()) {
-                count2++;
-                dataReceiver
-                    .add(res.getString("Time") + res.getString(storageGroup + "." + timesery));
-              }
-            }
-            assert ((dataSender.size() == dataReceiver.size()) && dataSender
-                .containsAll(dataReceiver));
-            statement.close();
-            statement1.close();
-          } catch (Exception e) {
-            logger.error("", e);
-          } finally {
-            if (connection != null) {
-              connection.close();
-            }
-            if (connection1 != null) {
-              connection1.close();
-            }
-          }
-        } catch (ClassNotFoundException | SQLException e) {
-          fail(e.getMessage());
-        }
-        if (count > 20) {
-          break;
-        }
-        logger.debug(String.valueOf(count1));
-        logger.debug(String.valueOf(count2));
-      }
-    }
-  }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/sender/SingleClientSyncTest.java b/server/src/test/java/org/apache/iotdb/db/sync/sender/SingleClientSyncTest.java
deleted file mode 100644
index 09efb71..0000000
--- a/server/src/test/java/org/apache/iotdb/db/sync/sender/SingleClientSyncTest.java
+++ /dev/null
@@ -1,559 +0,0 @@
-/*
- * 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.sync.sender;
-
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.HashSet;
-import java.util.Set;
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.StartupException;
-import org.apache.iotdb.db.exception.SyncConnectionException;
-import org.apache.iotdb.db.integration.Constant;
-import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.sync.conf.Constans;
-import org.apache.iotdb.db.sync.conf.SyncSenderConfig;
-import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.jdbc.Config;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The test is to run a complete sync function Before you run the test, make sure receiver has been
- * cleaned up and inited.
- */
-public class SingleClientSyncTest {
-
-  SyncSenderImpl fileSenderImpl = SyncSenderImpl.getInstance();
-  private IoTDBConfig conf = IoTDBDescriptor.getInstance().getConfig();
-  private String serverIpTest = "192.168.130.7";
-  private SyncSenderConfig config = SyncSenderDescriptor.getInstance().getConfig();
-  private Set<String> dataSender = new HashSet<>();
-  private Set<String> dataReceiver = new HashSet<>();
-  private boolean success = true;
-  private IoTDB deamon;
-  private static final String[] sqls1 = new String[]{"SET STORAGE GROUP TO root.vehicle",
-      "SET STORAGE GROUP TO root.test",
-      "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
-      "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN",
-      "CREATE TIMESERIES root.vehicle.d1.s2 WITH DATATYPE=FLOAT, ENCODING=RLE",
-      "CREATE TIMESERIES root.vehicle.d1.s3 WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
-      "CREATE TIMESERIES root.test.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
-      "CREATE TIMESERIES root.test.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN",
-      "CREATE TIMESERIES root.test.d1.g0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
-      "insert into root.vehicle.d0(timestamp,s0) values(10,100)",
-      "insert into root.vehicle.d0(timestamp,s0,s1) values(12,101,'102')",
-      "insert into root.vehicle.d0(timestamp,s1) values(19,'103')",
-      "insert into root.vehicle.d1(timestamp,s2) values(11,104.0)",
-      "insert into root.vehicle.d1(timestamp,s2,s3) values(15,105.0,true)",
-      "insert into root.vehicle.d1(timestamp,s3) values(17,false)",
-      "insert into root.vehicle.d0(timestamp,s0) values(20,1000)",
-      "insert into root.vehicle.d0(timestamp,s0,s1) values(22,1001,'1002')",
-      "insert into root.vehicle.d0(timestamp,s1) values(29,'1003')",
-      "insert into root.vehicle.d1(timestamp,s2) values(21,1004.0)",
-      "insert into root.vehicle.d1(timestamp,s2,s3) values(25,1005.0,true)",
-      "insert into root.vehicle.d1(timestamp,s3) values(27,true)",
-      "insert into root.test.d0(timestamp,s0) values(10,106)",
-      "insert into root.test.d0(timestamp,s0,s1) values(14,107,'108')",
-      "insert into root.test.d0(timestamp,s1) values(16,'109')",
-      "insert into root.test.d1.g0(timestamp,s0) values(1,110)",
-      "insert into root.test.d0(timestamp,s0) values(30,1006)",
-      "insert into root.test.d0(timestamp,s0,s1) values(34,1007,'1008')",
-      "insert into root.test.d0(timestamp,s1) values(36,'1090')",
-      "insert into root.test.d1.g0(timestamp,s0) values(10,1100)", "merge", "flush",};
-  private static final String[] sqls2 = new String[]{
-      "insert into root.vehicle.d0(timestamp,s0) values(6,120)",
-      "insert into root.vehicle.d0(timestamp,s0,s1) values(38,121,'122')",
-      "insert into root.vehicle.d0(timestamp,s1) values(9,'123')",
-      "insert into root.vehicle.d0(timestamp,s0) values(16,128)",
-      "insert into root.vehicle.d0(timestamp,s0,s1) values(18,189,'198')",
-      "insert into root.vehicle.d0(timestamp,s1) values(99,'1234')",
-      "insert into root.vehicle.d1(timestamp,s2) values(14,1024.0)",
-      "insert into root.vehicle.d1(timestamp,s2,s3) values(29,1205.0,true)",
-      "insert into root.vehicle.d1(timestamp,s3) values(33,true)",
-      "insert into root.test.d0(timestamp,s0) values(15,126)",
-      "insert into root.test.d0(timestamp,s0,s1) values(8,127,'128')",
-      "insert into root.test.d0(timestamp,s1) values(20,'129')",
-      "insert into root.test.d1.g0(timestamp,s0) values(14,430)",
-      "insert into root.test.d0(timestamp,s0) values(150,426)",
-      "insert into root.test.d0(timestamp,s0,s1) values(80,427,'528')",
-      "insert into root.test.d0(timestamp,s1) values(2,'1209')",
-      "insert into root.test.d1.g0(timestamp,s0) values(4,330)", "merge", "flush",};
-  private static final String[] sqls3 = new String[]{"SET STORAGE GROUP TO root.iotdb",
-      "SET STORAGE GROUP TO root.flush",
-      "CREATE TIMESERIES root.iotdb.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
-      "CREATE TIMESERIES root.iotdb.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN",
-      "CREATE TIMESERIES root.iotdb.d1.s2 WITH DATATYPE=FLOAT, ENCODING=RLE",
-      "CREATE TIMESERIES root.iotdb.d1.s3 WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
-      "CREATE TIMESERIES root.flush.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
-      "CREATE TIMESERIES root.flush.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN",
-      "CREATE TIMESERIES root.flush.d1.g0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
-      "insert into root.iotdb.d0(timestamp,s0) values(3,100)",
-      "insert into root.iotdb.d0(timestamp,s0,s1) values(22,101,'102')",
-      "insert into root.iotdb.d0(timestamp,s1) values(24,'103')",
-      "insert into root.iotdb.d1(timestamp,s2) values(21,104.0)",
-      "insert into root.iotdb.d1(timestamp,s2,s3) values(25,105.0,true)",
-      "insert into root.iotdb.d1(timestamp,s3) values(27,false)",
-      "insert into root.iotdb.d0(timestamp,s0) values(30,1000)",
-      "insert into root.iotdb.d0(timestamp,s0,s1) values(202,101,'102')",
-      "insert into root.iotdb.d0(timestamp,s1) values(44,'103')",
-      "insert into root.iotdb.d1(timestamp,s2) values(1,404.0)",
-      "insert into root.iotdb.d1(timestamp,s2,s3) values(250,10.0,true)",
-      "insert into root.iotdb.d1(timestamp,s3) values(207,false)",
-      "insert into root.flush.d0(timestamp,s0) values(20,106)",
-      "insert into root.flush.d0(timestamp,s0,s1) values(14,107,'108')",
-      "insert into root.flush.d1.g0(timestamp,s0) values(1,110)",
-      "insert into root.flush.d0(timestamp,s0) values(200,1006)",
-      "insert into root.flush.d0(timestamp,s0,s1) values(1004,1007,'1080')",
-      "insert into root.flush.d1.g0(timestamp,s0) values(1000,910)",
-      "insert into root.vehicle.d0(timestamp,s0) values(209,130)",
-      "insert into root.vehicle.d0(timestamp,s0,s1) values(206,131,'132')",
-      "insert into root.vehicle.d0(timestamp,s1) values(70,'33')",
-      "insert into root.vehicle.d1(timestamp,s2) values(204,14.0)",
-      "insert into root.vehicle.d1(timestamp,s2,s3) values(29,135.0,false)",
-      "insert into root.vehicle.d1(timestamp,s3) values(14,false)",
-      "insert into root.test.d0(timestamp,s0) values(19,136)",
-      "insert into root.test.d0(timestamp,s0,s1) values(7,137,'138')",
-      "insert into root.test.d0(timestamp,s1) values(30,'139')",
-      "insert into root.test.d1.g0(timestamp,s0) values(4,150)",
-      "insert into root.test.d0(timestamp,s0) values(1900,1316)",
-      "insert into root.test.d0(timestamp,s0,s1) values(700,1307,'1038')",
-      "insert into root.test.d0(timestamp,s1) values(3000,'1309')",
-      "insert into root.test.d1.g0(timestamp,s0) values(400,1050)", "merge", "flush",};
-  private boolean testFlag = Constant.testFlag;
-  private static final String SYNC_CLIENT = Constans.SYNC_CLIENT;
-  private static final Logger logger = LoggerFactory.getLogger(SingleClientSyncTest.class);
-
-  public static void main(String[] args) throws Exception {
-    SingleClientSyncTest singleClientPostBackTest = new SingleClientSyncTest();
-    singleClientPostBackTest.setUp();
-    singleClientPostBackTest.testPostback();
-    singleClientPostBackTest.tearDown();
-    System.exit(0);
-  }
-
-  public void setConfig() {
-    config.setUuidPath(
-        config.getDataDirectory() + SYNC_CLIENT + File.separator + Constans.UUID_FILE_NAME);
-    config.setLastFileInfo(
-        config.getDataDirectory() + SYNC_CLIENT + File.separator + Constans.LAST_LOCAL_FILE_NAME);
-    String[] sequenceFileDirectory = config.getSeqFileDirectory();
-    String[] snapshots = new String[config.getSeqFileDirectory().length];
-    for (int i = 0; i < config.getSeqFileDirectory().length; i++) {
-      sequenceFileDirectory[i] = new File(sequenceFileDirectory[i]).getAbsolutePath();
-      if (!sequenceFileDirectory[i].endsWith(File.separator)) {
-        sequenceFileDirectory[i] = sequenceFileDirectory[i] + File.separator;
-      }
-      snapshots[i] =
-          sequenceFileDirectory[i] + SYNC_CLIENT + File.separator + Constans.DATA_SNAPSHOT_NAME
-              + File.separator;
-    }
-    config.setSnapshotPaths(snapshots);
-    config.setSeqFileDirectory(sequenceFileDirectory);
-    config.setServerIp(serverIpTest);
-    fileSenderImpl.setConfig(config);
-  }
-
-  public void setUp() throws StartupException, IOException {
-    if (testFlag) {
-      EnvironmentUtils.closeStatMonitor();
-      deamon = IoTDB.getInstance();
-      deamon.active();
-      EnvironmentUtils.envSetUp();
-    }
-    setConfig();
-  }
-
-  public void tearDown() throws Exception {
-    if (testFlag) {
-      deamon.stop();
-      EnvironmentUtils.cleanEnv();
-    }
-    if (success) {
-      logger.debug("Test succeed!");
-    } else {
-      logger.debug("Test failed!");
-    }
-  }
-
-  public void testPostback() throws IOException, SyncConnectionException, ClassNotFoundException, SQLException, InterruptedException {
-    if (testFlag) {
-      // the first time to sync
-      logger.debug("It's the first time to sync!");
-      try {
-        Class.forName(Config.JDBC_DRIVER_NAME);
-        try (Connection connection = DriverManager
-            .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
-                "root");
-            Statement statement = connection.createStatement()) {
-          for (String sql : sqls1) {
-            statement.execute(sql);
-          }
-        }
-      } catch (SQLException | ClassNotFoundException e) {
-        fail(e.getMessage());
-      }
-
-      fileSenderImpl.sync();
-
-      // Compare data of sender and receiver
-      dataSender.clear();
-      try {
-        Class.forName(Config.JDBC_DRIVER_NAME);
-        try (Connection connection = DriverManager
-            .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
-                "root");
-            Statement statement = connection.createStatement()) {
-          boolean hasResultSet = statement.execute("select * from root.vehicle");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataSender.add(res.getString("Time") + res.getString("root.vehicle.d0.s0")
-                  + res.getString("root.vehicle.d0.s1") + res.getString("root.vehicle.d1.s2")
-                  + res.getString("root.vehicle.d1.s3"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.vehicle.d0.s0")
-                  + res.getString("root.vehicle.d0.s1") + res.getString("root.vehicle.d1.s2")
-                  + res.getString("root.vehicle.d1.s3"));
-            }
-          }
-          hasResultSet = statement.execute("select * from root.test");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataSender.add(res.getString("Time") + res.getString("root.test.d0.s0")
-                  + res.getString("root.test.d0.s1") + res.getString("root.test.d1.g0.s0"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.test.d0.s0")
-                  + res.getString("root.test.d0.s1") + res.getString("root.test.d1.g0.s0"));
-            }
-          }
-        } catch (Exception e) {
-          logger.error("", e);
-        }
-      } catch (ClassNotFoundException e) {
-        fail(e.getMessage());
-        Thread.currentThread().interrupt();
-      }
-
-      dataReceiver.clear();
-      try {
-        Class.forName(Config.JDBC_DRIVER_NAME);
-        try (Connection connection = DriverManager
-            .getConnection(String.format("jdbc:iotdb://%s:6667/", serverIpTest), "root",
-                "root");
-            Statement statement = connection.createStatement()) {
-
-          boolean hasResultSet = statement.execute("select * from root.vehicle");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataReceiver.add(res.getString("Time") + res.getString("root.vehicle.d0.s0")
-                  + res.getString("root.vehicle.d0.s1") + res.getString("root.vehicle.d1.s2")
-                  + res.getString("root.vehicle.d1.s3"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.vehicle.d0.s0")
-                  + res.getString("root.vehicle.d0.s1") + res.getString("root.vehicle.d1.s2")
-                  + res.getString("root.vehicle.d1.s3"));
-            }
-          }
-
-          hasResultSet = statement.execute("select * from root.test");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataReceiver.add(res.getString("Time") + res.getString("root.test.d0.s0")
-                  + res.getString("root.test.d0.s1") + res.getString("root.test.d1.g0.s0"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.test.d0.s0")
-                  + res.getString("root.test.d0.s1") + res.getString("root.test.d1.g0.s0"));
-            }
-          }
-        } catch (Exception e) {
-          logger.error("", e);
-        }
-      } catch (ClassNotFoundException e) {
-        fail(e.getMessage());
-        Thread.currentThread().interrupt();
-      }
-      logger.debug(String.valueOf(dataSender.size()));
-      logger.debug(String.valueOf(dataReceiver.size()));
-      logger.debug(dataSender.toString());
-      logger.debug(dataReceiver.toString());
-      if (!(dataSender.size() == dataReceiver.size() && dataSender.containsAll(dataReceiver))) {
-        success = false;
-        return;
-      }
-
-      // the second time to sync
-      logger.debug("It's the second time to sync!");
-      try {
-        Class.forName(Config.JDBC_DRIVER_NAME);
-        try (Connection connection = DriverManager
-            .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
-                "root");
-            Statement statement = connection.createStatement()) {
-          for (String sql : sqls2) {
-            statement.execute(sql);
-          }
-        } catch (Exception e) {
-          logger.error("", e);
-        }
-      } catch (ClassNotFoundException e) {
-        fail(e.getMessage());
-        Thread.currentThread().interrupt();
-      }
-
-      fileSenderImpl.sync();
-
-      // Compare data of sender and receiver
-      dataSender.clear();
-      try {
-        Class.forName(Config.JDBC_DRIVER_NAME);
-        try (Connection connection = DriverManager
-            .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
-                "root");
-            Statement statement = connection.createStatement()) {
-          boolean hasResultSet = statement.execute("select * from root.vehicle");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataSender.add(res.getString("Time") + res.getString("root.vehicle.d0.s0")
-                  + res.getString("root.vehicle.d0.s1") + res.getString("root.vehicle.d1.s2")
-                  + res.getString("root.vehicle.d1.s3"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.vehicle.d0.s0")
-                  + res.getString("root.vehicle.d0.s1") + res.getString("root.vehicle.d1.s2")
-                  + res.getString("root.vehicle.d1.s3"));
-            }
-          }
-          hasResultSet = statement.execute("select * from root.test");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataSender.add(res.getString("Time") + res.getString("root.test.d0.s0")
-                  + res.getString("root.test.d0.s1") + res.getString("root.test.d1.g0.s0"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.test.d0.s0")
-                  + res.getString("root.test.d0.s1") + res.getString("root.test.d1.g0.s0"));
-            }
-          }
-        } catch (Exception e) {
-          logger.error("", e);
-        }
-      } catch (ClassNotFoundException e) {
-        fail(e.getMessage());
-        Thread.currentThread().interrupt();
-      }
-
-      dataReceiver.clear();
-      {
-        Class.forName(Config.JDBC_DRIVER_NAME);
-        try (Connection connection = DriverManager
-            .getConnection(String.format("jdbc:iotdb://%s:6667/", serverIpTest), "root",
-                "root")) {
-          Statement statement = connection.createStatement();
-          boolean hasResultSet = statement.execute("select * from root.vehicle");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataReceiver.add(res.getString("Time") + res.getString("root.vehicle.d0.s0")
-                  + res.getString("root.vehicle.d0.s1") + res.getString("root.vehicle.d1.s2")
-                  + res.getString("root.vehicle.d1.s3"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.vehicle.d0.s0")
-                  + res.getString("root.vehicle.d0.s1") + res.getString("root.vehicle.d1.s2")
-                  + res.getString("root.vehicle.d1.s3"));
-            }
-          }
-          hasResultSet = statement.execute("select * from root.test");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataReceiver.add(res.getString("Time") + res.getString("root.test.d0.s0")
-                  + res.getString("root.test.d0.s1") + res.getString("root.test.d1.g0.s0"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.test.d0.s0")
-                  + res.getString("root.test.d0.s1") + res.getString("root.test.d1.g0.s0"));
-            }
-          }
-          statement.close();
-        } catch (Exception e) {
-          logger.error("", e);
-        }
-      }
-      logger.debug(String.valueOf(dataSender.size()));
-      logger.debug(String.valueOf(dataReceiver.size()));
-      logger.debug(dataSender.toString());
-      logger.debug(dataReceiver.toString());
-      if (!(dataSender.size() == dataReceiver.size() && dataSender.containsAll(dataReceiver))) {
-        success = false;
-        return;
-      }
-
-      // the third time to sync
-      logger.debug("It's the third time to sync!");
-      try {
-        Class.forName(Config.JDBC_DRIVER_NAME);
-        try (Connection connection = DriverManager
-            .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
-                "root");
-            Statement statement = connection.createStatement()) {
-          for (String sql : sqls3) {
-            statement.execute(sql);
-          }
-        } catch (Exception e) {
-          logger.error("", e);
-        }
-      } catch (ClassNotFoundException e) {
-        fail(e.getMessage());
-        Thread.currentThread().interrupt();
-      }
-
-      fileSenderImpl.sync();
-
-      // Compare data of sender and receiver
-      dataSender.clear();
-      try {
-        Class.forName(Config.JDBC_DRIVER_NAME);
-        try (Connection connection = DriverManager
-            .getConnection(String.format("jdbc:iotdb://%s:6667/", serverIpTest), "root",
-                "root");
-            Statement statement = connection.createStatement()) {
-          boolean hasResultSet = statement.execute("select * from root.vehicle");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataSender.add(res.getString("Time") + res.getString("root.vehicle.d0.s0")
-                  + res.getString("root.vehicle.d0.s1") + res.getString("root.vehicle.d1.s2")
-                  + res.getString("root.vehicle.d1.s3"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.vehicle.d0.s0")
-                  + res.getString("root.vehicle.d0.s1") + res.getString("root.vehicle.d1.s2")
-                  + res.getString("root.vehicle.d1.s3"));
-            }
-          }
-          hasResultSet = statement.execute("select * from root.test");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataSender.add(res.getString("Time") + res.getString("root.test.d0.s0")
-                  + res.getString("root.test.d0.s1") + res.getString("root.test.d1.g0.s0"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.test.d0.s0")
-                  + res.getString("root.test.d0.s1") + res.getString("root.test.d1.g0.s0"));
-            }
-          }
-          hasResultSet = statement.execute("select * from root.flush");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataSender.add(res.getString("Time") + res.getString("root.flush.d0.s0")
-                  + res.getString("root.flush.d0.s1") + res.getString("root.flush.d1.g0.s0"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.flush.d0.s0")
-                  + res.getString("root.flush.d0.s1") + res.getString("root.flush.d1.g0.s0"));
-            }
-          }
-          hasResultSet = statement.execute("select * from root.iotdb");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataSender.add(res.getString("Time") + res.getString("root.iotdb.d0.s0")
-                  + res.getString("root.iotdb.d0.s1") + res.getString("root.iotdb.d1.s2")
-                  + res.getString("root.iotdb.d1.s3"));
-              logger.debug(res.getString("Time") + res.getString("root.iotdb.d0.s0")
-                  + res.getString("root.iotdb.d0.s1") + res.getString("root.iotdb.d1.s2")
-                  + res.getString("root.iotdb.d1.s3"));
-            }
-          }
-        } catch (Exception e) {
-          logger.error("", e);
-        }
-      } catch (ClassNotFoundException e) {
-        fail(e.getMessage());
-        Thread.currentThread().interrupt();
-      }
-
-      dataReceiver.clear();
-      try {
-        Class.forName(Config.JDBC_DRIVER_NAME);
-        try (Connection connection = DriverManager
-            .getConnection("jdbc:iotdb://192.168.130.8:6667/", "root", "root")) {
-          Statement statement = connection.createStatement();
-          boolean hasResultSet = statement.execute("select * from root.vehicle");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataReceiver.add(res.getString("Time") + res.getString("root.vehicle.d0.s0")
-                  + res.getString("root.vehicle.d0.s1") + res.getString("root.vehicle.d1.s2")
-                  + res.getString("root.vehicle.d1.s3"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.vehicle.d0.s0")
-                  + res.getString("root.vehicle.d0.s1") + res.getString("root.vehicle.d1.s2")
-                  + res.getString("root.vehicle.d1.s3"));
-            }
-          }
-          hasResultSet = statement.execute("select * from root.test");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataReceiver.add(res.getString("Time") + res.getString("root.test.d0.s0")
-                  + res.getString("root.test.d0.s1") + res.getString("root.test.d1.g0.s0"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.test.d0.s0")
-                  + res.getString("root.test.d0.s1") + res.getString("root.test.d1.g0.s0"));
-            }
-          }
-          hasResultSet = statement.execute("select * from root.flush");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataReceiver.add(res.getString("Time") + res.getString("root.flush.d0.s0")
-                  + res.getString("root.flush.d0.s1") + res.getString("root.flush.d1.g0.s0"));
-              logger.debug(res.getString("Time") + " | " + res.getString("root.flush.d0.s0")
-                  + res.getString("root.flush.d0.s1") + res.getString("root.flush.d1.g0.s0"));
-            }
-          }
-          hasResultSet = statement.execute("select * from root.iotdb");
-          if (hasResultSet) {
-            ResultSet res = statement.getResultSet();
-            while (res.next()) {
-              dataReceiver.add(res.getString("Time") + res.getString("root.iotdb.d0.s0")
-                  + res.getString("root.iotdb.d0.s1") + res.getString("root.iotdb.d1.s2")
-                  + res.getString("root.iotdb.d1.s3"));
-              logger.debug(res.getString("Time") + res.getString("root.iotdb.d0.s0")
-                  + res.getString("root.iotdb.d0.s1") + res.getString("root.iotdb.d1.s2")
-                  + res.getString("root.iotdb.d1.s3"));
-            }
-          }
-          statement.close();
-        } catch (Exception e) {
-          logger.error("", e);
-        }
-      } catch (ClassNotFoundException e) {
-        fail(e.getMessage());
-        Thread.currentThread().interrupt();
-      }
-      logger.debug(String.valueOf(dataSender.size()));
-      logger.debug(String.valueOf(dataReceiver.size()));
-      logger.debug(String.valueOf(dataSender));
-      logger.debug(String.valueOf(dataReceiver));
-      if (!(dataSender.size() == dataReceiver.size() && dataSender.containsAll(dataReceiver))) {
-        success = false;
-      }
-    }
-  }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/sender/SyncFileManagerTest.java b/server/src/test/java/org/apache/iotdb/db/sync/sender/SyncFileManagerTest.java
deleted file mode 100644
index 2092f3f..0000000
--- a/server/src/test/java/org/apache/iotdb/db/sync/sender/SyncFileManagerTest.java
+++ /dev/null
@@ -1,374 +0,0 @@
-/*
- * 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.sync.sender;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Random;
-import java.util.Set;
-import org.apache.iotdb.db.sync.conf.Constans;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class SyncFileManagerTest {
-
-  private static final String POST_BACK_DIRECTORY_TEST = Constans.SYNC_CLIENT + File.separator;
-  private static final String LAST_FILE_INFO_TEST =
-      POST_BACK_DIRECTORY_TEST + Constans.LAST_LOCAL_FILE_NAME;
-  private static final String SENDER_FILE_PATH_TEST = POST_BACK_DIRECTORY_TEST + "data";
-  private SyncFileManager manager = SyncFileManager.getInstance();
-  private static final Logger logger = LoggerFactory.getLogger(SyncFileManagerTest.class);
-
-  @Before
-  public void setUp() throws IOException, InterruptedException {
-    File file = new File(LAST_FILE_INFO_TEST);
-    if (!file.getParentFile().exists()) {
-      file.getParentFile().mkdirs();
-    }
-    if (!file.exists() && !file.createNewFile()) {
-      logger.error("Can not create new file {}", file.getPath());
-    }
-    file = new File(SENDER_FILE_PATH_TEST);
-    if (!file.exists()) {
-      file.mkdirs();
-    }
-    manager.setCurrentLocalFiles(new HashMap<>());
-  }
-
-  @After
-  public void tearDown() throws InterruptedException {
-    delete(new File(POST_BACK_DIRECTORY_TEST));
-    new File(POST_BACK_DIRECTORY_TEST).delete();
-  }
-
-  public void delete(File file) {
-    if (file.isFile() || file.list().length == 0) {
-      file.delete();
-    } else {
-      File[] files = file.listFiles();
-      for (File f : files) {
-        delete(f);
-        f.delete();
-      }
-    }
-  }
-
-  @Test // It tests two classes : backupNowLocalFileInfo and getLastLocalFileList
-  public void testBackupCurrentLocalFileInfo() throws IOException {
-    Map<String, Set<String>> allFileList = new HashMap<>();
-
-    Random r = new Random(0);
-    for (int i = 0; i < 3; i++) {
-      for (int j = 0; j < 5; j++) {
-        if (!allFileList.containsKey(String.valueOf(i))) {
-          allFileList.put(String.valueOf(i), new HashSet<>());
-        }
-        String rand = String.valueOf(r.nextInt(10000));
-        String fileName =
-            SENDER_FILE_PATH_TEST + File.separator + i + File.separator + rand;
-        File file = new File(fileName);
-        allFileList.get(String.valueOf(i)).add(file.getPath());
-        if (!file.getParentFile().exists()) {
-          file.getParentFile().mkdirs();
-        }
-        if (!file.exists() && !file.createNewFile()) {
-          logger.error("Can not create new file {}", file.getPath());
-        }
-      }
-    }
-    Set<String> lastFileList;
-
-    // lastFileList is empty
-    manager.getLastLocalFileList(LAST_FILE_INFO_TEST);
-    lastFileList = manager.getLastLocalFiles();
-    assert (lastFileList.isEmpty());
-
-    // add some files
-    manager.getCurrentLocalFileList(new String[]{SENDER_FILE_PATH_TEST});
-    manager.backupNowLocalFileInfo(LAST_FILE_INFO_TEST);
-    manager.getLastLocalFileList(LAST_FILE_INFO_TEST);
-    lastFileList = manager.getLastLocalFiles();
-    for (Entry<String, Set<String>> entry : allFileList.entrySet()) {
-      assert (lastFileList.containsAll(entry.getValue()));
-    }
-
-    // add some files and delete some files
-    r = new Random(1);
-    for (int i = 0; i < 3; i++) {
-      for (int j = 0; j < 5; j++) {
-        if (!allFileList.containsKey(String.valueOf(i))) {
-          allFileList.put(String.valueOf(i), new HashSet<>());
-        }
-        String rand = String.valueOf(r.nextInt(10000));
-        String fileName =
-            SENDER_FILE_PATH_TEST + File.separator + i + File.separator + rand;
-        File file = new File(fileName);
-        allFileList.get(String.valueOf(i)).add(file.getPath());
-        if (!file.getParentFile().exists()) {
-          file.getParentFile().mkdirs();
-        }
-        if (!file.exists() && !file.createNewFile()) {
-          logger.error("Can not create new file {}", file.getPath());
-        }
-      }
-    }
-    int count = 0;
-    Map<String, Set<String>> deleteFile = new HashMap<>();
-    for (Entry<String, Set<String>> entry : allFileList.entrySet()) {
-      deleteFile.put(entry.getKey(), new HashSet<>());
-      for (String path : entry.getValue()) {
-        count++;
-        if (count % 3 == 0) {
-          deleteFile.get(entry.getKey()).add(path);
-        }
-      }
-    }
-    for (Entry<String, Set<String>> entry : deleteFile.entrySet()) {
-      for (String path : entry.getValue()) {
-        new File(path).delete();
-        allFileList.get(entry.getKey()).remove(path);
-      }
-    }
-    manager.getCurrentLocalFileList(new String[]{SENDER_FILE_PATH_TEST});
-    manager.backupNowLocalFileInfo(LAST_FILE_INFO_TEST);
-    manager.getLastLocalFileList(LAST_FILE_INFO_TEST);
-    lastFileList = manager.getLastLocalFiles();
-    for (Entry<String, Set<String>> entry : allFileList.entrySet()) {
-      assert (lastFileList.containsAll(entry.getValue()));
-    }
-  }
-
-  @Test
-  public void testGetCurrentLocalFileList() throws IOException {
-    Map<String, Set<String>> allFileList = new HashMap<>();
-    Map<String, Set<String>> fileList;
-
-    // nowLocalList is empty
-    manager.getCurrentLocalFileList(new String[]{SENDER_FILE_PATH_TEST});
-    fileList = manager.getCurrentLocalFiles();
-    assert (isEmpty(fileList));
-
-    // add some files
-    Random r = new Random(0);
-    for (int i = 0; i < 3; i++) {
-      for (int j = 0; j < 5; j++) {
-        if (!allFileList.containsKey(String.valueOf(i))) {
-          allFileList.put(String.valueOf(i), new HashSet<>());
-        }
-        String rand = String.valueOf(r.nextInt(10000));
-        String fileName =
-            SENDER_FILE_PATH_TEST + File.separator + i + File.separator + rand;
-        File file = new File(fileName);
-        allFileList.get(String.valueOf(i)).add(file.getPath());
-        if (!file.getParentFile().exists()) {
-          file.getParentFile().mkdirs();
-        }
-        if (!file.exists() && !file.createNewFile()) {
-          logger.error("Can not create new file {}", file.getPath());
-        }
-      }
-    }
-    manager.getCurrentLocalFileList(new String[]{SENDER_FILE_PATH_TEST});
-    fileList = manager.getCurrentLocalFiles();
-    assert (allFileList.size() == fileList.size());
-    for (Entry<String, Set<String>> entry : fileList.entrySet()) {
-      assert (allFileList.containsKey(entry.getKey()));
-      assert (allFileList.get(entry.getKey()).containsAll(entry.getValue()));
-    }
-
-    // delete some files and add some files
-    int count = 0;
-    Map<String, Set<String>> deleteFile = new HashMap<>();
-    for (Entry<String, Set<String>> entry : allFileList.entrySet()) {
-      deleteFile.put(entry.getKey(), new HashSet<>());
-      for (String path : entry.getValue()) {
-        count++;
-        if (count % 3 == 0) {
-          deleteFile.get(entry.getKey()).add(path);
-        }
-      }
-    }
-    for (Entry<String, Set<String>> entry : deleteFile.entrySet()) {
-      for (String path : entry.getValue()) {
-        new File(path).delete();
-        allFileList.get(entry.getKey()).remove(path);
-      }
-    }
-    r = new Random(1);
-    for (int i = 0; i < 3; i++) {
-      for (int j = 0; j < 5; j++) {
-        if (!allFileList.containsKey(String.valueOf(i))) {
-          allFileList.put(String.valueOf(i), new HashSet<>());
-        }
-        String rand = String.valueOf(r.nextInt(10000));
-        String fileName =
-            SENDER_FILE_PATH_TEST + File.separator + i + File.separator + rand;
-        File file = new File(fileName);
-        allFileList.get(String.valueOf(i)).add(file.getPath());
-        if (!file.getParentFile().exists()) {
-          file.getParentFile().mkdirs();
-        }
-        if (!file.exists() && !file.createNewFile()) {
-          logger.error("Can not create new file {}", file.getPath());
-        }
-      }
-    }
-    manager.setCurrentLocalFiles(new HashMap<>());
-    manager.getCurrentLocalFileList(new String[]{SENDER_FILE_PATH_TEST});
-    fileList = manager.getCurrentLocalFiles();
-    assert (allFileList.size() == fileList.size());
-    for (Entry<String, Set<String>> entry : fileList.entrySet()) {
-      assert (allFileList.containsKey(entry.getKey()));
-      logger.debug("allFileList");
-      for (String a : allFileList.get(entry.getKey())) {
-        logger.debug(a);
-      }
-      logger.debug("FileList");
-      for (String a : entry.getValue()) {
-        logger.debug(a);
-      }
-      assert (allFileList.get(entry.getKey()).containsAll(entry.getValue()));
-    }
-  }
-
-  @Test
-  public void testGetValidFileList() throws IOException {
-    Map<String, Set<String>> allFileList;
-    Map<String, Set<String>> newFileList = new HashMap<>();
-    Map<String, Set<String>> sendingFileList;
-    Set<String> lastLocalList;
-
-    // nowSendingList is empty
-
-    manager.setCurrentLocalFiles(new HashMap<>());
-    manager.getCurrentLocalFileList(new String[]{SENDER_FILE_PATH_TEST});
-    allFileList = manager.getCurrentLocalFiles();
-    manager.getLastLocalFileList(LAST_FILE_INFO_TEST);
-    lastLocalList = manager.getLastLocalFiles();
-    manager.getValidFileList();
-    assert (lastLocalList.isEmpty());
-    assert (isEmpty(allFileList));
-
-    // add some files
-    newFileList.clear();
-    manager.backupNowLocalFileInfo(LAST_FILE_INFO_TEST);
-    Random r = new Random(0);
-    for (int i = 0; i < 3; i++) {
-      for (int j = 0; j < 5; j++) {
-        if (!allFileList.containsKey(String.valueOf(i))) {
-          allFileList.put(String.valueOf(i), new HashSet<>());
-        }
-        if (!newFileList.containsKey(String.valueOf(i))) {
-          newFileList.put(String.valueOf(i), new HashSet<>());
-        }
-        String rand = String.valueOf(r.nextInt(10000));
-        String fileName =
-            SENDER_FILE_PATH_TEST + File.separator + i + File.separator + rand;
-        File file = new File(fileName);
-        allFileList.get(String.valueOf(i)).add(file.getPath());
-        newFileList.get(String.valueOf(i)).add(file.getPath());
-        if (!file.getParentFile().exists()) {
-          file.getParentFile().mkdirs();
-        }
-        if (!file.exists() && !file.createNewFile()) {
-          logger.error("Can not create new file {}", file.getPath());
-        }
-      }
-    }
-    manager.getCurrentLocalFileList(new String[]{SENDER_FILE_PATH_TEST});
-    allFileList = manager.getCurrentLocalFiles();
-    manager.backupNowLocalFileInfo(LAST_FILE_INFO_TEST);
-    manager.getLastLocalFileList(LAST_FILE_INFO_TEST);
-    manager.getValidFileList();
-    sendingFileList = manager.getValidAllFiles();
-    assert (sendingFileList.size() == newFileList.size());
-    for (Entry<String, Set<String>> entry : sendingFileList.entrySet()) {
-      assert (newFileList.containsKey(entry.getKey()));
-      assert (newFileList.get(entry.getKey()).containsAll(entry.getValue()));
-    }
-
-    // delete some files and add some files
-    int count = 0;
-    Map<String, Set<String>> deleteFile = new HashMap<>();
-    for (Entry<String, Set<String>> entry : allFileList.entrySet()) {
-      deleteFile.put(entry.getKey(), new HashSet<>());
-      for (String path : entry.getValue()) {
-        count++;
-        if (count % 3 == 0) {
-          deleteFile.get(entry.getKey()).add(path);
-        }
-      }
-    }
-    for (Entry<String, Set<String>> entry : deleteFile.entrySet()) {
-      for (String path : entry.getValue()) {
-        new File(path).delete();
-        allFileList.get(entry.getKey()).remove(path);
-      }
-    }
-    newFileList.clear();
-    r = new Random(1);
-    for (int i = 0; i < 3; i++) {
-      for (int j = 0; j < 5; j++) {
-        if (!allFileList.containsKey(String.valueOf(i))) {
-          allFileList.put(String.valueOf(i), new HashSet<>());
-        }
-        if (!newFileList.containsKey(String.valueOf(i))) {
-          newFileList.put(String.valueOf(i), new HashSet<>());
-        }
-        String rand = String.valueOf(r.nextInt(10000));
-        String fileName =
-            SENDER_FILE_PATH_TEST + File.separator + i + File.separator + rand;
-        File file = new File(fileName);
-        allFileList.get(String.valueOf(i)).add(file.getPath());
-        newFileList.get(String.valueOf(i)).add(file.getPath());
-        if (!file.getParentFile().exists()) {
-          file.getParentFile().mkdirs();
-        }
-        if (!file.exists() && !file.createNewFile()) {
-          logger.error("Can not create new file {}", file.getPath());
-        }
-      }
-    }
-    manager.getCurrentLocalFileList(new String[]{SENDER_FILE_PATH_TEST});
-    manager.getLastLocalFileList(LAST_FILE_INFO_TEST);
-    manager.getValidFileList();
-    sendingFileList = manager.getValidAllFiles();
-    assert (sendingFileList.size() == newFileList.size());
-    for (Entry<String, Set<String>> entry : sendingFileList.entrySet()) {
-      assert (newFileList.containsKey(entry.getKey()));
-      assert (newFileList.get(entry.getKey()).containsAll(entry.getValue()));
-    }
-  }
-
-  private boolean isEmpty(Map<String, Set<String>> sendingFileList) {
-    for (Entry<String, Set<String>> entry : sendingFileList.entrySet()) {
-      if (!entry.getValue().isEmpty()) {
-        return false;
-      }
-    }
-    return true;
-  }
-}
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManagerTest.java b/server/src/test/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManagerTest.java
new file mode 100644
index 0000000..06a7800
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/sync/sender/manage/SyncFileManagerTest.java
@@ -0,0 +1,295 @@
+/**
+ * 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.sync.sender.manage;
+
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+import static org.junit.Assert.assertTrue;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.db.exception.StartupException;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.sync.conf.SyncSenderConfig;
+import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.db.utils.FilePathUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SyncFileManagerTest {
+
+  private static final Logger logger = LoggerFactory.getLogger(SyncFileManagerTest.class);
+  private ISyncFileManager manager = SyncFileManager.getInstance();
+  private SyncSenderConfig config = SyncSenderDescriptor.getInstance().getConfig();
+  private String dataDir;
+
+  @Before
+  public void setUp()
+      throws IOException, InterruptedException, StartupException, DiskSpaceInsufficientException {
+    EnvironmentUtils.envSetUp();
+    dataDir = new File(DirectoryManager.getInstance().getNextFolderForSequenceFile())
+        .getParentFile().getAbsolutePath();
+    config.update(dataDir);
+  }
+
+  @After
+  public void tearDown() throws InterruptedException, IOException, StorageEngineException {
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void testGetValidFiles() throws IOException {
+    Map<String, Set<File>> allFileList = new HashMap<>();
+
+    Random r = new Random(0);
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 5; j++) {
+        if (!allFileList.containsKey(String.valueOf(i))) {
+          allFileList.put(String.valueOf(i), new HashSet<>());
+        }
+        String rand = r.nextInt(10000) + TSFILE_SUFFIX;
+        String fileName = FilePathUtils.regularizePath(dataDir) + IoTDBConstant.SEQUENCE_FLODER_NAME
+            + File.separator + i
+            + File.separator + rand;
+        File file = new File(fileName);
+        allFileList.get(String.valueOf(i)).add(file);
+        if (!file.getParentFile().exists()) {
+          file.getParentFile().mkdirs();
+        }
+        if (!file.exists() && !file.createNewFile()) {
+          logger.error("Can not create new file {}", file.getPath());
+        }
+        if (!new File(file.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).exists()
+            && !new File(file.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).createNewFile()) {
+          logger.error("Can not create new file {}", file.getPath());
+        }
+      }
+    }
+    Map<String, Set<File>> lastFileMap;
+    Map<String, Set<File>> curFileMap;
+    Map<String, Set<File>> deletedFilesMap;
+    Map<String, Set<File>> toBeSyncedFilesMap;
+
+    // lastFileList is empty
+    manager.getValidFiles(dataDir);
+    assertTrue(isEmpty(manager.getLastLocalFilesMap()));
+
+    updateLastLocalFiles(allFileList);
+
+    manager.getValidFiles(dataDir);
+    lastFileMap = manager.getLastLocalFilesMap();
+    for (Entry<String, Set<File>> entry : allFileList.entrySet()) {
+      assertTrue(lastFileMap.containsKey(entry.getKey()));
+      assertTrue(lastFileMap.get(entry.getKey()).containsAll(entry.getValue()));
+    }
+
+    // add some files
+    Map<String, Set<File>> correctToBeSyncedFiles = new HashMap<>();
+    r = new Random(1);
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 5; j++) {
+        if (!allFileList.containsKey(String.valueOf(i))) {
+          allFileList.put(String.valueOf(i), new HashSet<>());
+        }
+        correctToBeSyncedFiles.putIfAbsent(String.valueOf(i), new HashSet<>());
+        String rand = r.nextInt(10000) + TSFILE_SUFFIX;
+        String fileName =
+            FilePathUtils.regularizePath(dataDir) + IoTDBConstant.SEQUENCE_FLODER_NAME
+                + File.separator + i
+                + File.separator + rand;
+        File file = new File(fileName);
+        allFileList.get(String.valueOf(i)).add(file);
+        correctToBeSyncedFiles.get(String.valueOf(i)).add(file);
+        if (!file.getParentFile().exists()) {
+          file.getParentFile().mkdirs();
+        }
+        if (!file.exists() && !file.createNewFile()) {
+          logger.error("Can not create new file {}", file.getPath());
+        }
+        if (!new File(file.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).exists()
+            && !new File(file.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).createNewFile()) {
+          logger.error("Can not create new file {}", file.getPath());
+        }
+      }
+    }
+    manager.getValidFiles(dataDir);
+    curFileMap = manager.getCurrentSealedLocalFilesMap();
+    toBeSyncedFilesMap = manager.getToBeSyncedFilesMap();
+    for (Entry<String, Set<File>> entry : allFileList.entrySet()) {
+      assertTrue(curFileMap.containsKey(entry.getKey()));
+      assertTrue(curFileMap.get(entry.getKey()).containsAll(entry.getValue()));
+    }
+    for (Entry<String, Set<File>> entry : correctToBeSyncedFiles.entrySet()) {
+      assertTrue(toBeSyncedFilesMap.containsKey(entry.getKey()));
+      assertTrue(toBeSyncedFilesMap.get(entry.getKey()).containsAll(entry.getValue()));
+    }
+    updateLastLocalFiles(allFileList);
+    manager.getValidFiles(dataDir);
+    lastFileMap = manager.getLastLocalFilesMap();
+    for (Entry<String, Set<File>> entry : allFileList.entrySet()) {
+      assertTrue(lastFileMap.containsKey(entry.getKey()));
+      assertTrue(lastFileMap.get(entry.getKey()).containsAll(entry.getValue()));
+    }
+
+    // add some files and delete some files
+    correctToBeSyncedFiles.clear();
+    r = new Random(2);
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 5; j++) {
+        if (!allFileList.containsKey(String.valueOf(i))) {
+          allFileList.put(String.valueOf(i), new HashSet<>());
+        }
+        correctToBeSyncedFiles.putIfAbsent(String.valueOf(i), new HashSet<>());
+        String rand = r.nextInt(10000) + TSFILE_SUFFIX;
+        String fileName =
+            FilePathUtils.regularizePath(dataDir) + IoTDBConstant.SEQUENCE_FLODER_NAME
+                + File.separator + i
+                + File.separator + rand;
+        File file = new File(fileName);
+        allFileList.get(String.valueOf(i)).add(file);
+        correctToBeSyncedFiles.get(String.valueOf(i)).add(file);
+        if (!file.getParentFile().exists()) {
+          file.getParentFile().mkdirs();
+        }
+        if (!file.exists() && !file.createNewFile()) {
+          logger.error("Can not create new file {}", file.getPath());
+        }
+        if (!new File(file.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).exists()
+            && !new File(file.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).createNewFile()) {
+          logger.error("Can not create new file {}", file.getPath());
+        }
+      }
+    }
+    int count = 0;
+    Map<String, Set<File>> correctDeleteFile = new HashMap<>();
+    for (Entry<String, Set<File>> entry : allFileList.entrySet()) {
+      correctDeleteFile.put(entry.getKey(), new HashSet<>());
+      for (File file : entry.getValue()) {
+        count++;
+        if (count % 3 == 0 && lastFileMap.get(entry.getKey()).contains(file)) {
+          correctDeleteFile.get(entry.getKey()).add(file);
+        }
+      }
+    }
+    for (Entry<String, Set<File>> entry : correctDeleteFile.entrySet()) {
+      for (File file : entry.getValue()) {
+        file.delete();
+        new File(file.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).delete();
+        allFileList.get(entry.getKey()).remove(file);
+      }
+    }
+    manager.getValidFiles(dataDir);
+    lastFileMap = manager.getLastLocalFilesMap();
+    curFileMap = manager.getCurrentSealedLocalFilesMap();
+    deletedFilesMap = manager.getDeletedFilesMap();
+    toBeSyncedFilesMap = manager.getToBeSyncedFilesMap();
+    for (Entry<String, Set<File>> entry : allFileList.entrySet()) {
+      assertTrue(curFileMap.containsKey(entry.getKey()));
+      assertTrue(curFileMap.get(entry.getKey()).containsAll(entry.getValue()));
+    }
+    for (Entry<String, Set<File>> entry : correctDeleteFile.entrySet()) {
+      assertTrue(deletedFilesMap.containsKey(entry.getKey()));
+      assertTrue(deletedFilesMap.get(entry.getKey()).containsAll(entry.getValue()));
+    }
+    for (Entry<String, Set<File>> entry : correctToBeSyncedFiles.entrySet()) {
+      assertTrue(toBeSyncedFilesMap.containsKey(entry.getKey()));
+      assertTrue(toBeSyncedFilesMap.get(entry.getKey()).containsAll(entry.getValue()));
+    }
+
+    // add some invalid files
+    r = new Random(3);
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 5; j++) {
+        if (!allFileList.containsKey(String.valueOf(i))) {
+          allFileList.put(String.valueOf(i), new HashSet<>());
+        }
+        String rand = String.valueOf(r.nextInt(10000));
+        String fileName =
+            FilePathUtils.regularizePath(dataDir) + IoTDBConstant.SEQUENCE_FLODER_NAME
+                + File.separator + i
+                + File.separator + rand;
+        File file = new File(fileName);
+        allFileList.get(String.valueOf(i)).add(file);
+        if (!file.getParentFile().exists()) {
+          file.getParentFile().mkdirs();
+        }
+        if (!file.exists() && !file.createNewFile()) {
+          logger.error("Can not create new file {}", file.getPath());
+        }
+      }
+    }
+    manager.getValidFiles(dataDir);
+    lastFileMap = manager.getLastLocalFilesMap();
+    curFileMap = manager.getCurrentSealedLocalFilesMap();
+    deletedFilesMap = manager.getDeletedFilesMap();
+    toBeSyncedFilesMap = manager.getToBeSyncedFilesMap();
+    for (Entry<String, Set<File>> entry : curFileMap.entrySet()) {
+      assertTrue(allFileList.containsKey(entry.getKey()));
+      assertTrue(allFileList.get(entry.getKey()).size() != entry.getValue().size());
+      assertTrue(allFileList.get(entry.getKey()).containsAll(entry.getValue()));
+    }
+    for (Entry<String, Set<File>> entry : correctDeleteFile.entrySet()) {
+      assertTrue(deletedFilesMap.containsKey(entry.getKey()));
+      assertTrue(deletedFilesMap.get(entry.getKey()).containsAll(entry.getValue()));
+    }
+    for (Entry<String, Set<File>> entry : correctToBeSyncedFiles.entrySet()) {
+      assertTrue(toBeSyncedFilesMap.containsKey(entry.getKey()));
+      assertTrue(toBeSyncedFilesMap.get(entry.getKey()).containsAll(entry.getValue()));
+    }
+  }
+
+  private void updateLastLocalFiles(Map<String, Set<File>> lastLocalFilesMap) {
+    try (BufferedWriter bw = new BufferedWriter(
+        new FileWriter(new File(config.getLastFileInfoPath())))) {
+      for (Set<File> currentLocalFiles : lastLocalFilesMap.values()) {
+        for (File file : currentLocalFiles) {
+          bw.write(file.getAbsolutePath());
+          bw.newLine();
+        }
+        bw.flush();
+      }
+    } catch (IOException e) {
+      logger.error("Can not clear sync log {}", config.getLastFileInfoPath(), e);
+    }
+  }
+
+  private boolean isEmpty(Map<String, Set<File>> sendingFileList) {
+    for (Entry<String, Set<File>> entry : sendingFileList.entrySet()) {
+      if (!entry.getValue().isEmpty()) {
+        return false;
+      }
+    }
+    return true;
+  }
+}
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/sender/recover/SyncSenderLogAnalyzerTest.java b/server/src/test/java/org/apache/iotdb/db/sync/sender/recover/SyncSenderLogAnalyzerTest.java
new file mode 100644
index 0000000..70f5200
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/sync/sender/recover/SyncSenderLogAnalyzerTest.java
@@ -0,0 +1,163 @@
+/**
+ * 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.sync.sender.recover;
+
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.db.exception.StartupException;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.db.sync.conf.SyncSenderConfig;
+import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
+import org.apache.iotdb.db.sync.sender.manage.ISyncFileManager;
+import org.apache.iotdb.db.sync.sender.manage.SyncFileManager;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.db.utils.FilePathUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SyncSenderLogAnalyzerTest {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(SyncSenderLogAnalyzerTest.class);
+  private ISyncSenderLogger senderLogger;
+  private ISyncSenderLogAnalyzer senderLogAnalyzer;
+  private ISyncFileManager manager = SyncFileManager.getInstance();
+  private SyncSenderConfig config = SyncSenderDescriptor.getInstance().getConfig();
+  private String dataDir;
+
+  @Before
+  public void setUp()
+      throws IOException, InterruptedException, StartupException, DiskSpaceInsufficientException {
+    EnvironmentUtils.envSetUp();
+    dataDir = new File(DirectoryManager.getInstance().getNextFolderForSequenceFile())
+        .getParentFile().getAbsolutePath();
+    config.update(dataDir);
+    senderLogger = new SyncSenderLogger(
+        new File(config.getSenderFolderPath(), SyncConstant.SYNC_LOG_NAME));
+    senderLogAnalyzer = new SyncSenderLogAnalyzer(config.getSenderFolderPath());
+  }
+
+  @After
+  public void tearDown() throws InterruptedException, IOException, StorageEngineException {
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void recover() throws IOException {
+    Map<String, Set<File>> allFileList = new HashMap<>();
+
+    Random r = new Random(0);
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 5; j++) {
+        if (!allFileList.containsKey(String.valueOf(i))) {
+          allFileList.put(String.valueOf(i), new HashSet<>());
+        }
+        String rand = r.nextInt(10000) + TSFILE_SUFFIX;
+        String fileName = FilePathUtils.regularizePath(dataDir) + IoTDBConstant.SEQUENCE_FLODER_NAME
+            + File.separator + i
+            + File.separator + rand;
+        File file = new File(fileName);
+        allFileList.get(String.valueOf(i)).add(file);
+        if (!file.getParentFile().exists()) {
+          file.getParentFile().mkdirs();
+        }
+        if (!file.exists() && !file.createNewFile()) {
+          LOGGER.error("Can not create new file {}", file.getPath());
+        }
+        if (!new File(file.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).exists()
+            && !new File(file.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).createNewFile()) {
+          LOGGER.error("Can not create new file {}", file.getPath());
+        }
+      }
+    }
+    manager.getValidFiles(dataDir);
+    assertTrue(isEmpty(manager.getLastLocalFilesMap()));
+    senderLogger.startSyncTsFiles();
+    for (Set<File> newTsFiles : allFileList.values()) {
+      for (File file : newTsFiles) {
+        senderLogger.finishSyncTsfile(file);
+      }
+    }
+    senderLogger.close();
+
+    // recover log
+    senderLogAnalyzer.recover();
+    manager.getValidFiles(dataDir);
+    assertFalse(isEmpty(manager.getLastLocalFilesMap()));
+    Map<String, Set<File>> lastFilesMap = manager.getLastLocalFilesMap();
+    for (Entry<String, Set<File>> entry : allFileList.entrySet()) {
+      assertTrue(lastFilesMap.containsKey(entry.getKey()));
+      assertEquals(lastFilesMap.get(entry.getKey()).size(), entry.getValue().size());
+      assertTrue(lastFilesMap.get(entry.getKey()).containsAll(entry.getValue()));
+    }
+
+    // delete some files
+    assertFalse(new File(config.getSenderFolderPath(), SyncConstant.SYNC_LOG_NAME).exists());
+    senderLogger = new SyncSenderLogger(
+        new File(config.getSenderFolderPath(), SyncConstant.SYNC_LOG_NAME));
+    manager.getValidFiles(dataDir);
+    assertFalse(isEmpty(manager.getLastLocalFilesMap()));
+    senderLogger.startSyncDeletedFilesName();
+    for (Set<File> newTsFiles : allFileList.values()) {
+      for (File file : newTsFiles) {
+        senderLogger.finishSyncDeletedFileName(file);
+      }
+    }
+    senderLogger.close();
+    // recover log
+    senderLogAnalyzer.recover();
+    manager.getValidFiles(dataDir);
+    assertTrue(isEmpty(manager.getLastLocalFilesMap()));
+    assertTrue(isEmpty(manager.getDeletedFilesMap()));
+    Map<String, Set<File>> toBeSyncedFilesMap = manager.getToBeSyncedFilesMap();
+    for (Entry<String, Set<File>> entry : allFileList.entrySet()) {
+      assertTrue(toBeSyncedFilesMap.containsKey(entry.getKey()));
+      assertEquals(toBeSyncedFilesMap.get(entry.getKey()).size(), entry.getValue().size());
+      assertTrue(toBeSyncedFilesMap.get(entry.getKey()).containsAll(entry.getValue()));
+    }
+  }
+
+  private boolean isEmpty(Map<String, Set<File>> sendingFileList) {
+    for (Entry<String, Set<File>> entry : sendingFileList.entrySet()) {
+      if (!entry.getValue().isEmpty()) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+}
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/sender/recover/SyncSenderLoggerTest.java b/server/src/test/java/org/apache/iotdb/db/sync/sender/recover/SyncSenderLoggerTest.java
new file mode 100644
index 0000000..76c4137
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/sync/sender/recover/SyncSenderLoggerTest.java
@@ -0,0 +1,111 @@
+/**
+ * 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.sync.sender.recover;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.db.exception.StartupException;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.db.sync.conf.SyncSenderConfig;
+import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SyncSenderLoggerTest {
+
+  private SyncSenderLogger senderLogger;
+  private SyncSenderConfig config = SyncSenderDescriptor.getInstance().getConfig();
+  private String dataDir;
+
+  @Before
+  public void setUp()
+      throws IOException, InterruptedException, StartupException, DiskSpaceInsufficientException {
+    EnvironmentUtils.envSetUp();
+    dataDir = new File(DirectoryManager.getInstance().getNextFolderForSequenceFile())
+        .getParentFile().getAbsolutePath();
+    config.update(dataDir);
+  }
+
+  @After
+  public void tearDown() throws InterruptedException, IOException, StorageEngineException {
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void testSyncSenderLogger() throws IOException {
+    senderLogger = new SyncSenderLogger(
+        new File(config.getSenderFolderPath(), SyncConstant.SYNC_LOG_NAME));
+    Set<String> deletedFileNames = new HashSet<>();
+    Set<String> deletedFileNamesTest = new HashSet<>();
+    senderLogger.startSyncDeletedFilesName();
+    for (int i = 0; i < 100; i++) {
+      senderLogger
+          .finishSyncDeletedFileName(new File(config.getSenderFolderPath(), "deleted" + i));
+      deletedFileNames
+          .add(new File(config.getSenderFolderPath(), "deleted" + i).getAbsolutePath());
+    }
+    Set<String> toBeSyncedFiles = new HashSet<>();
+    Set<String> toBeSyncedFilesTest = new HashSet<>();
+    senderLogger.startSyncTsFiles();
+    for (int i = 0; i < 100; i++) {
+      senderLogger
+          .finishSyncTsfile(new File(config.getSenderFolderPath(), "new" + i));
+      toBeSyncedFiles
+          .add(new File(config.getSenderFolderPath(), "new" + i).getAbsolutePath());
+    }
+    senderLogger.close();
+    int count = 0;
+    int mode = 0;
+    try (BufferedReader br = new BufferedReader(
+        new FileReader(new File(config.getSenderFolderPath(), SyncConstant.SYNC_LOG_NAME)))) {
+      String line;
+      while ((line = br.readLine()) != null) {
+        count++;
+        if (line.equals(SyncSenderLogger.SYNC_DELETED_FILE_NAME_START)) {
+          mode = -1;
+        } else if (line.equals(SyncSenderLogger.SYNC_TSFILE_START)) {
+          mode = 1;
+        } else {
+          if (mode == -1) {
+            deletedFileNamesTest.add(line);
+          } else if (mode == 1) {
+            toBeSyncedFilesTest.add(line);
+          }
+        }
+      }
+    }
+    assertEquals(202, count);
+    assertEquals(deletedFileNames.size(), deletedFileNamesTest.size());
+    assertEquals(toBeSyncedFiles.size(), toBeSyncedFilesTest.size());
+    assertTrue(deletedFileNames.containsAll(deletedFileNamesTest));
+    assertTrue(toBeSyncedFiles.containsAll(toBeSyncedFilesTest));
+  }
+}
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/sender/transfer/DataTransferManagerTest.java b/server/src/test/java/org/apache/iotdb/db/sync/sender/transfer/DataTransferManagerTest.java
new file mode 100644
index 0000000..ec8472a
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/sync/sender/transfer/DataTransferManagerTest.java
@@ -0,0 +1,140 @@
+/**
+ * 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.sync.sender.transfer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.db.exception.StartupException;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.sync.conf.SyncSenderConfig;
+import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
+import org.apache.iotdb.db.sync.sender.recover.ISyncSenderLogAnalyzer;
+import org.apache.iotdb.db.sync.sender.recover.SyncSenderLogAnalyzer;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.db.utils.FilePathUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DataTransferManagerTest {
+
+  private static final Logger logger = LoggerFactory.getLogger(DataTransferManagerTest.class);
+  private IDataTransferManager manager = DataTransferManager.getInstance();
+  private SyncSenderConfig config = SyncSenderDescriptor.getInstance().getConfig();
+  private String dataDir;
+  private ISyncSenderLogAnalyzer senderLogAnalyzer;
+
+  @Before
+  public void setUp()
+      throws IOException, InterruptedException, StartupException, DiskSpaceInsufficientException {
+    EnvironmentUtils.envSetUp();
+    dataDir = new File(DirectoryManager.getInstance().getNextFolderForSequenceFile())
+        .getParentFile().getAbsolutePath();
+    config.update(dataDir);
+    senderLogAnalyzer = new SyncSenderLogAnalyzer(config.getSenderFolderPath());
+  }
+
+  @After
+  public void tearDown() throws InterruptedException, IOException, StorageEngineException {
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void makeFileSnapshot() throws IOException {
+    Map<String, Set<File>> allFileList = new HashMap<>();
+
+    Random r = new Random(0);
+    for (int i = 0; i < 3; i++) {
+      for (int j = 0; j < 5; j++) {
+        if (!allFileList.containsKey(String.valueOf(i))) {
+          allFileList.put(String.valueOf(i), new HashSet<>());
+        }
+        String rand = String.valueOf(r.nextInt(10000));
+        String fileName = FilePathUtils.regularizePath(dataDir) + IoTDBConstant.SEQUENCE_FLODER_NAME
+            + File.separator + i
+            + File.separator + rand;
+        File file = new File(fileName);
+        allFileList.get(String.valueOf(i)).add(file);
+        if (!file.getParentFile().exists()) {
+          file.getParentFile().mkdirs();
+        }
+        if (!file.exists() && !file.createNewFile()) {
+          logger.error("Can not create new file {}", file.getPath());
+        }
+        if (!new File(file.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).exists()
+            && !new File(file.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX).createNewFile()) {
+          logger.error("Can not create new file {}", file.getPath());
+        }
+      }
+    }
+
+    Map<String, Set<String>> dataFileMap = new HashMap<>();
+    File sequenceFile = new File(dataDir, IoTDBConstant.SEQUENCE_FLODER_NAME);
+    for (File sgFile : sequenceFile.listFiles()) {
+      dataFileMap.putIfAbsent(sgFile.getName(), new HashSet<>());
+      for (File tsfile : sgFile.listFiles()) {
+        if (!tsfile.getName().endsWith(TsFileResource.RESOURCE_SUFFIX)) {
+          ((DataTransferManager)manager).makeFileSnapshot(tsfile);
+        }
+        dataFileMap.get(sgFile.getName()).add(tsfile.getName());
+      }
+    }
+
+    assertTrue(new File(config.getSenderFolderPath()).exists());
+    assertTrue(new File(config.getSnapshotPath()).exists());
+
+    Map<String, Set<String>> snapFileMap = new HashMap<>();
+    for (File sgFile : new File(config.getSnapshotPath()).listFiles()) {
+      snapFileMap.putIfAbsent(sgFile.getName(), new HashSet<>());
+      for (File snapshotTsfile : sgFile.listFiles()) {
+        snapFileMap.get(sgFile.getName()).add(snapshotTsfile.getName());
+      }
+    }
+
+    assertEquals(dataFileMap.size(), snapFileMap.size());
+    for (Entry<String, Set<String>> entry : dataFileMap.entrySet()) {
+      String sg = entry.getKey();
+      Set<String> tsfiles = entry.getValue();
+      assertTrue(snapFileMap.containsKey(sg));
+      assertEquals(snapFileMap.get(sg).size(), tsfiles.size());
+      assertTrue(snapFileMap.get(sg).containsAll(tsfiles));
+    }
+
+    assertFalse(new File(config.getLastFileInfoPath()).exists());
+    senderLogAnalyzer.recover();
+    assertFalse(new File(config.getSnapshotPath()).exists());
+    assertTrue(new File(config.getLastFileInfoPath()).exists());
+  }
+}
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/test/SyncTestClient1.java b/server/src/test/java/org/apache/iotdb/db/sync/test/SyncTestClient1.java
deleted file mode 100644
index c543dc1..0000000
--- a/server/src/test/java/org/apache/iotdb/db/sync/test/SyncTestClient1.java
+++ /dev/null
@@ -1,253 +0,0 @@
-/*
- * 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.sync.test;
-
-import static org.apache.iotdb.db.sync.test.RandomNum.getRandomInt;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * SyncTestClient1 is used to generate data of whole timeseries (simulating jilian scene) to test stability of
- * sync function.
- */
-public class SyncTestClient1 {
-
-  private static final int TIME_INTERVAL = 0;
-  private static final int TOTAL_DATA = 2000000;
-  private static final int ABNORMAL_MAX_INT = 0;
-  private static final int ABNORMAL_MIN_INT = -10;
-  private static final int ABNORMAL_MAX_FLOAT = 0;
-  private static final int ABNORMAL_MIN_FLOAT = -10;
-  private static final int ABNORMAL_FREQUENCY = Integer.MAX_VALUE;
-  private static final int ABNORMAL_LENGTH = 0;
-  private static final int MIN_INT = 0;
-  private static final int MAX_INT = 14;
-  private static final int MIN_FLOAT = 20;
-  private static final int MAX_FLOAT = 30;
-  private static final int STRING_LENGTH = 5;
-  private static final int BATCH_SQL = 10000;
-  private static final Logger logger = LoggerFactory.getLogger(SyncTestClient1.class);
-
-  /**
-   * generate time series map from file.
-   *
-   * @param inputFilePath input file path
-   * @return Map
-   * @throws Exception Exception
-   */
-  public static Map<String, String> generateTimeseriesMapFromFile(String inputFilePath)
-      throws IOException {
-
-    Map<String, String> timeseriesMap = new HashMap<>();
-
-    File file = new File(inputFilePath);
-    try (BufferedReader reader = new BufferedReader(new FileReader(file))) {
-      String line;
-      while ((line = reader.readLine()) != null) {
-
-        String timeseries = line.split(" ")[2];
-        String dataType = line.split("DATATYPE = ")[1].split(",")[0].trim();
-        String encodingType = line.split("ENCODING = ")[1].split(";")[0].trim();
-        timeseriesMap.put(timeseries, dataType + "," + encodingType);
-      }
-    }
-    return timeseriesMap;
-
-  }
-
-  /**
-   * create time series.
-   *
-   * @param statement statement
-   * @param timeseriesMap time series map
-   */
-  public static void createTimeseries(Statement statement, Map<String, String> timeseriesMap) {
-
-    try {
-      String createTimeseriesSql = "CREATE TIMESERIES <timeseries> WITH DATATYPE=<datatype>, "
-          + "ENCODING=<encode>";
-
-      int sqlCount = 0;
-      for (Map.Entry<String, String> entry : timeseriesMap.entrySet()) {
-        String key = entry.getKey();
-        String properties = entry.getValue();
-        String sql = createTimeseriesSql.replace("<timeseries>", key)
-            .replace("<datatype>", Utils.getType(properties))
-            .replace("<encode>", Utils.getEncode(properties));
-        statement.addBatch(sql);
-        sqlCount++;
-        if (sqlCount >= BATCH_SQL) {
-          statement.executeBatch();
-          statement.clearBatch();
-          sqlCount = 0;
-        }
-      }
-      statement.executeBatch();
-      statement.clearBatch();
-    } catch (Exception e) {
-      logger.error("", e);
-    }
-  }
-
-  /**
-   * set storage group.
-   *
-   * @param statement statement
-   * @param storageGroupList storage group list
-   * @throws SQLException SQLException
-   */
-  public static void setStorageGroup(Statement statement, List<String> storageGroupList)
-      throws SQLException {
-    try {
-      String setStorageGroupSql = "SET STORAGE GROUP TO <prefixpath>";
-      for (String str : storageGroupList) {
-        String sql = setStorageGroupSql.replace("<prefixpath>", str);
-        statement.execute(sql);
-      }
-    } catch (Exception e) {
-      logger.error("", e);
-    }
-  }
-
-  /**
-   * random insert data.
-   *
-   * @param statement statement
-   * @param timeseriesMap time series map
-   * @throws Exception Exception
-   */
-  public static void randomInsertData(Statement statement, Map<String, String> timeseriesMap)
-      throws SQLException, InterruptedException {
-    String insertDataSql = "INSERT INTO %s (timestamp, %s) VALUES (%s, %s)";
-    int abnormalCount = 0;
-    int abnormalFlag = 1;
-    int sqlCount = 0;
-
-    for (int i = 0; i < TOTAL_DATA; i++) {
-
-      long time = System.currentTimeMillis();
-
-      if (i % ABNORMAL_FREQUENCY == 250) {
-        abnormalFlag = 0;
-      }
-
-      for (Map.Entry<String, String> entry : timeseriesMap.entrySet()) {
-        String key = entry.getKey();
-        String type = Utils.getType(entry.getValue());
-        String path = Utils.getPath(key);
-        String sensor = Utils.getSensor(key);
-        String sql = "";
-
-        if (type.equals("INT32")) {
-          int value;
-          if (abnormalFlag == 0) {
-            value = getRandomInt(ABNORMAL_MIN_INT, ABNORMAL_MAX_INT);
-          } else {
-            value = getRandomInt(MIN_INT, MAX_INT);
-          }
-          sql = String.format(insertDataSql, path, sensor, time, value);
-        } else if (type.equals("FLOAT")) {
-          float value;
-          if (abnormalFlag == 0) {
-            value = RandomNum.getRandomFloat(ABNORMAL_MIN_FLOAT, ABNORMAL_MAX_FLOAT);
-          } else {
-            value = RandomNum.getRandomFloat(MIN_FLOAT, MAX_FLOAT);
-          }
-          sql = String.format(insertDataSql, path, sensor, time, value);
-        } else if (type.equals("TEXT")) {
-          String value;
-          value = RandomNum.getRandomText(STRING_LENGTH);
-          sql = String.format(insertDataSql, path, sensor, time, "\"" + value + "\"");
-        }
-
-        statement.addBatch(sql);
-        sqlCount++;
-        if (sqlCount >= BATCH_SQL) {
-          statement.executeBatch();
-          statement.clearBatch();
-          sqlCount = 0;
-        }
-      }
-
-      if (abnormalFlag == 0) {
-        abnormalCount += 1;
-      }
-      if (abnormalCount >= ABNORMAL_LENGTH) {
-        abnormalCount = 0;
-        abnormalFlag = 1;
-      }
-    }
-    statement.executeBatch();
-    statement.clearBatch();
-  }
-
-  /**
-   * main function.
-   *
-   * @param args arguments
-   * @throws Exception Exception
-   */
-  public static void main(String[] args) throws Exception {
-
-    String path =
-        new File(System.getProperty(IoTDBConstant.IOTDB_HOME, null)).getParent() + File.separator
-            + "src"
-            + File.separator + "test" + File.separator + "resources" + File.separator
-            + "CreateTimeseries1.txt";
-    Map<String, String> timeseriesMap = generateTimeseriesMapFromFile(path);
-
-    List<String> storageGroupList = new ArrayList<>();
-    storageGroupList.add("root.vehicle_history");
-    storageGroupList.add("root.vehicle_alarm");
-    storageGroupList.add("root.vehicle_temp");
-    storageGroupList.add("root.range_event");
-
-    try (Connection connection = DriverManager
-        .getConnection("jdbc:iotdb://localhost:6667/", "root", "root");
-        Statement statement = connection.createStatement();) {
-      Class.forName("org.apache.iotdb.jdbc.IoTDBDriver");
-
-
-      setStorageGroup(statement, storageGroupList);
-      logger.debug("Finish set storage group.");
-      createTimeseries(statement, timeseriesMap);
-      logger.debug("Finish create timeseries.");
-      while (true) {
-        randomInsertData(statement, timeseriesMap);
-      }
-
-    } catch (Exception e) {
-      logger.error("", e);
-    }
-  }
-}
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/test/SyncTestClient2.java b/server/src/test/java/org/apache/iotdb/db/sync/test/SyncTestClient2.java
deleted file mode 100644
index 29e8bb6..0000000
--- a/server/src/test/java/org/apache/iotdb/db/sync/test/SyncTestClient2.java
+++ /dev/null
@@ -1,262 +0,0 @@
-/*
- * 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.sync.test;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * SyncTestClient2 is used to generate data of half timeseries (simulating jilian scene) to test stability of
- * sync function.
- */
-public class SyncTestClient2 {
-
-  private static final int TIME_INTERVAL = 0;
-  private static final int TOTAL_DATA = 2000000;
-  private static final int ABNORMAL_MAX_INT = 0;
-  private static final int ABNORMAL_MIN_INT = -10;
-  private static final int ABNORMAL_MAX_FLOAT = 0;
-  private static final int ABNORMAL_MIN_FLOAT = -10;
-  private static final int ABNORMAL_FREQUENCY = Integer.MAX_VALUE;
-  private static final int ABNORMAL_LENGTH = 0;
-  private static final int MIN_INT = 0;
-  private static final int MAX_INT = 14;
-  private static final int MIN_FLOAT = 20;
-  private static final int MAX_FLOAT = 30;
-  private static final int STRING_LENGTH = 5;
-  private static final int BATCH_SQL = 10000;
-  private static final Logger logger = LoggerFactory.getLogger(SyncTestClient2.class);
-
-  /**
-   * generate time series map from file.
-   *
-   * @param inputFilePath input file path
-   * @return map
-   * @throws Exception Exception
-   */
-  public static Map<String, String> generateTimeseriesMapFromFile(String inputFilePath)
-      throws IOException {
-
-    Map<String, String> timeseriesMap = new HashMap<>();
-
-    File file = new File(inputFilePath);
-    try (BufferedReader reader = new BufferedReader(new FileReader(file))) {
-      String line;
-      while ((line = reader.readLine()) != null) {
-
-        String timeseries = line.split(" ")[2];
-        String dataType = line.split("DATATYPE = ")[1].split(",")[0].trim();
-        String encodingType = line.split("ENCODING = ")[1].split(";")[0].trim();
-        timeseriesMap.put(timeseries, dataType + "," + encodingType);
-      }
-    }
-
-    return timeseriesMap;
-
-  }
-
-  /**
-   * create time series.
-   *
-   * @param statement statement
-   * @param timeseriesMap time series map
-   * @throws SQLException SQLException
-   */
-  public static void createTimeseries(Statement statement, Map<String, String> timeseriesMap)
-      throws SQLException {
-
-    try {
-      String createTimeseriesSql = "CREATE TIMESERIES <timeseries> WITH DATATYPE=<datatype>, "
-          + "ENCODING=<encode>";
-
-      int sqlCount = 0;
-
-      for (Map.Entry<String, String> entry : timeseriesMap.entrySet()) {
-        String key = entry.getKey();
-        String properties = entry.getValue();
-        String sql = createTimeseriesSql.replace("<timeseries>", key)
-            .replace("<datatype>", Utils.getType(properties))
-            .replace("<encode>", Utils.getEncode(properties));
-
-        statement.addBatch(sql);
-        sqlCount++;
-        if (sqlCount >= BATCH_SQL) {
-          statement.executeBatch();
-          statement.clearBatch();
-          sqlCount = 0;
-        }
-      }
-      statement.executeBatch();
-      statement.clearBatch();
-    } catch (Exception e) {
-      logger.error("", e);
-    }
-  }
-
-  /**
-   * set storage group.
-   *
-   * @param statement statement
-   * @param storageGroupList storage group list
-   * @throws SQLException SQLException
-   */
-  public static void setStorageGroup(Statement statement, List<String> storageGroupList)
-      throws SQLException {
-
-    try {
-      String setStorageGroupSql = "SET STORAGE GROUP TO <prefixpath>";
-      for (String str : storageGroupList) {
-        String sql = setStorageGroupSql.replace("<prefixpath>", str);
-        statement.execute(sql);
-      }
-    } catch (Exception e) {
-      logger.error("", e);
-    }
-  }
-
-  /**
-   * randomly insert data.
-   *
-   * @param statement statement
-   * @param timeseriesMap time series map
-   */
-  public static void randomInsertData(Statement statement, Map<String, String> timeseriesMap)
-      throws SQLException, InterruptedException {
-    String insertDataSql = "INSERT INTO %s (timestamp, %s) VALUES (%s, %s)";
-    int abnormalCount = 0;
-    int abnormalFlag = 1;
-
-    int sqlCount = 0;
-
-    for (int i = 0; i < TOTAL_DATA; i++) {
-
-      long time = System.currentTimeMillis();
-
-      if (i % ABNORMAL_FREQUENCY == 250) {
-        abnormalFlag = 0;
-      }
-
-      for (Map.Entry<String, String> entry : timeseriesMap.entrySet()) {
-        String key = entry.getKey();
-        String type = Utils.getType(entry.getValue());
-        String path = Utils.getPath(key);
-        String sensor = Utils.getSensor(key);
-        String sql = "";
-
-        if (type.equals("INT32")) {
-          int value;
-          if (abnormalFlag == 0) {
-            value = RandomNum.getRandomInt(ABNORMAL_MIN_INT, ABNORMAL_MAX_INT);
-          } else {
-            value = RandomNum.getRandomInt(MIN_INT, MAX_INT);
-          }
-          sql = String.format(insertDataSql, path, sensor, time, value);
-        } else if (type.equals("FLOAT")) {
-          float value;
-          if (abnormalFlag == 0) {
-            value = RandomNum.getRandomFloat(ABNORMAL_MIN_FLOAT, ABNORMAL_MAX_FLOAT);
-          } else {
-            value = RandomNum.getRandomFloat(MIN_FLOAT, MAX_FLOAT);
-          }
-          sql = String.format(insertDataSql, path, sensor, time,value);
-        } else if (type.equals("TEXT")) {
-          String value;
-          value = RandomNum.getRandomText(STRING_LENGTH);
-          sql = String.format(insertDataSql, path, sensor, time, "\"" + value + "\"");
-        }
-
-        statement.addBatch(sql);
-        sqlCount++;
-        if (sqlCount >= BATCH_SQL) {
-          statement.executeBatch();
-          statement.clearBatch();
-          sqlCount = 0;
-        }
-      }
-
-      if (abnormalFlag == 0) {
-        abnormalCount += 1;
-      }
-      if (abnormalCount >= ABNORMAL_LENGTH) {
-        abnormalCount = 0;
-        abnormalFlag = 1;
-      }
-    }
-    statement.executeBatch();
-    statement.clearBatch();
-  }
-
-  /**
-   * main function.
-   *
-   * @param args arguments
-   * @throws Exception Exception
-   */
-  public static void main(String[] args) throws Exception {
-
-    Statement statement = null;
-
-    String path =
-        new File(System.getProperty(IoTDBConstant.IOTDB_HOME, null)).getParent() + File.separator
-            + "src"
-            + File.separator + "test" + File.separator + "resources" + File.separator
-            + "CreateTimeseries2.txt";
-    Map<String, String> timeseriesMap = generateTimeseriesMapFromFile(path);
-
-    List<String> storageGroupList = new ArrayList<>();
-    storageGroupList.add("root.vehicle_history1");
-    storageGroupList.add("root.vehicle_alarm1");
-    storageGroupList.add("root.vehicle_temp1");
-    storageGroupList.add("root.range_event1");
-
-    try (Connection connection = DriverManager
-        .getConnection("jdbc:iotdb://localhost:6667/", "root", "root")) {
-      Class.forName("org.apache.iotdb.jdbc.IoTDBDriver");
-      statement = connection.createStatement();
-
-      setStorageGroup(statement, storageGroupList);
-      logger.debug("Finish set storage group.");
-      createTimeseries(statement, timeseriesMap);
-      logger.debug("Finish create timeseries.");
-      while (true) {
-        randomInsertData(statement, timeseriesMap);
-      }
-
-    } catch (Exception e) {
-      logger.error("", e);
-    } finally {
-      if (statement != null) {
-        statement.close();
-      }
-    }
-  }
-}
\ No newline at end of file
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/test/SyncTestClient3.java b/server/src/test/java/org/apache/iotdb/db/sync/test/SyncTestClient3.java
deleted file mode 100644
index 6e1da5c..0000000
--- a/server/src/test/java/org/apache/iotdb/db/sync/test/SyncTestClient3.java
+++ /dev/null
@@ -1,282 +0,0 @@
-/*
- * 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.sync.test;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.iotdb.db.conf.IoTDBConstant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * SyncTestClient3 is used to generate data of another half timeseries (simulating jilian scene) which is
- * different to those in SyncTestClient2 to test stability of sync function.
- */
-public class SyncTestClient3 {
-
-  private static final int TIME_INTERVAL = 0;
-  private static final int TOTAL_DATA = 2000000;
-  private static final int ABNORMAL_MAX_INT = 0;
-  private static final int ABNORMAL_MIN_INT = -10;
-  private static final int ABNORMAL_MAX_FLOAT = 0;
-  private static final int ABNORMAL_MIN_FLOAT = -10;
-  private static final int ABNORMAL_FREQUENCY = Integer.MAX_VALUE;
-  private static final int ABNORMAL_LENGTH = 0;
-  private static final int MIN_INT = 0;
-  private static final int MAX_INT = 14;
-  private static final int MIN_FLOAT = 20;
-  private static final int MAX_FLOAT = 30;
-  private static final int STRING_LENGTH = 5;
-  private static final int BATCH_SQL = 10000;
-  private static final Logger logger = LoggerFactory.getLogger(SyncTestClient3.class);
-
-  /**
-   * generate time series map from file.
-   *
-   * @param inputFilePath input file path
-   * @return map
-   * @throws Exception Exception
-   */
-  public static Map<String, String> generateTimeseriesMapFromFile(String inputFilePath)
-      throws IOException {
-
-    Map<String, String> timeseriesMap = new HashMap<>();
-
-    File file = new File(inputFilePath);
-    try (BufferedReader reader = new BufferedReader(new FileReader(file))) {
-      String line;
-      while ((line = reader.readLine()) != null) {
-        String timeseries = line.split(" ")[2];
-        String dataType = line.split("DATATYPE = ")[1].split(",")[0].trim();
-        String encodingType = line.split("ENCODING = ")[1].split(";")[0].trim();
-        timeseriesMap.put(timeseries, dataType + "," + encodingType);
-      }
-    }
-
-    return timeseriesMap;
-
-  }
-
-  /**
-   * create time series.
-   *
-   * @param statement statement
-   * @param timeseriesMap time series map
-   * @throws SQLException SQLException
-   */
-  public static void createTimeseries(Statement statement, Statement statement1,
-      Map<String, String> timeseriesMap)
-      throws SQLException {
-
-    try {
-      String createTimeseriesSql = "CREATE TIMESERIES <timeseries> WITH DATATYPE=<datatype>, "
-          + "ENCODING=<encode>";
-
-      int sqlCount = 0;
-
-      for (Map.Entry<String, String> entry : timeseriesMap.entrySet()) {
-        String key = entry.getKey();
-        String properties = entry.getValue();
-        String sql = createTimeseriesSql.replace("<timeseries>", key)
-            .replace("<datatype>", Utils.getType(properties))
-            .replace("<encode>", Utils.getEncode(properties));
-
-        statement.addBatch(sql);
-        statement1.addBatch(sql);
-        sqlCount++;
-        if (sqlCount >= BATCH_SQL) {
-          statement.executeBatch();
-          statement.clearBatch();
-          statement1.executeBatch();
-          statement1.clearBatch();
-          sqlCount = 0;
-        }
-      }
-      statement.executeBatch();
-      statement.clearBatch();
-      statement1.executeBatch();
-      statement1.clearBatch();
-    } catch (Exception e) {
-      logger.error("", e);
-    }
-  }
-
-  /**
-   * set storage group.
-   *
-   * @param statement statement
-   * @param storageGroupList storage group list
-   * @throws SQLException SQLException
-   */
-  public static void setStorageGroup(Statement statement, Statement statement1,
-      List<String> storageGroupList)
-      throws SQLException {
-    try {
-      String setStorageGroupSql = "SET STORAGE GROUP TO <prefixpath>";
-      for (String str : storageGroupList) {
-        String sql = setStorageGroupSql.replace("<prefixpath>", str);
-        statement.execute(sql);
-        statement1.execute(sql);
-      }
-    } catch (Exception e) {
-      logger.error("", e);
-    }
-  }
-
-  /**
-   * randomly insert data.
-   *
-   * @param statement statement
-   * @param timeseriesMap time series map
-   * @throws Exception Exception
-   */
-  public static void randomInsertData(Statement statement, Statement statement1,
-      Map<String, String> timeseriesMap) throws InterruptedException, SQLException {
-    String insertDataSql = "INSERT INTO %s (timestamp, %s) VALUES (%s, %s)";
-    int abnormalCount = 0;
-    int abnormalFlag = 1;
-
-    int sqlCount = 0;
-
-    for (int i = 0; i < TOTAL_DATA; i++) {
-
-      long time = System.currentTimeMillis();
-
-      if (i % ABNORMAL_FREQUENCY == 250) {
-        abnormalFlag = 0;
-      }
-
-      for (Map.Entry<String, String> entry : timeseriesMap.entrySet()) {
-        String key = entry.getKey();
-        String type = Utils.getType(entry.getValue());
-        String path = Utils.getPath(key);
-        String sensor = Utils.getSensor(key);
-        String sql = "";
-
-        if (type.equals("INT32")) {
-          int value;
-          if (abnormalFlag == 0) {
-            value = RandomNum.getRandomInt(ABNORMAL_MIN_INT, ABNORMAL_MAX_INT);
-          } else {
-            value = RandomNum.getRandomInt(MIN_INT, MAX_INT);
-          }
-          sql = String.format(insertDataSql, path, sensor, time, value);
-        } else if (type.equals("FLOAT")) {
-          float value;
-          if (abnormalFlag == 0) {
-            value = RandomNum.getRandomFloat(ABNORMAL_MIN_FLOAT, ABNORMAL_MAX_FLOAT);
-          } else {
-            value = RandomNum.getRandomFloat(MIN_FLOAT, MAX_FLOAT);
-          }
-          sql = String.format(insertDataSql, path, sensor, time, value);
-        } else if (type.equals("TEXT")) {
-          String value;
-          value = RandomNum.getRandomText(STRING_LENGTH);
-          sql = String.format(insertDataSql, path, sensor, time, "\"" + value + "\"");
-        }
-
-        statement.addBatch(sql);
-        statement1.addBatch(sql);
-        sqlCount++;
-        if (sqlCount >= BATCH_SQL) {
-          statement.executeBatch();
-          statement.clearBatch();
-          statement1.executeBatch();
-          statement1.clearBatch();
-          sqlCount = 0;
-        }
-      }
-
-      if (abnormalFlag == 0) {
-        abnormalCount += 1;
-      }
-      if (abnormalCount >= ABNORMAL_LENGTH) {
-        abnormalCount = 0;
-        abnormalFlag = 1;
-      }
-    }
-    statement.executeBatch();
-    statement.clearBatch();
-    statement1.executeBatch();
-    statement1.clearBatch();
-  }
-
-  /**
-   * main function.
-   *
-   * @param args arguments
-   * @throws Exception Exception
-   */
-  public static void main(String[] args) throws Exception {
-
-    Statement statement = null;
-    Statement statement1 = null;
-
-    String path =
-        new File(System.getProperty(IoTDBConstant.IOTDB_HOME, null)).getParent() + File.separator
-            + "src"
-            + File.separator + "test" + File.separator + "resources" + File.separator
-            + "CreateTimeseries3.txt";
-    Map<String, String> timeseriesMap = generateTimeseriesMapFromFile(path);
-
-    List<String> storageGroupList = new ArrayList<>();
-    storageGroupList.add("root.vehicle_history2");
-    storageGroupList.add("root.vehicle_alarm2");
-    storageGroupList.add("root.vehicle_temp2");
-    storageGroupList.add("root.range_event2");
-
-    try (Connection connection1 = DriverManager
-        .getConnection("jdbc:iotdb://192.168.130.17:6667/", "root", "root")) {
... 91 lines suppressed ...