You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2022/04/13 07:17:48 UTC

[iotdb] branch master updated: New sync framework with TsFileSync (#5353)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new f60c584f4d New sync framework with TsFileSync (#5353)
f60c584f4d is described below

commit f60c584f4dd34c508b56a29f498584e8729294a5
Author: yschengzi <87...@users.noreply.github.com>
AuthorDate: Wed Apr 13 15:17:43 2022 +0800

    New sync framework with TsFileSync (#5353)
---
 .../org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4   |  67 +-
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4  |  24 +
 .../org/apache/iotdb/cluster/ClusterIoTDB.java     |   1 -
 .../apache/iotdb/cluster/utils/StatusUtils.java    |   6 +-
 docs/UserGuide/Maintenance-Tools/Sync-Tool.md      | 396 +++++++---
 docs/zh/UserGuide/Maintenance-Tools/Sync-Tool.md   | 493 ++++++++++---
 .../sync/IoTDBSyncReceiverCollectorIT.java         | 513 +++++++++++++
 .../db/integration/sync/IoTDBSyncReceiverIT.java   | 357 +++++++++
 .../sync/IoTDBSyncReceiverLoaderIT.java            | 210 ++++++
 .../db/integration/sync/IoTDBSyncSenderIT.java     | 304 ++++++++
 .../iotdb/db/integration/sync/SyncTestUtil.java    | 203 ++++++
 .../db/integration/sync/TransportClientMock.java   |  69 ++
 .../src/test/resources/iotdb-engine.properties     |   3 +-
 integration/src/test/resources/logback-test.xml    |   2 +-
 .../iotdb/commons/concurrent/ThreadName.java       |   5 +-
 .../apache/iotdb/commons/conf/IoTDBConstant.java   |  14 +
 .../apache/iotdb/commons/service/ServiceType.java  |   2 +
 .../apache/iotdb/commons/utils/StatusUtils.java    |   3 -
 .../resources/conf/iotdb-engine.properties         |  19 +-
 .../resources/conf/iotdb-sync-client.properties    |  35 -
 .../assembly/resources/tools/start-sync-client.bat |  71 --
 .../assembly/resources/tools/start-sync-client.sh  |  54 --
 .../assembly/resources/tools/stop-sync-client.bat  |  23 -
 .../assembly/resources/tools/stop-sync-client.sh   |  30 -
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |  38 +-
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |  35 +-
 .../org/apache/iotdb/db/engine/StorageEngine.java  |  12 -
 .../iotdb/db/engine/modification/Deletion.java     |  23 +
 .../iotdb/db/engine/storagegroup/DataRegion.java   |  63 +-
 .../db/engine/storagegroup/TsFileManager.java      |  38 +
 .../db/engine/storagegroup/TsFileProcessor.java    |  23 +
 .../dataregion/StorageGroupManager.java            |  10 +
 .../sync/PipeDataLoadBearableException.java}       |  10 +-
 .../sync/PipeDataLoadException.java}               |  10 +-
 .../sync/PipeDataLoadUnbearableException.java}     |  10 +-
 .../sync/PipeException.java}                       |  21 +-
 .../sync/PipeServerException.java}                 |  21 +-
 .../PipeSinkException.java}                        |  24 +-
 .../apache/iotdb/db/metadata/LocalConfigNode.java  |  11 +
 .../iotdb/db/metadata/mtree/MTreeBelowSG.java      |   1 +
 .../traverser/collector/MeasurementCollector.java  |   4 +
 .../db/metadata/schemaregion/SchemaRegion.java     |  20 +
 .../apache/iotdb/db/qp/constant/SQLConstant.java   |  26 +
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  | 189 ++++-
 .../org/apache/iotdb/db/qp/logical/Operator.java   |  14 +-
 .../db/qp/logical/sys/CreatePipeOperator.java      |  69 ++
 .../db/qp/logical/sys/CreatePipeSinkOperator.java  |  61 ++
 .../logical/sys/DropPipeOperator.java}             |  32 +-
 .../logical/sys/DropPipeSinkOperator.java}         |  33 +-
 .../logical/sys/ShowPipeOperator.java}             |  33 +-
 .../logical/sys/ShowPipeServerOperator.java}       |  26 +-
 .../logical/sys/ShowPipeSinkOperator.java}         |  33 +-
 .../logical/sys/ShowPipeSinkTypeOperator.java}     |  27 +-
 .../logical/sys/StartPipeOperator.java}            |  32 +-
 .../logical/sys/StartPipeServerOperator.java}      |  28 +-
 .../logical/sys/StopPipeOperator.java}             |  32 +-
 .../logical/sys/StopPipeServerOperator.java}       |  28 +-
 .../apache/iotdb/db/qp/physical/PhysicalPlan.java  |  10 +
 .../physical/sys/CreateAlignedTimeSeriesPlan.java  |  15 +-
 .../iotdb/db/qp/physical/sys/CreatePipePlan.java   | 110 +++
 .../db/qp/physical/sys/CreatePipeSinkPlan.java     |  98 +++
 .../db/qp/physical/sys/CreateTimeSeriesPlan.java   |  10 +
 .../physical/sys/DropPipeSinkPlan.java}            |  28 +-
 .../physical/sys/OperatePipePlan.java}             |  28 +-
 .../physical/sys/ShowPipePlan.java}                |  22 +-
 .../physical/sys/ShowPipeServerPlan.java}          |  11 +-
 .../physical/sys/ShowPipeSinkPlan.java}            |  23 +-
 .../physical/sys/ShowPipeSinkTypePlan.java}        |  11 +-
 .../apache/iotdb/db/qp/physical/sys/ShowPlan.java  |   6 +-
 .../db/qp/physical/sys/StartPipeServerPlan.java    |  56 ++
 .../db/qp/physical/sys/StopPipeServerPlan.java     |  56 ++
 .../apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java    | 190 ++++-
 .../apache/iotdb/db/qp/utils/DatetimeUtils.java    |  15 +
 .../java/org/apache/iotdb/db/service/DataNode.java |   2 -
 .../java/org/apache/iotdb/db/service/IoTDB.java    |   7 +-
 .../apache/iotdb/db/sync/conf/SyncConstant.java    |  85 +--
 .../apache/iotdb/db/sync/conf/SyncPathUtil.java    | 100 +++
 .../iotdb/db/sync/conf/SyncSenderConfig.java       | 127 ----
 .../iotdb/db/sync/conf/SyncSenderDescriptor.java   | 124 ----
 .../org/apache/iotdb/db/sync/package-info.java     |  38 -
 .../iotdb/db/sync/pipedata/DeletionPipeData.java   |  85 +++
 .../apache/iotdb/db/sync/pipedata/PipeData.java    |  96 +++
 .../iotdb/db/sync/pipedata/SchemaPipeData.java     |  98 +++
 .../iotdb/db/sync/pipedata/TsFilePipeData.java     | 177 +++++
 .../sync/pipedata/queue/BufferedPipeDataQueue.java | 438 +++++++++++
 .../queue/PipeDataQueue.java}                      |  21 +-
 .../sync/pipedata/queue/PipeDataQueueFactory.java  |  57 ++
 .../iotdb/db/sync/receiver/ReceiverService.java    | 278 +++++++
 .../db/sync/receiver/collector/Collector.java      | 170 +++++
 .../db/sync/receiver/load/DeletionLoader.java      |  48 ++
 .../iotdb/db/sync/receiver/load/FileLoader.java    | 203 ------
 .../db/sync/receiver/load/FileLoaderManager.java   | 213 ------
 .../iotdb/db/sync/receiver/load/IFileLoader.java   |  50 --
 .../iotdb/db/sync/receiver/load/ILoadLogger.java   |  57 --
 .../receiver/load/{LoadType.java => ILoader.java}  |  12 +-
 .../iotdb/db/sync/receiver/load/LoadLogger.java    |  72 --
 .../iotdb/db/sync/receiver/load/SchemaLoader.java  |  60 ++
 .../iotdb/db/sync/receiver/load/TsFileLoader.java  |  67 ++
 .../iotdb/db/sync/receiver/manager/PipeInfo.java   |  85 +++
 .../db/sync/receiver/manager/PipeMessage.java      |  76 ++
 .../db/sync/receiver/manager/ReceiverManager.java  | 216 ++++++
 .../sync/receiver/recover/ISyncReceiverLogger.java |  50 --
 .../receiver/recover/SyncReceiverLogAnalyzer.java  | 154 ----
 .../sync/receiver/recover/SyncReceiverLogger.java  |  72 --
 .../db/sync/receiver/recovery/ReceiverLog.java     | 127 ++++
 .../receiver/recovery/ReceiverLogAnalyzer.java     | 157 ++++
 .../db/sync/receiver/transfer/SyncServiceImpl.java | 370 ----------
 .../db/sync/sender/manage/ISyncFileManager.java    |  72 --
 .../db/sync/sender/manage/SyncFileManager.java     | 291 --------
 .../db/sync/sender/manager/SchemaSyncManager.java  | 163 +++++
 .../db/sync/sender/manager/TsFileSyncManager.java  | 118 +++
 .../iotdb/db/sync/sender/pipe/IoTDBPipeSink.java   |  97 +++
 .../org/apache/iotdb/db/sync/sender/pipe/Pipe.java | 106 +++
 .../pipe/PipeSink.java}                            |  30 +-
 .../iotdb/db/sync/sender/pipe/TsFilePipe.java      | 334 +++++++++
 .../sender/recover/ISyncSenderLogAnalyzer.java     |  47 --
 .../db/sync/sender/recover/ISyncSenderLogger.java  |  67 --
 .../sync/sender/recover/SyncSenderLogAnalyzer.java | 128 ----
 .../db/sync/sender/recover/SyncSenderLogger.java   |  72 --
 .../db/sync/sender/recovery/SenderLogAnalyzer.java | 172 +++++
 .../db/sync/sender/recovery/SenderLogger.java      | 141 ++++
 .../db/sync/sender/recovery/TsFilePipeLogger.java  | 150 ++++
 .../db/sync/sender/service/SenderService.java      | 417 +++++++++++
 .../db/sync/sender/service/TransportHandler.java   | 127 ++++
 .../iotdb/db/sync/sender/transfer/ISyncClient.java |  95 ---
 .../iotdb/db/sync/sender/transfer/SyncClient.java  | 810 ---------------------
 .../client/ITransportClient.java}                  |  20 +-
 .../db/sync/transport/client/TransportClient.java  | 527 ++++++++++++++
 .../conf/TransportConfig.java}                     |  26 +-
 .../conf/TransportConstant.java}                   |  21 +-
 .../server/TransportServerManager.java}            |  80 +-
 .../server/TransportServerManagerMBean.java}       |   6 +-
 .../server/TransportServerThriftHandler.java}      |  30 +-
 .../transport/server/TransportServiceImpl.java     | 385 ++++++++++
 .../apache/iotdb/db/utils/EnvironmentUtils.java    |   8 +
 .../org/apache/iotdb/db/utils/FileLoaderUtils.java |   2 +-
 .../java/org/apache/iotdb/db/utils/SyncUtils.java  | 109 ---
 .../iotdb/db/qp/physical/PhysicalPlanTest.java     |  28 +
 .../sync/pipedata/BufferedPipeDataQueueTest.java   | 542 ++++++++++++++
 .../iotdb/db/sync/pipedata/PipeDataTest.java       |  86 +++
 .../db/sync/receiver/load/FileLoaderTest.java      | 405 -----------
 .../sync/receiver/manager/ReceiverManagerTest.java |  98 +++
 .../recover/SyncReceiverLogAnalyzerTest.java       | 229 ------
 .../receiver/recover/SyncReceiverLoggerTest.java   | 115 ---
 .../receiver/recovery/ReceiverLogAnalyzerTest.java | 124 ++++
 .../db/sync/sender/manage/SyncFileManagerTest.java | 350 ---------
 .../sender/recover/SyncSenderLogAnalyzerTest.java  | 201 -----
 .../sync/sender/recover/SyncSenderLoggerTest.java  | 112 ---
 .../db/sync/sender/transfer/SyncClientTest.java    | 161 ----
 .../db/sync/transport/TransportServiceTest.java    | 205 ++++++
 server/src/test/resources/iotdb-engine.properties  |   3 +-
 server/src/test/resources/logback-test.xml         |   2 +-
 .../java/org/apache/iotdb/rpc/TSStatusCode.java    |   4 +-
 thrift-sync/src/main/thrift/sync.thrift            |  51 --
 thrift-sync/src/main/thrift/transport.thrift       |  90 +++
 155 files changed, 10290 insertions(+), 5789 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
index cc1d3797e5..fc0234fb29 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
@@ -31,7 +31,7 @@ singleStatement
     ;
 
 statement
-    : ddlStatement | dmlStatement | dclStatement | utilityStatement
+    : ddlStatement | dmlStatement | dclStatement | utilityStatement | syncStatement
     ;
 
 ddlStatement
@@ -64,6 +64,10 @@ utilityStatement
     | showQueryProcesslist | killQuery | grantWatermarkEmbedding | revokeWatermarkEmbedding
     | loadConfiguration | loadTimeseries | loadFile | removeFile | unloadFile;
 
+syncStatement
+    : startPipeServer | stopPipeServer | showPipeServer
+    | createPipeSink | showPipeSinkType | showPipeSink | dropPipeSink
+    | createPipe | showPipe | stopPipe | startPipe | dropPipe;
 
 /**
  * 2. Data Definition Language (DDL)
@@ -677,9 +681,68 @@ unloadFile
     : UNLOAD srcFileName=STRING_LITERAL dstFileDir=STRING_LITERAL
     ;
 
+/**
+ * 6. syncStatement
+ */
+
+// pipesink statement
+createPipeSink
+    : CREATE PIPESINK pipeSinkName=ID AS pipeSinkType=ID (LR_BRACKET syncAttributeClauses RR_BRACKET)?
+    ;
+
+showPipeSinkType
+    : SHOW PIPESINKTYPE
+    ;
+
+showPipeSink
+    : SHOW ((PIPESINK (pipeSinkName=ID)?) | PIPESINKS)
+    ;
+
+dropPipeSink
+    : DROP PIPESINK pipeSinkName=ID
+    ;
+
+// pipe statement
+createPipe
+    : CREATE PIPE pipeName=ID TO pipeSinkName=ID (FROM LR_BRACKET selectStatement RR_BRACKET)? (WITH syncAttributeClauses)?
+    ;
+
+showPipe
+    : SHOW ((PIPE (pipeName=ID)?) | PIPES)
+    ;
+
+stopPipe
+    : STOP PIPE pipeName=ID
+    ;
+
+startPipe
+    : START PIPE pipeName=ID
+    ;
+
+dropPipe
+    : DROP PIPE pipeName=ID
+    ;
+
+// attribute clauses
+syncAttributeClauses
+    : propertyClause (COMMA propertyClause)*
+    ;
+
+// sync receiver
+startPipeServer
+    : START PIPESERVER
+    ;
+
+stopPipeServer
+    : STOP PIPESERVER
+    ;
+
+showPipeServer
+    : SHOW PIPESERVER
+    ;
 
 /**
- * 6. Common Clauses
+ * 7. Common Clauses
  */
 
 // IoTDB Objects
diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
index 803fe25f23..48a415de66 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
@@ -362,6 +362,30 @@ PATHS
     : P A T H S
     ;
 
+PIPE
+    : P I P E
+    ;
+
+PIPES
+    : P I P E S
+    ;
+
+PIPESERVER
+    : P I P E S E R V E R
+    ;
+
+PIPESINK
+    : P I P E S I N K
+    ;
+
+PIPESINKS
+    : P I P E S I N K S
+    ;
+
+PIPESINKTYPE
+    : P I P E S I N K T Y P E
+    ;
+
 PREVIOUS
     : P R E V I O U S
     ;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/ClusterIoTDB.java b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterIoTDB.java
index 4b56784505..9138a98ca2 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/ClusterIoTDB.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/ClusterIoTDB.java
@@ -229,7 +229,6 @@ public class ClusterIoTDB implements ClusterIoTDBMBean {
     IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
     // init server's configuration first, because the cluster configuration may read settings from
     // the server's configuration.
-    config.setSyncEnable(false);
     // auto create schema is took over by cluster module, so we disable it in the server module.
     config.setAutoCreateSchemaEnabled(false);
     // check cluster config
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/utils/StatusUtils.java b/cluster/src/main/java/org/apache/iotdb/cluster/utils/StatusUtils.java
index 9291e05a40..97a71a3344 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/utils/StatusUtils.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/utils/StatusUtils.java
@@ -102,9 +102,6 @@ public class StatusUtils {
       case SYSTEM_CHECK_ERROR:
         status.setMessage("Meet error while system checking. ");
         break;
-      case SYNC_DEVICE_OWNER_CONFLICT_ERROR:
-        status.setMessage("Sync device owners conflict. ");
-        break;
       case SYNC_CONNECTION_EXCEPTION:
         status.setMessage("Meet error while sync connecting. ");
         break;
@@ -201,6 +198,9 @@ public class StatusUtils {
       case PARSE_LOG_ERROR:
         status.setMessage("Parse log error.");
         break;
+      case PIPESINK_ERROR:
+        status.setMessage("PipeSink error.");
+        break;
       default:
         status.setMessage("");
         break;
diff --git a/docs/UserGuide/Maintenance-Tools/Sync-Tool.md b/docs/UserGuide/Maintenance-Tools/Sync-Tool.md
index a97a3ab775..d2685608f4 100644
--- a/docs/UserGuide/Maintenance-Tools/Sync-Tool.md
+++ b/docs/UserGuide/Maintenance-Tools/Sync-Tool.md
@@ -19,172 +19,332 @@
 
 -->
 
-# Collaboration of Edge and Cloud
+# Edge-Cloud Collaboration
 
-## TsFile Sync Tool
+## 1.Introduction
 
-### Introduction
-The Sync Tool is an IoTDB suite tool that periodically uploads persistent tsfiles from the sender disk to the receiver and loads them.
+The Sync Tool is an IoTDB suite tool that continuously uploads the timeseries data from the edge (sender) to the cloud(receiver).
 
-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 a separate script (see Sections `Usage` for details). The frequency cycle of the sync can be set by the user. 
+On the sender side of the sync, the sync module is embedded in the IoTDB engine. You should start an IoTDB before using the Sync tool.
 
-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 for a separate port, which can be set by the user (see Section `Configuration` for details). 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, as detailed in Section `Configuration`. 
+You can use SQL commands to start or close a synchronization task at the sender, and you can check the status of the synchronization task at any time. At the receiving end, you can set the IP white list to specify the access IP address range of sender.
 
-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.
+## 2.Model definition
 
-> Note: Before using the sync tool, the client and server need to be configured separately. The configuration is detailed in Sections Configuration.
+![img](https://y8dp9fjm8f.feishu.cn/space/api/box/stream/download/asynccode/?code=ODc2ZTdjZGE3ODBiODY0MjJiOTBmYmY4NGE1NDlkNDZfR1pZZWhzQTdBQjFLNHhOdlFQTVo1T0hsOWtSU25LenRfVG9rZW46Ym94Y25IWktNd0hEN2hBTFFwY1lDQlBmS0xmXzE2NDg3MDI5NTg6MTY0ODcwNjU1OF9WNA)
 
-### 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 scenario above, 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.
+Two machines A and B, which are installed with iotdb, we want to continuously synchronize the data from A to B. To better describe this process, we introduce the following concepts.
 
-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.
+- Pipe
+  - It refers to a synchronization task. In the above case, we can see that there is a data flow pipeline connecting A and B.
+  - A pipe has three states, RUNNING, STOP and DROP, which respectively indicate running, pause and permanent cancellation.
+- PipeSink
+  - It refers to the receiving end. In the above case, pipesink is machine B. At present, the pipesink type only supports IoTDB, that is, the receiver is the IoTDB instance installed on B.
+  - Pipeserver: when the type of pipesink is IoTDB, you need to open the pipeserver service of IoTDB to process the pipe data.
 
-### Precautions for Use
+## 3.Precautions for Use
 
-Statement "alter timeseries add tag" will not effect the receiver when a sync-tool is running.
+- The Sync Tool only supports for many-to-one, that is, one sender should connect to exactly one receiver. One receiver can receive data from more senders.
+- The sender can only have one pipe in non drop status. If you want to create a new pipe, please drop the current pipe.
+- When one or more senders send data to a receiver, there should be no intersection between the respective device path sets of these senders and receivers, otherwise unexpected errors may occur.
+  - e.g. When sender A includes path `root.sg.d.s`, sender B also includes the path `root.sg.d.s`, sender A deletes storage group `root.sg` will also delete all data of B stored in the path `root.sg.d.s` at receiver.
+- Synchronization between the two machines is not supported at present.
+- The Sync Tool only synchronizes insertions, deletions, metadata creations and deletions, do not support TTL settings, trigger and other operations.
 
-### 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:
+## 4.Quick Start
 
-|parameter: is_sync_enable||
-|--- |--- |
-|Description |Sync function switch, which is configured as true to indicate that the receiver is allowed to receive the data from the sender and load it. When set to false, it means that the receiver is not allowed to receive the data from any sender. |
-|Type|Boolean|
-|Default|false|
-|Modalities for Entry into Force after Modification|Restart receiver|
+Execute the following SQL statements at the sender and receiver to quickly start a data synchronization task between two IoTDB. For complete SQL statements and configuration matters, please see the `parameter configuration`and `SQL` sections. For more usage examples, please refer to the `usage examples` section.
 
+### 4.1 Receiver
 
-|parameter: IP_white_list||
-|--- |--- |
-|Description |Set up a white list of sender IP addresses, which is expressed in the form of network segments and separated by commas between multiple network segments. When the sender transfers data to the receiver, only when the IP address of the sender is within the network segment set by the whitelist can the receiver allow the sync operation. If the whitelist is empty, the receiver does not allow any sender to sync data. The default receiver accepts all IP sync requests.|
-|Type|String|
-|Default|0.0.0.0/0|
-|Modalities for Entry into Force after Modification|Restart receiver|
+- Start PipeServer.
 
+```
+IoTDB> START PIPESERVER
+```
+
+- Stop PipeServer(should execute after dropping all pipes which connect to this receiver).
+
+```
+IOTDB> STOP PIPESERVER
+```
+
+### 4.2 Sender
+
+- Create a pipesink with IoTDB type.
+
+```
+IoTDB> CREATE PIPESINK central_iotdb AS IoTDB (IP='There is your goal IP')
+```
+
+- Establish a pipe(before creation, ensure that PipeServer is running on receiver).
+
+```
+IoTDB> CREATE PIPE my_pipe TO central_iotDB
+```
+
+- Start this pipe.
+
+```
+IoTDB> START PIPE my_pipe
+```
+
+- Show pipe's status.
+
+```
+IoTDB> SHOW PIPES
+```
+
+- Stop this pipe.
+
+```
+IoTDB> STOP PIPE my_pipe
+```
 
-|parameter: sync_server_port||
-|--- |--- |
-|Description |Sync receiver port to listen. Make sure that the port is not a system reserved port and is not occupied. This paramter is valid only when the parameter is_sync_enable is set to TRUE. |
-|Type|Short Int : [0,65535]|
-|Default|5555|
-|Modalities for Entry into Force after Modification|Restart receiver|
+- Continue this pipe.
 
-#### Sync Sender
-The parameters of the sync sender are configured in a separate configuration file iotdb-sync-client.properties with the installation directory of ```$IOTDB_HOME/conf/iotdb-sync-client.properties```. In this configuration file, there are five parameters related to the sync sender. The configuration instructions are as follows:
+```
+IoTDB> START PIPE my_pipe
+```
+
+- Drop this pipe(delete all information about this pipe).
+
+```
+IoTDB> DROP PIPE my_pipe
+```
+
+## 5.Parameter Configuration
 
-|parameter: server_ip||
-|--- |--- |
-|Description |IP address of sync receiver. |
-|Type|String|
-|Default|127.0.0.1|
-|Modalities for Entry into Force after Modification|Restart client|
+All parameters are in `$IOTDB_ HOME$/conf/iotdb-engine`, after all modifications are completed, execute `load configuration` and it will take effect immediately.
 
+### 5.1 Sender
 
-|parameter: server_port||
-|--- |--- |
-|Description |Listening port of sync receiver, it is necessary to ensure that the port is consistent with the configuration of the listening port set in receiver. |
-|Type|Short Int : [0,65535]|
-|Default|5555|
-|Modalities for Entry into Force after Modification|Restart client|
+| **Parameter Name** | **max_number_of_sync_file_retry**                            |
+| ------------------ | ------------------------------------------------------------ |
+| Description        | The maximum number of retries when the sender fails to synchronize files to the receiver. |
+| Data type          | Int : [0,2147483647]                                         |
+| Default value      | 5                                                            |
 
 
-|parameter: sync_period_in_second||
-|--- |--- |
-|Description |The period time of sync process, the time unit is second. |
-|Type|Int : [0,2147483647]|
-|Default|600|
-|Modalities for Entry into Force after Modification|Restart client|
 
+### 5.2 Receiver
 
-|parameter: iotdb_schema_directory||
-|--- |--- |
-|Description |The absolute path of the sender's IoTDB schema file, such as `$IOTDB_HOME/data/system/schema/mlog.txt` (if the user does not manually set the path of schema metadata, 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. |
-|Type|String|
-|Modalities for Entry into Force after Modification|Restart client|
+| **Parameter Name** | **ip_white_list**                                            |
+| ------------------ | ------------------------------------------------------------ |
+| Description        | Set the white list of IP addresses of the sending end of the synchronization, which is expressed in the form of network segments, and multiple network segments are separated by commas. When the sender synchronizes data to the receiver, the receiver allows synchronization only when the IP address of the sender is within the network segment set in the white list. If the whitelist is empty, the receiver does not allow any sender to synchronize data. By default, the re [...]
+| Data type          | String                                                       |
+| Default value      | 0.0.0.0/0                                                    |
 
-|parameter: sync_storage_groups||
-|--- |--- |
-|Description |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. |
-|Type|String|
-|Example|root.sg1, root.sg2|
-|Modalities for Entry into Force after Modification|Restart client|
 
 
-|parameter: max_number_of_sync_file_retry||
-|--- |--- |
-|Description |The maximum number of retry when syncing a file to receiver fails. |
-|Type|Int : [0,2147483647]|
-|Example|5|
-|Modalities for Entry into Force after Modification|Restart client|
+| **Parameter Name** | **sync_server_port**                                         |
+| ------------------ | ------------------------------------------------------------ |
+| Description        | The port which the receiver listens, please ensure this port is not occupied by other applications. |
+| Data type          | Short Int : [0,65535]                                        |
+| Default value      | 6670                                                         |
 
 
-### Usage
-#### Start Sync Receiver
-1. Set up parameters of sync receiver. For example:
+
+## 6.SQL
+
+### 6.1 Sender
+
+- Create a pipesink with IoTDB type, where IP and port are optional parameters.
 
 ```
-	####################
-	### Sync Server Configuration
-	####################
-	# Whether to open the sync_server_port for receiving data from sync client, the default is closed
-	is_sync_enable=false
-	# Sync server port to listen
-	sync_server_port=5555
-	# White IP list of Sync client.
-	# Please use the form of network segment to present the range of IP, for example: 192.168.0.0/16
-	# If there are more than one IP segment, please separate them by commas
-	# The default is to allow all IP to sync
-	ip_white_list=0.0.0.0/0
+IoTDB> CREATE PIPESINK <PipeSinkName> AS IoTDB [(IP='127.0.0.1',port=6670);]
 ```
 
-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.
+- Show all pipesink types supported by IoTDB.
 
+```Plain%20Text
+IoTDB> SHOW PIPESINKTYPE
+IoTDB>
++-----+
+| type|
++-----+
+|IoTDB|
++-----+
+```
 
-#### Stop Sync Receiver
-Stop IoTDB and the sync receiver will be closed at the same time.
+- Show all pipesinks' definition, the results set has three columns, name, pipesink's type and pipesink's attributes.
+
+```
+IoTDB> SHOW PIPESINKS
+IoTDB> SHOW PIPESINK [PipeSinkName]
+IoTDB> 
++-----------+-----+------------------------+
+|       name| type|              attributes|
++-----------+-----+------------------------+
+|my_pipesink|IoTDB|ip='127.0.0.1',port=6670|
++-----------+-----+------------------------+
+```
 
-#### Start Sync Sender
-1. Set up parameters of sync sender. For example:
+- Drop the pipesink with PipeSinkName parameter.
 
 ```
-	# Sync receiver server address
-	server_ip=127.0.0.1
-	# Sync receiver server port
-	server_port=5555
-	# The period time of sync process, the time unit is second.
-	sync_period_in_second=600
-	# 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
-	# The maximum number of retry when syncing a file to receiver fails.
-	max_number_of_sync_file_retry=5
+IoTDB> DROP PIPESINK <PipeSinkName>
 ```
 
-2. Start sync sender
-Users can use the scripts under the ```$IOTDB_HOME/tools``` folder to start the sync sender.
-For Linux and Mac OS X users:
+- Create a pipe.
+- At present, the SELECT statement only supports `**` (i.e. data in all timeseries), the FROM statement only supports `root`, and the WHERE statement only supports the start time of the specified time.
+- If the `SyncDelOp` parameter is true, the deletions of sender will not be synchronized to receiver.
+
+```
+IoTDB> CREATE PIPE my_pipe TO my_iotdb [FROM (select ** from root WHERE time>='yyyy-mm-dd HH:MM:SS' )] [WITH SyncDelOp=true]
 ```
-  Shell >$IOTDB_HOME/tools/start-sync-client.sh
+
+- Show all pipes' status.
+- create time, the creation time of this pipe; name, the name of this pipe; pipesink, the pipesink this pipe connects to; status, this pipe's status.
+- Message, the status message of this pipe. When pipe runs normally, this column is usually empty. When an exception occurs, messages may appear in  following two states.
+  - WARN, this indicates that a data loss or other error has occurred, but the pipe will remain running.
+  - ERROR, this indicates that the network is interrupted for a long time or there is a problem at the receiving end. The pipe is stopped and set to STOP state.
+
 ```
-For Windows users:
+IoTDB> SHOW PIPES
+IoTDB>
++-----------------------+-------+-----------+------+-------+
+|            create time|   name|   pipeSink|status|message|
++-----------------------+-------+-----------+------+-------+
+|2022-03-30T20:58:30.689|my_pipe|my_pipesink|  STOP|       |
++-----------------------+-------+-----------+------+-------+
+```
+
+- Show the pipe status with PipeName. When the PipeName is empty,it is the same with `Show PIPES`.
+
 ```
-  Shell >$IOTDB_HOME\tools\start-sync-client.bat
+IoTDB> SHOW PIPE [PipeName]
 ```
 
+- Stop the pipe with PipeName.
 
-#### Stop Sync Sender
-Users can use the scripts under the ```$IOTDB_HOME/tools``` folder to stop the sync sender.
-For Linux and Mac OS X users:
 ```
-  Shell >$IOTDB_HOME/tools/stop-sync-client.sh
+IoTDB> STOP PIPE <PipeName>
 ```
-For Windows users:
+
+- Continue the pipe with PipeName.
+
+```
+IoTDB> START PIPE <PipeName>
+```
+
+- Drop the pipe with PipeName(delete all information about this pipe).
+
 ```
-  Shell >$IOTDB_HOME\tools\stop-sync-client.bat
+IoTDB> DROP PIPE <PipeName>
 ```
 
+#### 6.2 Receiver
+
+- Start the PipeServer service.
+
+```
+IoTDB> START PIPESERVER
+```
+
+- Stop the PipeServer service.
+
+```
+IoTDB> STOP PIPESERVER
+```
+
+- Show the information of PipeServer.
+- True means the PipeServer is running, otherwise not.
+
+```
+IoTDB> SHOW PIPESERVER STATUS
++----------+
+|    enalbe|
++----------+
+|true/false|
++----------+
+```
+
+## 7. Usage Examples
+
+##### Goal
+
+- Create a synchronize task from sender IoTDB to receiver IoTDB.
+- Sender wants to synchronize the data after 2022-3-30 00:00:00.
+- Sender does not want to synchronize the deletions.
+- Sender has an unstable network environment, needs more retries.
+- Receiver only wants to receive data from this sender(sender ip 192.168.0.1).
+
+##### **Receiver**
+
+- `vi conf/iotdb-engine.properties`  to config the parameters,set the IP white list to 192.168.0.1/1 to receive and only receive data from sender.
+
+```
+####################
+### PIPE Server Configuration
+####################
+# PIPE server port to listen
+# Datatype: int
+# pipe_server_port=6670
+
+# White IP list of Sync client.
+# Please use the form of network segment to present the range of IP, for example: 192.168.0.0/16
+# If there are more than one IP segment, please separate them by commas
+# The default is to allow all IP to sync
+# Datatype: String
+ip_white_list=192.168.0.1/1
+```
+
+- Start PipeServer service at receiver.
+
+```
+IoTDB> START PIPESERVER
+```
+
+- Show PipeServer's status, a `True` result means running correctly.
+
+```
+IoTDB> SHOW PIPESERVER STATUS
+```
+
+##### Sender
+
+- Config the `max_number_of_sync_file_retry` parameter to 10.
+
+```
+####################
+### PIPE Sender Configuration
+####################
+# The maximum number of retry when syncing a file to receiver fails.
+max_number_of_sync_file_retry=10
+```
+
+- Create pipesink with IoTDB type, input ip address 192.168.0.1, port 6670.
+
+```
+IoTDB> CREATE PIPESINK my_iotdb AS IoTDB (IP='192.168.0.2',PORT=6670)
+```
+
+- Create pipe connect to my_iotdb, input the start time 2022-03-30 00:00:00 in WHERE statments, set the `SyncDelOp` to false.
+
+```
+IoTDB> CREATE PIPE p TO my_iotdb FROM (select ** from root where time>='2022-03-30 00:00:00') WITH SyncDelOp=false
+```
+
+- Show the status of pipe p.
+
+```
+IoTDB> SHOW PIPE p
+```
+
+- Drop the pipe p.
+
+```
+IoTDB> DROP PIPE p
+```
+
+## 8.Q&A
+
+- Execute `STOP PIPESERVER` to close IoTDB PipeServer service with message 
+
+  ```
+  Msg: 328: Failed to stop pipe server because there is pipe still running.
+  ```
+
+  - Cause by: There is a running pipe connected to this receiver.
+  - Solution: Execute `STOP PIPE <PipeName>` to stop pipe, then stop PipeServer service.
diff --git a/docs/zh/UserGuide/Maintenance-Tools/Sync-Tool.md b/docs/zh/UserGuide/Maintenance-Tools/Sync-Tool.md
index 11a3da27b9..34a5090e61 100644
--- a/docs/zh/UserGuide/Maintenance-Tools/Sync-Tool.md
+++ b/docs/zh/UserGuide/Maintenance-Tools/Sync-Tool.md
@@ -19,171 +19,440 @@
 
 -->
 
-# 端云协同 
+# 端云协同
 
-## TsFile 同步工具
+## 1.介绍
 
-### 介绍
+同步工具是持续将边缘端(发送端) IoTDB 中的时间序列数据上传并加载至云端(接收端) IoTDB 的套件工具。
 
-同步工具是定期将本地磁盘中新增的已持久化的 tsfile 文件上传至云端并加载到 IoTDB 的套件工具。
+同步工具的发送端内嵌于 IoTDB 的引擎,使用同步工具需要首先启动IoTDB。
 
-在同步工具的发送端,同步模块是一个独立的进程,独立于本地的 IoTDB。通过独立的脚本进行启动和关闭(详见本章节`使用方式`),同步的频率周期可由用户设置。
+可以在发送端使用 SQL 命令来启动或者关闭一个同步任务,并且可以随时查看同步任务的状态。在接收端,您可以通过设置IP白名单来规定准入IP地址范围。
 
-在同步工具的接收端,同步模块内嵌于 IoTDB 的引擎,和 IoTDB 处于同一个进程中。同步模块监听一个独立的端口,该端口可由用户设置(详见本章节`配置参数`)。用户使用前,需要在同步接收端设置同步白名单,以网段形式表示,接收端的同步模块只接受位于白名单网段中的发送端同步的数据。
+## 2.模型定义
 
-同步工具具有多对一的发送-接受模式,即一个同步接收端可以同时接受多个同步发送端传输的数据,一个同步发送端只能向一个同步接收端发送数据。
+![img](https://y8dp9fjm8f.feishu.cn/space/api/box/stream/download/asynccode/?code=ODYwOTUxMGI4YjI0N2FlOWFlZmI4MDcxNDlmZDE1MGZfQ1hzcVM1bXJxUkthd1hta3hSaDdnZW1aZHh2cE5iTUxfVG9rZW46Ym94Y25CRjF5dXd0QkVEYzVrSnJHcUdkd3VlXzE2NDkwNTkxNDc6MTY0OTA2Mjc0N19WNA)
 
-> 注意:在使用同步工具前,同步工具的接收端和发送端需要单独配置。
+假设目前有两台机器A和B都安装了IoTDB,希望将A上的数据不断同步至B中。为了更好地描述这个过程,我们引入以下概念。
 
-### 应用场景
+- Pipe
+  - 指一次同步任务,在上述案例中,我们可以看作在A和B之间有一根数据流管道连接了A和B。
+  - 一个Pipe有三种状态,RUNNING,STOP,DROP,分别表示正在运行,暂停和永久取消。
+- PipeSink
+  - 指接收端,在上述案例中,PipeSink即是B这台机器。PipeSink的类型目前仅支持IoTDB,即接收端为B上安装的IoTDB实例。
+  -  PipeServer:当PipeSink的类型为IoTDB的时候,需要打开IoTDB的PipeServer服务来让Pipe数据得到处理。
 
-以一个工厂应用为例,通常有多个分厂和多个总厂,每个分厂中使用一个 IoTDB 实例收集数据,然后将数据定时汇总到总厂中进行备份或者分析等,一个总厂可以接收来自多个分厂的数据,在这种场景下每个 IoTDB 实例所管理的设备各不相同。
+## 3.注意事项
 
-在 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 同步的数据。
+- 发送端只能有一个非DROP状态的Pipe,如果想创建一个新的Pipe,请取消当前Pipe。
 
-### 注意事项
+- 当有一个或多个发送端指向一个接收端时,这些发送端和接收端各自的设备路径集合之间应当没有交集,否则可能产生不可预料错误 
 
-sync功能开启之后,发送端的 alter timeseries add tag 语句将不会被同步到接收端
+  。
 
-### 配置参数
+  - 例如:当发送端A包括路径`root.sg.d.s`,发送端B也包括路径`root.sg.d.s`,当发送端A删除`root.sg`存储组时将也会在接收端删除所有B在接收端的`root.sg.d.s`中存放的数据。
 
-#### 同步工具接收端
-同步工具接收端的参数配置位于 IoTDB 的配置文件 iotdb-engine.properties 中,其安装目录为$IOTDB_HOME/conf/iotdb-engine.properties。在该配置文件中,有四个参数和同步接收端有关,配置说明如下:
+- 两台机器之间目前不支持相互同步。
 
-|参数名|is_sync_enable|
-|--- |--- |
-|描述|同步功能开关,配置为 true 表示接收端允许接收同步的数据并加载,设置为 false 的时候表示接收端不允许接收同步的数据|
-|类型|Boolean|
-|默认值|false|
-|改后生效方式|重启服务生效|
+- 同步工具仅同步所有对数据插入和删除,元数据的创建和删除,如TTL的设置,Trigger,CQ等其他操作均不同步。
 
-|参数名|ip_white_list|
-|--- |--- |
-|描述|设置同步功能发送端 IP 地址的白名单,以网段的形式表示,多个网段之间用逗号分隔。发送端向接收端同步数据时,只有当该发送端 IP 地址处于该白名单设置的网段范围内,接收端才允许同步操作。如果白名单为空,则接收端不允许任何发送端同步数据。默认接收端接受全部 IP 的同步请求。|
-|类型|String|
-|默认值|0.0.0.0/0|
-|改后生效方式|重启服务生效|
+## 4.快速上手
 
-|参数名|sync_server_port|
-|--- |--- |
-|描述|同步接收端服务器监听接口,请确认该端口不是系统保留端口并且未被占用。参数 is_sync_enable 设置为 true 时有效,参数 is_sync_enable 设置为 false 时无效|
-|类型|Short Int : [0,65535]|
-|默认值|5555|
-|改后生效方式|重启服务生效|
+在发送端和接收端执行如下语句即可快速开始两个 IoTDB 之间的数据同步,完整的 SQL 语句和配置事项请查看`配置参数`和`SQL`两节,更多使用范例请参考`使用范例`节。
 
-#### 同步工具发送端
-同步功能发送端的参数配置在一个单独的配置文件中,其安装目录为```$IOTDB_HOME/conf/iotdb-sync-client.properties```。在该配置文件中,有五个参数和同步发送端有关,配置说明如下:
+#### 4.1接收端
 
-|参数名|server_ip|
-|--- |--- |
-|描述|同步接收端的 IP 地址|
-|类型|String|
-|默认值|127.0.0.1|
-|改后生效方式|重启同步功能发送端生效|
+- 开启PipeServer
 
-|参数名|server_port|
-|--- |--- |
-|描述|同步接收端服务器监听端口,需要保证该端口和同步接收端配置的监听端口一致|
-|类型|Short Int : [0,65535]|
-|默认值|5555|
-|改后生效方式|重启同步功能发送端生效|
+```
+IoTDB> START PIPESERVER
+```
+
+- 关闭PipeServer(在所有发送端取消了Pipe之后执行)
+
+```
+IOTDB> STOP PIPESERVER
+```
+
+#### 4.2发送端
+
+- 创建接收端为 IoTDB 类型的 Pipe Sink
+
+```
+IoTDB> CREATE PIPESINK my_iotdb AS IoTDB (ip='输入你的IP')
+```
+
+- 创建同步任务Pipe(开启之前请确保接收端 IoTDB 的 PipeServer 已经启动)
+
+```
+IoTDB> CREATE PIPE my_pipe TO my_iotdb
+```
+
+- 开始同步任务
+
+```
+IoTDB> START PIPE my_pipe
+```
+
+- 显示所有同步任务状态
+
+```
+IoTDB> SHOW PIPES
+```
+
+- 暂停任务
+
+```
+IoTDB> STOP PIPE my_pipe
+```
+
+- 继续被暂停的任务
+
+```
+IoTDB> START PIPE my_pipe
+```
+
+- 关闭任务(状态信息可被删除)
+
+```
+IoTDB> DROP PIPE my_pipe
+```
+
+## 5.配置参数
+
+所有参数修改均在`$IOTDB_HOME$/conf/iotdb-engine.properties`中,所有修改完成之后执行`load configuration`之后即可立刻生效。
+
+#### 5.1发送端相关
+
+| **参数名** | **max_number_of_sync_file_retry**          |
+| ---------- | ------------------------------------------ |
+| 描述       | 发送端同步文件到接收端失败时的最大重试次数 |
+| 类型       | Int : [0,2147483647]                       |
+| 默认值     | 5                                          |
+
+
+
+#### 5.2接收端相关
+
+| **参数名** | **ip_white_list**                                            |
+| ---------- | ------------------------------------------------------------ |
+| 描述       | 设置同步功能发送端 IP 地址的白名单,以网段的形式表示,多个网段之间用逗号分隔。发送端向接收端同步数据时,只有当该发送端 IP 地址处于该白名单设置的网段范围内,接收端才允许同步操作。如果白名单为空,则接收端不允许任何发送端同步数据。默认接收端接受全部 IP 的同步请求。 |
+| 类型       | String                                                       |
+| 默认值     | 0.0.0.0/0                                                    |
+
+
+
+| **参数名** | ***pipe_server_port***                                       |
+| ---------- | ------------------------------------------------------------ |
+| 描述       | 同步接收端服务器监听接口,请确认该端口不是系统保留端口并且未被占用。 |
+| 类型       | Short Int : [0,65535]                                        |
+| 默认值     | 6670                                                         |
+
+
+
+## 6.SQL
+
+#### 6.1发送端
+
+- 创建接收端为 IoTDB 类型的 Pipe Sink,其中IP和port是可选参数
+
+```
+IoTDB> CREATE PIPESINK <PipeSinkName> AS IoTDB [(ip='127.0.0.1',port=6670);]
+```
+
+- 显示当前所能支持的 Pipe Sink 类型
+
+```Plain%20Text
+IoTDB> SHOW PIPESINKTYPE
+IoTDB>
++-----+
+| type|
++-----+
+|IoTDB|
++-----+
+```
+
+- 显示当前所有 Pipe Sink 定义,结果集有三列,分别表示pipesink的名字,pipesink的类型,pipesink的属性
+
+```
+IoTDB> SHOW PIPESINKS
+IoTDB> SHOW PIPESINK [PipeSinkName]
+IoTDB> 
++-----------+-----+------------------------+
+|       name| type|              attributes|
++-----------+-----+------------------------+
+|my_pipesink|IoTDB|ip='127.0.0.1',port=6670|
++-----------+-----+------------------------+
+```
+
+- 删除 Pipe Sink 信息
+
+```
+IoTDB> DROP PIPESINK <PipeSinkName>
+```
+
+- 创建同步任务
+  - 其中Select语句目前仅支持`**`(即所有序列中的数据),FROM语句目前仅支持`root`,Where语句仅支持指定time的起始时间
+  - `SyncDelOp`参数为true时会同步删除数据操作,否则不同步删除数据操作
+
+```
+IoTDB> CREATE PIPE my_pipe TO my_iotdb [FROM (select ** from root WHERE time>=yyyy-mm-dd HH:MM:SS)] [WITH SyncDelOp=true]
+```
+
+- 显示所有同步任务状态
+
+> 该指令在发送端和接收端均可执行
+
+- create time,pipe的创建时间
 
-|参数名|sync_period_in_second|
-|--- |--- |
-|描述|同步周期,两次同步任务开始时间的间隔,单位为秒 (s)|
-|类型|Int : [0,2147483647]|
-|默认值|600|
-|改后生效方式|重启同步功能发送端生效|
+- name,pipe的名字
 
-|参数名|sync_storage_groups|
-|--- |--- |
-|描述|进行同步的存储组列表,存储组间用逗号分隔;若列表设置为空表示同步所有存储组,默认为空|
-|类型|String|
-|示例|root.sg1, root.sg2|
-|改后生效方式|重启同步功能发送端生效|
+- role,当前IoTDB在pipe中的角色,可能有两种角色:
+  - sender,当前IoTDB为同步发送端
 
-|参数名|max_number_of_sync_file_retry|
-|--- |--- |
-|描述|发送端同步文件到接收端失败时的最大重试次数|
-|类型|Int : [0,2147483647]|
-|示例|5|
-|改后生效方式|重启同步功能发送端生效|
+- receiver,当前IoTDB为同步接收端
 
-### 使用方式
+- remote,pipe的对端信息
+  - 当role为receiver时,这一字段值为发送端ip
 
-#### 启动同步功能接收端
+- 当role为sender时,这一字段值为pipeSink名称
 
-1. 配置接收端的参数,例如:
+- status,pipe状态
+- message,pipe运行信息,当pipe正常运行时,这一字段通常为空,当出现异常时,可能出现两种状态:
+  - WARN状态,这表明发生了数据丢失或者其他错误,但是Pipe会保持运行
+  - ERROR状态,这表明发生了网络长时间中断或者接收端出现问题,Pipe被停止,置为STOP状态
 
 ```
-	####################
-	### Sync Server Configuration
-	####################
+IoTDB> SHOW PIPES
+IoTDB>
++-----------------------+--------+--------+-------------+---------+-------+
+|            create time|   name |    role|       remote|   status|message|
++-----------------------+--------+--------+-------------+---------+-------+
+|2022-03-30T20:58:30.689|my_pipe1|  sender|  my_pipesink|     STOP|       |
++-----------------------+--------+--------+-------------+---------+-------+ 
+|2022-03-31T12:55:28.129|my_pipe2|receiver|192.168.11.11|  RUNNING|       |
++-----------------------+--------+--------+-------------+---------+-------+
+```
+
+- 显示指定同步任务状态,当未指定PipeName时,与`SHOW PIPES`等效
 
-	# Whether to open the sync_server_port for receiving data from sync client, the default is closed
-	is_sync_enable=true
+```
+IoTDB> SHOW PIPE [PipeName]
+```
+
+- 暂停任务
+
+```
+IoTDB> STOP PIPE <PipeName>
+```
+
+- 继续被暂停的任务
+
+```
+IoTDB> START PIPE <PipeName>
+```
 
-	# Sync server port to listen
-	sync_server_port=5555
+- 关闭任务(状态信息可被删除)
 
-	# White IP list of Sync client.
-	# Please use the form of network segment to present the range of IP, for example: 192.168.0.0/16
-	# If there are more than one IP segment, please separate them by commas
-	# The default is to allow all IP to sync
-	ip_white_list=0.0.0.0/0
+```
+IoTDB> DROP PIPE <PipeName>
 ```
 
-2. 启动 IoTDB 引擎,同步功能接收端会同时启动,启动时 LOG 日志会出现`IoTDB: start SYNC ServerService successfully`字样,表示同步接收端启动成功。
+#### 6.2接收端
 
-#### 关闭同步功能接收端
+- 启动本地的 IoTDB Pipe Server
 
-关闭 IoTDB,同步功能接收端会同时关闭。
+```
+IoTDB> START PIPESERVER
+```
 
-#### 启动同步功能发送端
-1. 配置发送端的参数
+- 关闭本地的 IoTDB Pipe Server
 
 ```
-	# Sync receiver server address
-	server_ip=127.0.0.1
+IoTDB> STOP PIPESERVER
+```
+
+- 显示本地 Pipe Server 的信息
+  - true表示PipeServer正在运行,false表示PipeServer停止服务
+
+```
+IoTDB> SHOW PIPESERVER
++----------+
+|    enalbe|
++----------+
+|true/false|
++----------+
+```
+
+## 7.使用示例
 
-	# Sync receiver server port
-	server_port=5555
+#### **目标**
 
-	# The period time of sync process, the time unit is second.
-	sync_period_in_second=600
+- 创建一个从边端 IoTDB 到 云端 IoTDB 的 同步工作
+- 边端希望同步从2022年3月30日0时之后的数据
+- 边端不希望同步所有的删除操作
+- 边端处于弱网环境,需要配置更多的重试次数
+- 云端IoTDB仅接受来自边端的IoTDB的数据
 
-	# 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
+#### **接收端操作**
 
-	# The maximum number of retry when syncing a file to receiver fails.
-	max_number_of_sync_file_retry=5
+- `vi conf/iotdb-engine.properties` 配置云端参数,将白名单设置为仅接收来自IP为 192.168.0.1的边端的数据
 
 ```
-2. 启动同步功能发送端
+####################
+### PIPE Server Configuration
+####################
+# PIPE server port to listen
+# Datatype: int
+# pipe_server_port=6670
+
+# White IP list of Sync client.
+# Please use the form of network segment to present the range of IP, for example: 192.168.0.0/16
+# If there are more than one IP segment, please separate them by commas
+# The default is to allow all IP to sync
+# Datatype: String
+ip_white_list=192.168.0.1/32
+```
+
+- 云端启动 IoTDB 同步接收端
 
-用户可以使用```$IOTDB_HOME/tools```文件夹下的脚本启动同步功能的发送端
-Linux 系统与 MacOS 系统启动命令如下:
 ```
-  Shell >$IOTDB_HOME/tools/start-sync-client.sh
+IoTDB> START PIPESERVER
 ```
-Windows 系统启动命令如下:
+
+- 云端显示 IoTDB 同步接收端信息,如果结果为true则表示正确启动
+
 ```
-  Shell >$IOTDB_HOME\tools\start-sync-client.bat
+IoTDB> SHOW PIPESERVER
 ```
 
-#### 关闭同步功能发送端
+#### **发送端操作**
+
+- 配置边端参数,将`max_number_of_sync_file_retry`参数设置为10
 
-用户可以使用```$IOTDB_HOME/tools```文件夹下的脚本关闭同步功能的发送端。
-Linux 系统与 MacOS 系统停止命令如下:
 ```
-  Shell >$IOTDB_HOME/tools/stop-sync-client.sh
+####################
+### PIPE Sender Configuration
+####################
+# The maximum number of retry when syncing a file to receiver fails.
+max_number_of_sync_file_retry=10
 ```
-Windows 系统停止命令如下:
+
+- 创建云端PipeSink,指定类型为IoTDB,指定云端IP地址为192.168.0.1,指定云端的PipeServer服务端口为6670
+
 ```
-  Shell >$IOTDB_HOME\tools\stop-sync-client.bat
+IoTDB> CREATE PIPESINK my_iotdb AS IoTDB (ip='192.168.0.1',port=6670)
 ```
+
+- 创建Pipe,指定连接到my_iotdb的PipeSink,在WHREE子句中输入开始时间点2022年3月30日0时,将SyncDelOp置为false
+
+```
+IoTDB> CREATE PIPE p TO my_iotdb FROM (select ** from root where time>=2022-03-30 00:00:00) WITH SyncDelOp=false
+```
+
+- 启动Pipe
+
+```Plain%20Text
+IoTDB> START PIPE p
+```
+
+- 显示同步任务状态
+
+```
+IoTDB> SHOW PIPE p
+```
+
+#### 结果验证
+
+在发送端执行以下SQL
+
+```SQL
+SET STORAGE GROUP TO root.vehicle;
+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;
+insert into root.vehicle.d0(timestamp,s0) values(now(),10);
+insert into root.vehicle.d0(timestamp,s0,s1) values(now(),12,'12');
+insert into root.vehicle.d0(timestamp,s1) values(now(),'14');
+insert into root.vehicle.d1(timestamp,s2) values(now(),16.0);
+insert into root.vehicle.d1(timestamp,s2,s3) values(now(),18.0,true);
+insert into root.vehicle.d1(timestamp,s3) values(now(),false);
+flush;
+```
+
+在发送端和接受端执行查询,可查询到相同的结果
+
+```Plain%20Text
+IoTDB> select ** from root.vehicle
++-----------------------------+------------------+------------------+------------------+------------------+
+|             Time|root.vehicle.d0.s0|root.vehicle.d0.s1|root.vehicle.d1.s3|root.vehicle.d1.s2|
++-----------------------------+------------------+------------------+------------------+------------------+
+|2022-04-03T20:08:17.127+08:00|        10|       null|       null|       null|
+|2022-04-03T20:08:17.358+08:00|        12|        12|       null|       null|
+|2022-04-03T20:08:17.393+08:00|       null|        14|       null|       null|
+|2022-04-03T20:08:17.538+08:00|       null|       null|       null|       16.0|
+|2022-04-03T20:08:17.753+08:00|       null|       null|       true|       18.0|
+|2022-04-03T20:08:18.263+08:00|       null|       null|       false|       null|
++-----------------------------+------------------+------------------+------------------+------------------+
+Total line number = 6
+It costs 0.134s
+```
+
+## 8.常见问题
+
+- 执行 
+
+  ```
+  STOP PIPESERVER
+  ```
+
+   关闭本地的 IoTDB Pipe Server 时提示 
+
+  ```
+  Msg: 328: Failed to stop pipe server because there is pipe still running.
+  ```
+
+  - 原因:接收端有正在运行的同步任务
+  - 解决方案:在发送端先执行 `STOP PIPE` PipeName 停止任务,后关闭 IoTDB Pipe Server
+
+- 执行 
+
+  ```
+  CREATE PIPE mypipe
+  ```
+
+    提示  
+
+  ```
+  Msg: 411: Create transport for pipe mypipe error, because CREATE request connects to receiver 127.0.0.1:6670 error..
+  ```
+
+  - 原因:接收端未启动或接收端无法连接
+  - 解决方案:在接收端执行 `SHOW PIPESERVER` 检查是否启动接收端,若未启动使用 `START PIPESERVER` 启动;检查接收端`iotdb-engine.properties`中的白名单是否包含发送端ip。
+
+- 执行 
+
+  ```
+  DROP PIPESINK pipesinkName
+  ```
+
+   提示 
+
+  ```
+  Msg: 411: Can not drop pipeSink demo, because pipe mypipe is using it.
+  ```
+
+  - 原因:不允许删除有正在运行的PIPE所使用的 PipeSink
+  - 解决方案:在发送端执行 `SHOW PIPE`,停止使用该 PipeSink 的 PIPE
+
+- 发送端创建 PIPE 提示 
+
+  ```
+  Msg: 411: Pipe p is RUNNING, please retry after drop it.
+  ```
+
+  - 原因:已有运行中的 PIPE
+  - 解决方案:执行 `DROP PIPE p `后重试
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/sync/IoTDBSyncReceiverCollectorIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/sync/IoTDBSyncReceiverCollectorIT.java
new file mode 100644
index 0000000000..67df8a3012
--- /dev/null
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/sync/IoTDBSyncReceiverCollectorIT.java
@@ -0,0 +1,513 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.integration.sync;
+
+import org.apache.iotdb.commons.concurrent.ThreadName;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.db.sync.conf.SyncPathUtil;
+import org.apache.iotdb.db.sync.pipedata.DeletionPipeData;
+import org.apache.iotdb.db.sync.pipedata.PipeData;
+import org.apache.iotdb.db.sync.pipedata.SchemaPipeData;
+import org.apache.iotdb.db.sync.pipedata.TsFilePipeData;
+import org.apache.iotdb.db.sync.pipedata.queue.BufferedPipeDataQueue;
+import org.apache.iotdb.db.sync.pipedata.queue.PipeDataQueueFactory;
+import org.apache.iotdb.db.sync.receiver.collector.Collector;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+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.write.schema.MeasurementSchema;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+@Category({LocalStandaloneTest.class})
+public class IoTDBSyncReceiverCollectorIT {
+  private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBSyncReceiverCollectorIT.class);
+  protected static boolean enableSeqSpaceCompaction;
+  protected static boolean enableUnseqSpaceCompaction;
+  protected static boolean enableCrossSpaceCompaction;
+  /** create tsfile and move to tmpDir for sync test */
+  File tmpDir = new File("target/synctest");
+
+  String pipeName1 = "pipe1";
+  String remoteIp1 = "192.168.0.11";
+  long createdTime1 = System.currentTimeMillis();
+  File pipeLogDir1 =
+      new File(SyncPathUtil.getReceiverPipeLogDir(pipeName1, remoteIp1, createdTime1));
+  String pipeName2 = "pipe2";
+  String remoteIp2 = "192.168.0.22";
+  long createdTime2 = System.currentTimeMillis();
+  File pipeLogDir2 =
+      new File(SyncPathUtil.getReceiverPipeLogDir(pipeName2, remoteIp2, createdTime2));
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.envSetUp();
+    enableSeqSpaceCompaction =
+        IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
+    enableUnseqSpaceCompaction =
+        IoTDBDescriptor.getInstance().getConfig().isEnableUnseqSpaceCompaction();
+    enableCrossSpaceCompaction =
+        IoTDBDescriptor.getInstance().getConfig().isEnableCrossSpaceCompaction();
+    IoTDBDescriptor.getInstance().getConfig().setEnableSeqSpaceCompaction(false);
+    IoTDBDescriptor.getInstance().getConfig().setEnableUnseqSpaceCompaction(false);
+    IoTDBDescriptor.getInstance().getConfig().setEnableCrossSpaceCompaction(false);
+    SyncTestUtil.insertData();
+    EnvironmentUtils.shutdownDaemon();
+    File srcDir = new File(IoTDBDescriptor.getInstance().getConfig().getDataDirs()[0]);
+    FileUtils.moveDirectory(srcDir, tmpDir);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    IoTDBDescriptor.getInstance().getConfig().setEnableSeqSpaceCompaction(enableSeqSpaceCompaction);
+    IoTDBDescriptor.getInstance()
+        .getConfig()
+        .setEnableUnseqSpaceCompaction(enableUnseqSpaceCompaction);
+    IoTDBDescriptor.getInstance()
+        .getConfig()
+        .setEnableCrossSpaceCompaction(enableCrossSpaceCompaction);
+    FileUtils.deleteDirectory(tmpDir);
+    FileUtils.deleteDirectory(pipeLogDir1);
+    FileUtils.deleteDirectory(pipeLogDir2);
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void testOnePipe() throws Exception {
+    // 1. restart IoTDB
+    EnvironmentUtils.cleanEnv();
+    EnvironmentUtils.envSetUp();
+
+    // 2. prepare pipelog and pipeDataQueue
+    if (!pipeLogDir1.exists()) {
+      pipeLogDir1.mkdirs();
+    }
+    DataOutputStream outputStream =
+        new DataOutputStream(
+            new FileOutputStream(new File(pipeLogDir1, SyncConstant.COMMIT_LOG_NAME), true));
+    outputStream.writeLong(-1);
+    outputStream.close();
+    int serialNum = 0;
+    File pipeLog1 = new File(pipeLogDir1.getPath(), SyncConstant.getPipeLogName(serialNum));
+    DataOutputStream pipeLogOutput = new DataOutputStream(new FileOutputStream(pipeLog1, false));
+    List<PhysicalPlan> planList = new ArrayList<>();
+    planList.add(new SetStorageGroupPlan(new PartialPath("root.vehicle")));
+    planList.add(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.vehicle.d0.s0"),
+            new MeasurementSchema("s0", TSDataType.INT32, TSEncoding.RLE)));
+    planList.add(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.vehicle.d0.s1"),
+            new MeasurementSchema("s1", TSDataType.TEXT, TSEncoding.PLAIN)));
+    planList.add(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.vehicle.d1.s2"),
+            new MeasurementSchema("s2", TSDataType.FLOAT, TSEncoding.RLE)));
+    planList.add(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.vehicle.d1.s3"),
+            new MeasurementSchema("s3", TSDataType.BOOLEAN, TSEncoding.PLAIN)));
+    planList.add(new SetStorageGroupPlan(new PartialPath("root.sg1")));
+    planList.add(
+        new CreateAlignedTimeSeriesPlan(
+            new PartialPath("root.sg1.d1"),
+            Arrays.asList("s1", "s2", "s3", "s4", "s5"),
+            Arrays.asList(
+                TSDataType.FLOAT,
+                TSDataType.INT32,
+                TSDataType.INT64,
+                TSDataType.BOOLEAN,
+                TSDataType.TEXT),
+            Arrays.asList(
+                TSEncoding.RLE,
+                TSEncoding.GORILLA,
+                TSEncoding.RLE,
+                TSEncoding.RLE,
+                TSEncoding.PLAIN),
+            Arrays.asList(
+                CompressionType.SNAPPY,
+                CompressionType.SNAPPY,
+                CompressionType.SNAPPY,
+                CompressionType.SNAPPY,
+                CompressionType.SNAPPY),
+            null,
+            null,
+            null));
+    for (PhysicalPlan plan : planList) {
+      PipeData pipeData = new SchemaPipeData(plan, serialNum++);
+      pipeData.serialize(pipeLogOutput);
+    }
+    pipeLogOutput.close();
+    File pipeLog2 = new File(pipeLogDir1.getPath(), SyncConstant.getPipeLogName(serialNum));
+    pipeLogOutput = new DataOutputStream(new FileOutputStream(pipeLog2, false));
+    List<File> tsFiles = SyncTestUtil.getTsFilePaths(tmpDir);
+    for (File f : tsFiles) {
+      PipeData pipeData = new TsFilePipeData(f.getPath(), serialNum++);
+      pipeData.serialize(pipeLogOutput);
+    }
+    pipeLogOutput.close();
+
+    Deletion deletion = new Deletion(new PartialPath("root.vehicle.**"), 0, 33, 38);
+    PipeData pipeData = new DeletionPipeData(deletion, serialNum++);
+    BufferedPipeDataQueue pipeDataQueue =
+        PipeDataQueueFactory.getBufferedPipeDataQueue(
+            SyncPathUtil.getReceiverPipeLogDir(pipeName1, remoteIp1, createdTime1));
+    pipeDataQueue.offer(pipeData);
+
+    // 3. create and start collector
+    Collector collector = new Collector();
+    collector.startCollect();
+
+    // 4. start collect pipe
+    collector.startPipe(pipeName1, remoteIp1, createdTime1);
+
+    // 5. if all pipeData has been loaded into IoTDB, check result
+    CountDownLatch latch = new CountDownLatch(1);
+    ExecutorService es1 = Executors.newSingleThreadExecutor();
+    int finalSerialNum = serialNum - 1;
+    es1.execute(
+        () -> {
+          while (true) {
+            if (pipeDataQueue.getCommitSerialNumber() == finalSerialNum) {
+              break;
+            }
+            try {
+              Thread.sleep(100);
+            } catch (InterruptedException e) {
+              e.printStackTrace();
+            }
+          }
+          latch.countDown();
+        });
+    es1.shutdown();
+    try {
+      latch.await(30, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+    String sql1 = "select * from root.vehicle.*";
+    String[] retArray1 =
+        new String[] {
+          "6,120,null,null,null",
+          "9,null,123,null,null",
+          "16,128,null,null,16.0",
+          "18,189,198,true,18.0",
+          "20,null,null,false,null",
+          "29,null,null,true,1205.0",
+          "99,null,1234,null,null"
+        };
+    String[] columnNames1 = {
+      "root.vehicle.d0.s0", "root.vehicle.d0.s1", "root.vehicle.d1.s3", "root.vehicle.d1.s2"
+    };
+    SyncTestUtil.checkResult(sql1, columnNames1, retArray1);
+    // 5.2 check aligned timeseries
+    String sql2 = "select * from root.sg1.d1";
+    String[] retArray2 =
+        new String[] {
+          "1,1.0,1,null,true,aligned_test1",
+          "2,2.0,2,null,null,aligned_test2",
+          "3,3.0,null,null,false,aligned_test3",
+          "4,4.0,4,null,true,aligned_test4",
+          "5,130000.0,130000,130000,false,aligned_unseq_test1",
+          "6,6.0,6,6,true,null",
+          "7,7.0,7,7,false,aligned_test7",
+          "8,8.0,8,8,null,aligned_test8",
+          "9,9.0,9,9,false,aligned_test9",
+        };
+    String[] columnNames2 = {
+      "root.sg1.d1.s1", "root.sg1.d1.s2", "root.sg1.d1.s3", "root.sg1.d1.s4", "root.sg1.d1.s5",
+    };
+    SyncTestUtil.checkResult(sql2, columnNames2, retArray2);
+
+    // 6. stop pipe, check interrupt thread
+    collector.stopPipe(pipeName1, remoteIp1, createdTime1);
+    Thread.sleep(1000);
+    Deletion deletion1 = new Deletion(new PartialPath("root.vehicle.**"), 0, 0, 99);
+    PipeData pipeData1 = new DeletionPipeData(deletion1, serialNum++);
+    pipeDataQueue.offer(pipeData1);
+    Thread.sleep(1000);
+    SyncTestUtil.checkResult(sql1, columnNames1, retArray1);
+
+    // 7. stop collector, check release thread pool
+    collector.stopCollect();
+    Thread.sleep(1000);
+    for (Thread t : Thread.getAllStackTraces().keySet()) {
+      if (t.getName().contains(ThreadName.SYNC_RECEIVER_COLLECTOR.getName())) {
+        Assert.fail();
+      }
+    }
+    pipeDataQueue.clear();
+  }
+
+  @Test
+  public void testMultiplePipe() throws Exception {
+    // 1. restart IoTDB
+    EnvironmentUtils.cleanEnv();
+    EnvironmentUtils.envSetUp();
+
+    // 2. prepare pipelog and pipeDataQueue
+    if (!pipeLogDir1.exists()) {
+      pipeLogDir1.mkdirs();
+    }
+    if (!pipeLogDir2.exists()) {
+      pipeLogDir2.mkdirs();
+    }
+    // 2.1 prepare for pipe1
+    DataOutputStream outputStream =
+        new DataOutputStream(
+            new FileOutputStream(new File(pipeLogDir1, SyncConstant.COMMIT_LOG_NAME), true));
+    outputStream.writeLong(-1);
+    outputStream.close();
+    int serialNum1 = 0;
+    File pipeLog1 = new File(pipeLogDir1.getPath(), SyncConstant.getPipeLogName(serialNum1));
+    DataOutputStream pipeLogOutput = new DataOutputStream(new FileOutputStream(pipeLog1, false));
+    List<PhysicalPlan> planList = new ArrayList<>();
+    planList.add(new SetStorageGroupPlan(new PartialPath("root.vehicle")));
+    planList.add(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.vehicle.d0.s0"),
+            new MeasurementSchema("s0", TSDataType.INT32, TSEncoding.RLE)));
+    planList.add(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.vehicle.d0.s1"),
+            new MeasurementSchema("s1", TSDataType.TEXT, TSEncoding.PLAIN)));
+    planList.add(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.vehicle.d1.s2"),
+            new MeasurementSchema("s2", TSDataType.FLOAT, TSEncoding.RLE)));
+    planList.add(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.vehicle.d1.s3"),
+            new MeasurementSchema("s3", TSDataType.BOOLEAN, TSEncoding.PLAIN)));
+    for (PhysicalPlan plan : planList) {
+      PipeData pipeData = new SchemaPipeData(plan, serialNum1++);
+      pipeData.serialize(pipeLogOutput);
+    }
+    pipeLogOutput.close();
+    File pipeLog2 = new File(pipeLogDir1.getPath(), SyncConstant.getPipeLogName(serialNum1));
+    pipeLogOutput = new DataOutputStream(new FileOutputStream(pipeLog2, false));
+    List<File> tsFiles =
+        SyncTestUtil.getTsFilePaths(new File(tmpDir, "sequence" + File.separator + "root.vehicle"));
+    for (File f : tsFiles) {
+      PipeData pipeData = new TsFilePipeData(f.getPath(), serialNum1++);
+      pipeData.serialize(pipeLogOutput);
+    }
+    tsFiles =
+        SyncTestUtil.getTsFilePaths(
+            new File(tmpDir, "unsequence" + File.separator + "root.vehicle"));
+    for (File f : tsFiles) {
+      PipeData pipeData = new TsFilePipeData(f.getPath(), serialNum1++);
+      pipeData.serialize(pipeLogOutput);
+    }
+    Deletion deletion = new Deletion(new PartialPath("root.vehicle.**"), 0, 33, 38);
+    PipeData pipeData = new DeletionPipeData(deletion, serialNum1++);
+    pipeData.serialize(pipeLogOutput);
+    pipeLogOutput.close();
+
+    // 2.2 prepare for pipe2
+    int serialNum2 = 0;
+    outputStream =
+        new DataOutputStream(
+            new FileOutputStream(new File(pipeLogDir2, SyncConstant.COMMIT_LOG_NAME), true));
+    outputStream.writeLong(-1);
+    outputStream.close();
+    pipeLog1 = new File(pipeLogDir2.getPath(), SyncConstant.getPipeLogName(serialNum2));
+    pipeLogOutput = new DataOutputStream(new FileOutputStream(pipeLog1, false));
+    pipeData =
+        new SchemaPipeData(new SetStorageGroupPlan(new PartialPath("root.sg1")), serialNum2++);
+    pipeData.serialize(pipeLogOutput);
+    pipeData =
+        new SchemaPipeData(
+            new CreateAlignedTimeSeriesPlan(
+                new PartialPath("root.sg1.d1"),
+                Arrays.asList("s1", "s2", "s3", "s4", "s5"),
+                Arrays.asList(
+                    TSDataType.FLOAT,
+                    TSDataType.INT32,
+                    TSDataType.INT64,
+                    TSDataType.BOOLEAN,
+                    TSDataType.TEXT),
+                Arrays.asList(
+                    TSEncoding.RLE,
+                    TSEncoding.GORILLA,
+                    TSEncoding.RLE,
+                    TSEncoding.RLE,
+                    TSEncoding.PLAIN),
+                Arrays.asList(
+                    CompressionType.SNAPPY,
+                    CompressionType.SNAPPY,
+                    CompressionType.SNAPPY,
+                    CompressionType.SNAPPY,
+                    CompressionType.SNAPPY),
+                null,
+                null,
+                null),
+            serialNum2++);
+    pipeData.serialize(pipeLogOutput);
+    pipeLogOutput.close();
+    pipeLog2 = new File(pipeLogDir2.getPath(), SyncConstant.getPipeLogName(serialNum2));
+    pipeLogOutput = new DataOutputStream(new FileOutputStream(pipeLog2, false));
+    tsFiles =
+        SyncTestUtil.getTsFilePaths(new File(tmpDir, "sequence" + File.separator + "root.sg1"));
+    for (File f : tsFiles) {
+      pipeData = new TsFilePipeData(f.getPath(), serialNum2++);
+      pipeData.serialize(pipeLogOutput);
+    }
+    tsFiles =
+        SyncTestUtil.getTsFilePaths(new File(tmpDir, "unsequence" + File.separator + "root.sg1"));
+    for (File f : tsFiles) {
+      pipeData = new TsFilePipeData(f.getPath(), serialNum2++);
+      pipeData.serialize(pipeLogOutput);
+    }
+    pipeLogOutput.close();
+
+    // 3. create and start collector
+    BufferedPipeDataQueue pipeDataQueue1 =
+        PipeDataQueueFactory.getBufferedPipeDataQueue(
+            SyncPathUtil.getReceiverPipeLogDir(pipeName1, remoteIp1, createdTime1));
+    BufferedPipeDataQueue pipeDataQueue2 =
+        PipeDataQueueFactory.getBufferedPipeDataQueue(
+            SyncPathUtil.getReceiverPipeLogDir(pipeName2, remoteIp2, createdTime2));
+    Collector collector = new Collector();
+    collector.startCollect();
+
+    // 4. start collect pipe
+    collector.startPipe(pipeName1, remoteIp1, createdTime1);
+    collector.startPipe(pipeName2, remoteIp2, createdTime2);
+
+    // 5. if all pipeData has been loaded into IoTDB, check result
+    CountDownLatch latch = new CountDownLatch(2);
+    ExecutorService es1 = Executors.newSingleThreadExecutor();
+    int finalSerialNum1 = serialNum1 - 1;
+    int finalSerialNum2 = serialNum2 - 1;
+    es1.execute(
+        () -> {
+          while (true) {
+            if (pipeDataQueue1.getCommitSerialNumber() == finalSerialNum1) {
+              break;
+            }
+            try {
+              Thread.sleep(100);
+            } catch (InterruptedException e) {
+              e.printStackTrace();
+            }
+          }
+          latch.countDown();
+          while (true) {
+            if (pipeDataQueue2.getCommitSerialNumber() == finalSerialNum2) {
+              break;
+            }
+            try {
+              Thread.sleep(100);
+            } catch (InterruptedException e) {
+              e.printStackTrace();
+            }
+          }
+          latch.countDown();
+        });
+    es1.shutdown();
+    try {
+      latch.await(30, TimeUnit.SECONDS);
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+    String sql1 = "select * from root.vehicle.*";
+    String[] retArray1 =
+        new String[] {
+          "6,120,null,null,null",
+          "9,null,123,null,null",
+          "16,128,null,null,16.0",
+          "18,189,198,true,18.0",
+          "20,null,null,false,null",
+          "29,null,null,true,1205.0",
+          "99,null,1234,null,null"
+        };
+    String[] columnNames1 = {
+      "root.vehicle.d0.s0", "root.vehicle.d0.s1", "root.vehicle.d1.s3", "root.vehicle.d1.s2"
+    };
+    SyncTestUtil.checkResult(sql1, columnNames1, retArray1);
+    // 5.2 check aligned timeseries
+    String sql2 = "select * from root.sg1.d1";
+    String[] retArray2 =
+        new String[] {
+          "1,1.0,1,null,true,aligned_test1",
+          "2,2.0,2,null,null,aligned_test2",
+          "3,3.0,null,null,false,aligned_test3",
+          "4,4.0,4,null,true,aligned_test4",
+          "5,130000.0,130000,130000,false,aligned_unseq_test1",
+          "6,6.0,6,6,true,null",
+          "7,7.0,7,7,false,aligned_test7",
+          "8,8.0,8,8,null,aligned_test8",
+          "9,9.0,9,9,false,aligned_test9",
+        };
+    String[] columnNames2 = {
+      "root.sg1.d1.s1", "root.sg1.d1.s2", "root.sg1.d1.s3", "root.sg1.d1.s4", "root.sg1.d1.s5",
+    };
+    SyncTestUtil.checkResult(sql2, columnNames2, retArray2);
+
+    // 6. stop pipe, check interrupt thread
+    collector.stopPipe(pipeName1, remoteIp1, createdTime1);
+    collector.stopPipe(pipeName2, remoteIp2, createdTime2);
+    Thread.sleep(1000);
+    Deletion deletion1 = new Deletion(new PartialPath("root.vehicle.**"), 0, 0, 99);
+    PipeData pipeData1 = new DeletionPipeData(deletion1, serialNum1++);
+    pipeDataQueue1.offer(pipeData1);
+    Thread.sleep(1000);
+    SyncTestUtil.checkResult(sql1, columnNames1, retArray1);
+
+    // 7. stop collector, check release thread pool
+    collector.stopCollect();
+    Thread.sleep(1000);
+    for (Thread t : Thread.getAllStackTraces().keySet()) {
+      if (t.getName().contains(ThreadName.SYNC_RECEIVER_COLLECTOR.getName())) {
+        Assert.fail();
+      }
+    }
+    pipeDataQueue2.clear();
+    pipeDataQueue1.clear();
+  }
+}
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/sync/IoTDBSyncReceiverIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/sync/IoTDBSyncReceiverIT.java
new file mode 100644
index 0000000000..736b2954d3
--- /dev/null
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/sync/IoTDBSyncReceiverIT.java
@@ -0,0 +1,357 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.integration.sync;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.exception.sync.PipeServerException;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.*;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.db.sync.pipedata.DeletionPipeData;
+import org.apache.iotdb.db.sync.pipedata.PipeData;
+import org.apache.iotdb.db.sync.pipedata.SchemaPipeData;
+import org.apache.iotdb.db.sync.pipedata.TsFilePipeData;
+import org.apache.iotdb.db.sync.receiver.ReceiverService;
+import org.apache.iotdb.db.sync.sender.pipe.Pipe;
+import org.apache.iotdb.db.sync.sender.pipe.Pipe.PipeStatus;
+import org.apache.iotdb.db.sync.sender.pipe.TsFilePipe;
+import org.apache.iotdb.db.sync.transport.client.TransportClient;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+import org.apache.iotdb.service.transport.thrift.RequestType;
+import org.apache.iotdb.service.transport.thrift.ResponseType;
+import org.apache.iotdb.service.transport.thrift.SyncRequest;
+import org.apache.iotdb.service.transport.thrift.SyncResponse;
+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.write.schema.MeasurementSchema;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+@Category({LocalStandaloneTest.class})
+public class IoTDBSyncReceiverIT {
+  private static final Logger logger = LoggerFactory.getLogger(IoTDBSyncReceiverIT.class);
+
+  /** create tsfile and move to tmpDir for sync test */
+  File tmpDir = new File("target/synctest");
+
+  protected static boolean enableSeqSpaceCompaction;
+  protected static boolean enableUnseqSpaceCompaction;
+  protected static boolean enableCrossSpaceCompaction;
+
+  String pipeName1 = "pipe1";
+  String remoteIp1;
+  long createdTime1 = System.currentTimeMillis();
+
+  TransportClient client;
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.envSetUp();
+    enableSeqSpaceCompaction =
+        IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
+    enableUnseqSpaceCompaction =
+        IoTDBDescriptor.getInstance().getConfig().isEnableUnseqSpaceCompaction();
+    enableCrossSpaceCompaction =
+        IoTDBDescriptor.getInstance().getConfig().isEnableCrossSpaceCompaction();
+    IoTDBDescriptor.getInstance().getConfig().setEnableSeqSpaceCompaction(false);
+    IoTDBDescriptor.getInstance().getConfig().setEnableUnseqSpaceCompaction(false);
+    IoTDBDescriptor.getInstance().getConfig().setEnableCrossSpaceCompaction(false);
+    SyncTestUtil.insertData();
+    EnvironmentUtils.shutdownDaemon();
+    File srcDir = new File(IoTDBDescriptor.getInstance().getConfig().getDataDirs()[0]);
+    if (tmpDir.exists()) {
+      FileUtils.deleteDirectory(tmpDir);
+    }
+    FileUtils.moveDirectory(srcDir, tmpDir);
+    EnvironmentUtils.cleanEnv();
+    EnvironmentUtils.envSetUp();
+    try {
+      ReceiverService.getInstance().startPipeServer();
+      new Socket("localhost", 6670).close();
+    } catch (Exception e) {
+      Assert.fail("Failed to start pipe server because " + e.getMessage());
+    }
+    Pipe pipe = new TsFilePipe(createdTime1, pipeName1, null, 0, false);
+    client = new TransportClient(pipe, "127.0.0.1", 6670);
+    remoteIp1 = InetAddress.getLocalHost().getHostAddress();
+    client.handshake();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    IoTDBDescriptor.getInstance().getConfig().setEnableSeqSpaceCompaction(enableSeqSpaceCompaction);
+    IoTDBDescriptor.getInstance()
+        .getConfig()
+        .setEnableUnseqSpaceCompaction(enableUnseqSpaceCompaction);
+    IoTDBDescriptor.getInstance()
+        .getConfig()
+        .setEnableCrossSpaceCompaction(enableCrossSpaceCompaction);
+    FileUtils.deleteDirectory(tmpDir);
+    client.close();
+    EnvironmentUtils.cleanEnv();
+  }
+
+  /** cannot stop */
+  @Test
+  public void testStopPipeServerCheck() {
+    logger.info("testStopPipeServerCheck");
+    ReceiverService.getInstance()
+        .receiveMsg(new SyncRequest(RequestType.CREATE, pipeName1, remoteIp1, createdTime1));
+    ReceiverService.getInstance()
+        .receiveMsg(new SyncRequest(RequestType.START, pipeName1, remoteIp1, createdTime1));
+    try {
+      ReceiverService.getInstance().stopPipeServer();
+      Assert.fail("Should not stop pipe server");
+    } catch (PipeServerException e) {
+      // nothing
+    }
+    ReceiverService.getInstance()
+        .receiveMsg(new SyncRequest(RequestType.DROP, pipeName1, remoteIp1, createdTime1));
+  }
+
+  @Test
+  public void testPipeOperation() {
+    logger.info("testPipeOperation");
+    String[] columnNames = {"create time", "name", "role", "remote", "status", "message"};
+    String showPipeSql = "SHOW PIPE";
+    try {
+      // create
+      client.heartbeat(new SyncRequest(RequestType.CREATE, pipeName1, remoteIp1, createdTime1));
+      String[] retArray =
+          new String[] {
+            String.format(
+                "%s,%s,%s,%s,%s,%s",
+                DatetimeUtils.convertLongToDate(createdTime1),
+                pipeName1,
+                "receiver",
+                remoteIp1,
+                PipeStatus.STOP.name(),
+                "")
+          };
+      SyncTestUtil.checkResult(showPipeSql, columnNames, retArray, false);
+      // start
+      client.heartbeat(new SyncRequest(RequestType.START, pipeName1, remoteIp1, createdTime1));
+      retArray =
+          new String[] {
+            String.format(
+                "%s,%s,%s,%s,%s,%s",
+                DatetimeUtils.convertLongToDate(createdTime1),
+                pipeName1,
+                "receiver",
+                remoteIp1,
+                PipeStatus.RUNNING.name(),
+                "")
+          };
+      SyncTestUtil.checkResult(showPipeSql, columnNames, retArray, false);
+      // stop
+      client.heartbeat(new SyncRequest(RequestType.STOP, pipeName1, remoteIp1, createdTime1));
+      retArray =
+          new String[] {
+            String.format(
+                "%s,%s,%s,%s,%s,%s",
+                DatetimeUtils.convertLongToDate(createdTime1),
+                pipeName1,
+                "receiver",
+                remoteIp1,
+                PipeStatus.STOP.name(),
+                "")
+          };
+      SyncTestUtil.checkResult(showPipeSql, columnNames, retArray, false);
+      // drop
+      client.heartbeat(new SyncRequest(RequestType.DROP, pipeName1, remoteIp1, createdTime1));
+      retArray =
+          new String[] {
+            String.format(
+                "%s,%s,%s,%s,%s,%s",
+                DatetimeUtils.convertLongToDate(createdTime1),
+                pipeName1,
+                "receiver",
+                remoteIp1,
+                PipeStatus.DROP.name(),
+                "")
+          };
+      SyncTestUtil.checkResult(showPipeSql, columnNames, retArray, false);
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testReceiveDataAndLoad() {
+    logger.info("testReceiveDataAndLoad");
+    try {
+      // 1. create pipe
+      client.heartbeat(new SyncRequest(RequestType.CREATE, pipeName1, remoteIp1, createdTime1));
+      client.heartbeat(new SyncRequest(RequestType.START, pipeName1, remoteIp1, createdTime1));
+
+      // 2. send pipe data
+      int serialNum = 0;
+      List<PhysicalPlan> planList = new ArrayList<>();
+      planList.add(new SetStorageGroupPlan(new PartialPath("root.vehicle")));
+      planList.add(
+          new CreateTimeSeriesPlan(
+              new PartialPath("root.vehicle.d0.s0"),
+              new MeasurementSchema("s0", TSDataType.INT32, TSEncoding.RLE)));
+      planList.add(
+          new CreateTimeSeriesPlan(
+              new PartialPath("root.vehicle.d0.s1"),
+              new MeasurementSchema("s1", TSDataType.TEXT, TSEncoding.PLAIN)));
+      planList.add(
+          new CreateTimeSeriesPlan(
+              new PartialPath("root.vehicle.d1.s2"),
+              new MeasurementSchema("s2", TSDataType.FLOAT, TSEncoding.RLE)));
+      planList.add(
+          new CreateTimeSeriesPlan(
+              new PartialPath("root.vehicle.d1.s3"),
+              new MeasurementSchema("s3", TSDataType.BOOLEAN, TSEncoding.PLAIN)));
+      planList.add(new SetStorageGroupPlan(new PartialPath("root.sg1")));
+      planList.add(
+          new CreateAlignedTimeSeriesPlan(
+              new PartialPath("root.sg1.d1"),
+              Arrays.asList("s1", "s2", "s3", "s4", "s5"),
+              Arrays.asList(
+                  TSDataType.FLOAT,
+                  TSDataType.INT32,
+                  TSDataType.INT64,
+                  TSDataType.BOOLEAN,
+                  TSDataType.TEXT),
+              Arrays.asList(
+                  TSEncoding.RLE,
+                  TSEncoding.GORILLA,
+                  TSEncoding.RLE,
+                  TSEncoding.RLE,
+                  TSEncoding.PLAIN),
+              Arrays.asList(
+                  CompressionType.SNAPPY,
+                  CompressionType.SNAPPY,
+                  CompressionType.SNAPPY,
+                  CompressionType.SNAPPY,
+                  CompressionType.SNAPPY),
+              null,
+              null,
+              null));
+      planList.add(new SetStorageGroupPlan(new PartialPath("root.sg1")));
+      for (PhysicalPlan plan : planList) {
+        client.senderTransport(new SchemaPipeData(plan, serialNum++));
+      }
+      List<File> tsFiles = SyncTestUtil.getTsFilePaths(tmpDir);
+      for (File f : tsFiles) {
+        client.senderTransport(new TsFilePipeData(f.getPath(), serialNum++));
+      }
+      Deletion deletion = new Deletion(new PartialPath("root.vehicle.**"), 0, 33, 38);
+      PipeData pipeData = new DeletionPipeData(deletion, serialNum++);
+      client.senderTransport(pipeData);
+
+      // wait collector to load pipe data
+      Thread.sleep(1000);
+
+      // 3. check result
+      String sql1 = "select * from root.vehicle.*";
+      String[] retArray1 =
+          new String[] {
+            "6,120,null,null,null",
+            "9,null,123,null,null",
+            "16,128,null,null,16.0",
+            "18,189,198,true,18.0",
+            "20,null,null,false,null",
+            "29,null,null,true,1205.0",
+            "99,null,1234,null,null"
+          };
+      String[] columnNames1 = {
+        "root.vehicle.d0.s0", "root.vehicle.d0.s1", "root.vehicle.d1.s3", "root.vehicle.d1.s2"
+      };
+      SyncTestUtil.checkResult(sql1, columnNames1, retArray1);
+      String sql2 = "select * from root.sg1.d1";
+      String[] retArray2 =
+          new String[] {
+            "1,1.0,1,null,true,aligned_test1",
+            "2,2.0,2,null,null,aligned_test2",
+            "3,3.0,null,null,false,aligned_test3",
+            "4,4.0,4,null,true,aligned_test4",
+            "5,130000.0,130000,130000,false,aligned_unseq_test1",
+            "6,6.0,6,6,true,null",
+            "7,7.0,7,7,false,aligned_test7",
+            "8,8.0,8,8,null,aligned_test8",
+            "9,9.0,9,9,false,aligned_test9",
+          };
+      String[] columnNames2 = {
+        "root.sg1.d1.s1", "root.sg1.d1.s2", "root.sg1.d1.s3", "root.sg1.d1.s4", "root.sg1.d1.s5",
+      };
+      SyncTestUtil.checkResult(sql2, columnNames2, retArray2);
+
+      // 4. stop pipe
+      client.heartbeat(new SyncRequest(RequestType.STOP, pipeName1, remoteIp1, createdTime1));
+      Thread.sleep(500);
+      client.senderTransport(
+          new DeletionPipeData(
+              new Deletion(new PartialPath("root.vehicle.**"), 0, 0, 99), serialNum++));
+      Thread.sleep(1000);
+      SyncTestUtil.checkResult(sql1, columnNames1, retArray1);
+      // check heartbeat
+      SyncResponse response1 =
+          ReceiverService.getInstance()
+              .receiveMsg(
+                  new SyncRequest(RequestType.HEARTBEAT, pipeName1, remoteIp1, createdTime1));
+      Assert.assertEquals(ResponseType.WARN, response1.type);
+
+      // 5. restart pipe
+      client.heartbeat(new SyncRequest(RequestType.START, pipeName1, remoteIp1, createdTime1));
+      Thread.sleep(1000);
+      SyncTestUtil.checkResult(sql1, columnNames1, new String[] {});
+      // check heartbeat
+      SyncResponse response2 =
+          ReceiverService.getInstance()
+              .receiveMsg(
+                  new SyncRequest(RequestType.HEARTBEAT, pipeName1, remoteIp1, createdTime1));
+      Assert.assertEquals(ResponseType.INFO, response2.type);
+
+      // 6. drop pipe
+      client.heartbeat(new SyncRequest(RequestType.DROP, pipeName1, remoteIp1, createdTime1));
+      Thread.sleep(500);
+      client.senderTransport(
+          new DeletionPipeData(
+              new Deletion(new PartialPath("root.sg1.**"), 0, 0, 99), serialNum++));
+      Thread.sleep(1000);
+      SyncTestUtil.checkResult(sql2, columnNames2, retArray2);
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.fail(e.getMessage());
+    }
+  }
+}
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/sync/IoTDBSyncReceiverLoaderIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/sync/IoTDBSyncReceiverLoaderIT.java
new file mode 100644
index 0000000000..0ec5011bb1
--- /dev/null
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/sync/IoTDBSyncReceiverLoaderIT.java
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.integration.sync;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.db.sync.receiver.load.DeletionLoader;
+import org.apache.iotdb.db.sync.receiver.load.ILoader;
+import org.apache.iotdb.db.sync.receiver.load.SchemaLoader;
+import org.apache.iotdb.db.sync.receiver.load.TsFileLoader;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+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.write.schema.MeasurementSchema;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.*;
+
+@Category({LocalStandaloneTest.class})
+public class IoTDBSyncReceiverLoaderIT {
+  private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBSyncReceiverLoaderIT.class);
+  protected static boolean enableSeqSpaceCompaction;
+  protected static boolean enableUnseqSpaceCompaction;
+  protected static boolean enableCrossSpaceCompaction;
+  /** create tsfile and move to tmpDir for sync test */
+  File tmpDir = new File("target/synctest");
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.envSetUp();
+    enableSeqSpaceCompaction =
+        IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
+    enableUnseqSpaceCompaction =
+        IoTDBDescriptor.getInstance().getConfig().isEnableUnseqSpaceCompaction();
+    enableCrossSpaceCompaction =
+        IoTDBDescriptor.getInstance().getConfig().isEnableCrossSpaceCompaction();
+    IoTDBDescriptor.getInstance().getConfig().setEnableSeqSpaceCompaction(false);
+    IoTDBDescriptor.getInstance().getConfig().setEnableUnseqSpaceCompaction(false);
+    IoTDBDescriptor.getInstance().getConfig().setEnableCrossSpaceCompaction(false);
+    SyncTestUtil.insertData();
+    EnvironmentUtils.shutdownDaemon();
+    File srcDir = new File(IoTDBDescriptor.getInstance().getConfig().getDataDirs()[0]);
+    FileUtils.moveDirectory(srcDir, tmpDir);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    IoTDBDescriptor.getInstance().getConfig().setEnableSeqSpaceCompaction(enableSeqSpaceCompaction);
+    IoTDBDescriptor.getInstance()
+        .getConfig()
+        .setEnableUnseqSpaceCompaction(enableUnseqSpaceCompaction);
+    IoTDBDescriptor.getInstance()
+        .getConfig()
+        .setEnableCrossSpaceCompaction(enableCrossSpaceCompaction);
+    FileUtils.deleteDirectory(tmpDir);
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void test() throws Exception {
+    // 1. restart IoTDB
+    EnvironmentUtils.cleanEnv();
+    EnvironmentUtils.envSetUp();
+
+    // 2. test for SchemaLoader
+    List<PhysicalPlan> planList = new ArrayList<>();
+    planList.add(new SetStorageGroupPlan(new PartialPath("root.vehicle")));
+    planList.add(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.vehicle.d0.s0"),
+            new MeasurementSchema("s0", TSDataType.INT32, TSEncoding.RLE)));
+    planList.add(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.vehicle.d0.s1"),
+            new MeasurementSchema("s1", TSDataType.TEXT, TSEncoding.PLAIN)));
+    planList.add(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.vehicle.d1.s2"),
+            new MeasurementSchema("s2", TSDataType.FLOAT, TSEncoding.RLE)));
+    planList.add(
+        new CreateTimeSeriesPlan(
+            new PartialPath("root.vehicle.d1.s3"),
+            new MeasurementSchema("s3", TSDataType.BOOLEAN, TSEncoding.PLAIN)));
+    planList.add(new SetStorageGroupPlan(new PartialPath("root.sg1")));
+    planList.add(
+        new CreateAlignedTimeSeriesPlan(
+            new PartialPath("root.sg1.d1"),
+            Arrays.asList("s1", "s2", "s3", "s4", "s5"),
+            Arrays.asList(
+                TSDataType.FLOAT,
+                TSDataType.INT32,
+                TSDataType.INT64,
+                TSDataType.BOOLEAN,
+                TSDataType.TEXT),
+            Arrays.asList(
+                TSEncoding.RLE,
+                TSEncoding.GORILLA,
+                TSEncoding.RLE,
+                TSEncoding.RLE,
+                TSEncoding.PLAIN),
+            Arrays.asList(
+                CompressionType.SNAPPY,
+                CompressionType.SNAPPY,
+                CompressionType.SNAPPY,
+                CompressionType.SNAPPY,
+                CompressionType.SNAPPY),
+            null,
+            null,
+            null));
+    for (PhysicalPlan plan : planList) {
+      ILoader planLoader = new SchemaLoader(plan);
+      try {
+        planLoader.load();
+      } catch (Exception e) {
+        e.printStackTrace();
+        Assert.fail();
+      }
+    }
+
+    // 3. test for TsFileLoader
+    List<File> tsFiles = SyncTestUtil.getTsFilePaths(tmpDir);
+    for (File tsfile : tsFiles) {
+      ILoader tsFileLoader = new TsFileLoader(tsfile);
+      try {
+        tsFileLoader.load();
+      } catch (Exception e) {
+        e.printStackTrace();
+        Assert.fail();
+      }
+    }
+
+    // 4. test for DeletionPlanLoader
+    Deletion deletion = new Deletion(new PartialPath("root.vehicle.**"), 0, 33, 38);
+    ILoader deletionLoader = new DeletionLoader(deletion);
+    try {
+      deletionLoader.load();
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.fail();
+    }
+
+    // 5. check result after loading
+    // 5.1 check normal timeseries
+    String sql1 = "select * from root.vehicle.*";
+    String[] retArray1 =
+        new String[] {
+          "6,120,null,null,null",
+          "9,null,123,null,null",
+          "16,128,null,null,16.0",
+          "18,189,198,true,18.0",
+          "20,null,null,false,null",
+          "29,null,null,true,1205.0",
+          "99,null,1234,null,null"
+        };
+    String[] columnNames1 = {
+      "root.vehicle.d0.s0", "root.vehicle.d0.s1", "root.vehicle.d1.s3", "root.vehicle.d1.s2"
+    };
+    SyncTestUtil.checkResult(sql1, columnNames1, retArray1);
+    // 5.2 check aligned timeseries
+    String sql2 = "select * from root.sg1.d1";
+    String[] retArray2 =
+        new String[] {
+          "1,1.0,1,null,true,aligned_test1",
+          "2,2.0,2,null,null,aligned_test2",
+          "3,3.0,null,null,false,aligned_test3",
+          "4,4.0,4,null,true,aligned_test4",
+          "5,130000.0,130000,130000,false,aligned_unseq_test1",
+          "6,6.0,6,6,true,null",
+          "7,7.0,7,7,false,aligned_test7",
+          "8,8.0,8,8,null,aligned_test8",
+          "9,9.0,9,9,false,aligned_test9",
+        };
+    String[] columnNames2 = {
+      "root.sg1.d1.s1", "root.sg1.d1.s2", "root.sg1.d1.s3", "root.sg1.d1.s4", "root.sg1.d1.s5",
+    };
+    SyncTestUtil.checkResult(sql2, columnNames2, retArray2);
+  }
+}
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/sync/IoTDBSyncSenderIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/sync/IoTDBSyncSenderIT.java
new file mode 100644
index 0000000000..c9fa74c7f7
--- /dev/null
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/sync/IoTDBSyncSenderIT.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.integration.sync;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.sync.pipedata.PipeData;
+import org.apache.iotdb.db.sync.pipedata.SchemaPipeData;
+import org.apache.iotdb.db.sync.pipedata.TsFilePipeData;
+import org.apache.iotdb.db.sync.sender.pipe.TsFilePipe;
+import org.apache.iotdb.db.sync.sender.service.SenderService;
+import org.apache.iotdb.db.sync.sender.service.TransportHandler;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+import org.apache.iotdb.jdbc.Config;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+import java.util.List;
+
+@Category({LocalStandaloneTest.class})
+public class IoTDBSyncSenderIT {
+  private boolean enableSeqSpaceCompaction;
+  private boolean enableUnseqSpaceCompaction;
+  private boolean enableCrossSpaceCompaction;
+
+  private static final String pipeSinkName = "test_pipesink";
+  private static final String pipeName = "test_pipe";
+
+  private TsFilePipe pipe;
+  private TransportClientMock mock;
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.envSetUp();
+    enableSeqSpaceCompaction =
+        IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
+    enableUnseqSpaceCompaction =
+        IoTDBDescriptor.getInstance().getConfig().isEnableUnseqSpaceCompaction();
+    enableCrossSpaceCompaction =
+        IoTDBDescriptor.getInstance().getConfig().isEnableCrossSpaceCompaction();
+    IoTDBDescriptor.getInstance().getConfig().setEnableSeqSpaceCompaction(false);
+    IoTDBDescriptor.getInstance().getConfig().setEnableUnseqSpaceCompaction(false);
+    IoTDBDescriptor.getInstance().getConfig().setEnableCrossSpaceCompaction(false);
+    Class.forName(Config.JDBC_DRIVER_NAME);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    IoTDBDescriptor.getInstance().getConfig().setEnableSeqSpaceCompaction(enableSeqSpaceCompaction);
+    IoTDBDescriptor.getInstance()
+        .getConfig()
+        .setEnableUnseqSpaceCompaction(enableUnseqSpaceCompaction);
+    IoTDBDescriptor.getInstance()
+        .getConfig()
+        .setEnableCrossSpaceCompaction(enableCrossSpaceCompaction);
+    EnvironmentUtils.shutdownDaemon();
+    EnvironmentUtils.cleanEnv();
+  }
+
+  private void prepareSchema() throws Exception {
+    try (Connection connection =
+            DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      statement.execute("set storage group to root.sg1");
+      statement.execute("set storage group to root.sg2");
+      statement.execute("create timeseries root.sg1.d1.s1 with datatype=int32, encoding=PLAIN");
+      statement.execute("create timeseries root.sg1.d1.s2 with datatype=float, encoding=RLE");
+      statement.execute("create timeseries root.sg1.d1.s3 with datatype=TEXT, encoding=PLAIN");
+      statement.execute("create timeseries root.sg1.d2.s4 with datatype=int64, encoding=PLAIN");
+      statement.execute("create timeseries root.sg2.d1.s0 with datatype=int32, encoding=PLAIN");
+      statement.execute("create timeseries root.sg2.d2.s1 with datatype=boolean, encoding=PLAIN");
+    }
+  }
+
+  private void prepareIns1() throws Exception { // add one seq tsfile in sg1
+    try (Connection connection =
+            DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      statement.execute("insert into root.sg1.d1(timestamp, s1, s2, s3) values(1, 1, 16.0, 'a')");
+      statement.execute("insert into root.sg1.d1(timestamp, s1, s2, s3) values(2, 2, 25.16, 'b')");
+      statement.execute("insert into root.sg1.d1(timestamp, s1, s2, s3) values(3, 3, 65.25, 'c')");
+      statement.execute(
+          "insert into root.sg1.d1(timestamp, s1, s2, s3) values(16, 25, 100.0, 'd')");
+      statement.execute("insert into root.sg1.d2(timestamp, s4) values(1, 1)");
+      statement.execute("flush");
+    }
+  }
+
+  private void prepareIns2() throws Exception { // add one seq tsfile in sg1
+    try (Connection connection =
+            DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      statement.execute(
+          "insert into root.sg1.d1(timestamp, s1, s2, s3) values(100, 65, 16.25, 'e')");
+      statement.execute(
+          "insert into root.sg1.d1(timestamp, s1, s2, s3) values(65, 100, 25.0, 'f')");
+      statement.execute("insert into root.sg1.d2(timestamp, s4) values(200, 100)");
+      statement.execute("flush");
+    }
+  }
+
+  private void prepareIns3()
+      throws
+          Exception { // add one seq tsfile in sg1, one unseq tsfile in sg1, one seq tsfile in sg2
+    try (Connection connection =
+            DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      statement.execute("insert into root.sg2.d1(timestamp, s0) values(100, 100)");
+      statement.execute("insert into root.sg2.d1(timestamp, s0) values(65, 65)");
+      statement.execute("insert into root.sg2.d2(timestamp, s1) values(1, true)");
+      statement.execute(
+          "insert into root.sg1.d1(timestamp, s1, s2, s3) values(25, 16, 65.16, 'g')");
+      statement.execute(
+          "insert into root.sg1.d1(timestamp, s1, s2, s3) values(200, 100, 16.65, 'h')");
+      statement.execute("flush");
+    }
+  }
+
+  private void preparePipeAndSetMock() throws Exception {
+    try (Connection connection =
+            DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      statement.execute("start pipeserver");
+      statement.execute("create pipesink " + pipeSinkName + " as iotdb");
+      statement.execute("create pipe " + pipeName + " to " + pipeSinkName);
+      pipe = (TsFilePipe) SenderService.getInstance().getRunningPipe();
+      mock =
+          new TransportClientMock(SenderService.getInstance().getRunningPipe(), "127.0.0.1", 2333);
+      TransportHandler handler =
+          new TransportHandler(
+              mock, pipeName, SenderService.getInstance().getRunningPipe().getCreateTime());
+      SenderService.getInstance().setTransportHandler(handler);
+      Thread.sleep(1000L);
+      statement.execute("stop pipeserver");
+    }
+  }
+
+  private void startPipe() throws Exception {
+    try (Connection connection =
+            DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      statement.execute("start pipe " + pipeName);
+    }
+  }
+
+  private void stopPipe() throws Exception {
+    try (Connection connection =
+            DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      statement.execute("stop pipe " + pipeName);
+    }
+  }
+
+  private void dropPipe() throws Exception {
+    try (Connection connection =
+            DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      statement.execute("drop pipe " + pipeName);
+    }
+  }
+
+  private void checkResult(List<PipeData> list) { // check ins1, ins2, ins3
+    Assert.assertEquals(list.size(), 13);
+    for (int i = 0; i < 8; i++) {
+      Assert.assertTrue(list.get(i) instanceof SchemaPipeData);
+    }
+    for (int i = 9; i < list.size(); i++) {
+      Assert.assertTrue(list.get(i) instanceof TsFilePipeData);
+    }
+  }
+
+  @Test
+  public void testHistoryInsert() {
+    try {
+      prepareSchema();
+      prepareIns1();
+      prepareIns2();
+      prepareIns3();
+
+      preparePipeAndSetMock();
+      startPipe();
+      Thread.sleep(1000L);
+      checkResult(mock.getPipeDataList());
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.fail();
+    } finally {
+      try {
+        dropPipe();
+        Thread.sleep(1000L);
+      } catch (Exception e) {
+        e.printStackTrace();
+        Assert.fail();
+      }
+    }
+  }
+
+  @Test
+  public void testHistoryAndRealTimeInsert() {
+    try {
+      prepareSchema();
+      prepareIns1();
+      prepareIns2();
+
+      preparePipeAndSetMock();
+      startPipe();
+      Thread.sleep(1000L);
+      prepareIns3();
+      Thread.sleep(1000L);
+      checkResult(mock.getPipeDataList());
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.fail();
+    } finally {
+      try {
+        dropPipe();
+        Thread.sleep(1000L);
+      } catch (Exception e) {
+        e.printStackTrace();
+        Assert.fail();
+      }
+    }
+  }
+
+  @Test
+  public void testStopAndStartInsert() {
+    try {
+      prepareSchema();
+      prepareIns1();
+
+      preparePipeAndSetMock();
+      startPipe();
+      prepareIns2();
+      stopPipe();
+      prepareIns3();
+      startPipe();
+      Thread.sleep(1000L);
+      checkResult(mock.getPipeDataList());
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.fail();
+    } finally {
+      try {
+        dropPipe();
+        Thread.sleep(1000L);
+      } catch (Exception e) {
+        e.printStackTrace();
+        Assert.fail();
+      }
+    }
+  }
+
+  @Test
+  public void testRealTimeSchemaAndStopInsert() {
+    try {
+      preparePipeAndSetMock();
+      prepareSchema();
+      startPipe();
+      prepareIns1();
+      stopPipe();
+      prepareIns2();
+      startPipe();
+      prepareIns3();
+      stopPipe();
+
+      Thread.sleep(1000L);
+      checkResult(mock.getPipeDataList());
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.fail();
+    } finally {
+      try {
+        dropPipe();
+        Thread.sleep(1000L);
+      } catch (Exception e) {
+        e.printStackTrace();
+        Assert.fail();
+      }
+    }
+  }
+}
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/sync/SyncTestUtil.java b/integration/src/test/java/org/apache/iotdb/db/integration/sync/SyncTestUtil.java
new file mode 100644
index 0000000000..a17f16fb36
--- /dev/null
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/sync/SyncTestUtil.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.integration.sync;
+
+import org.apache.iotdb.jdbc.Config;
+import org.apache.iotdb.tsfile.utils.FilePathUtils;
+
+import org.junit.Assert;
+
+import java.io.File;
+import java.sql.*;
+import java.util.*;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class SyncTestUtil {
+
+  private static final String[] sqls =
+      new String[] {
+        "SET STORAGE GROUP TO root.vehicle",
+        "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",
+        "SET STORAGE GROUP TO root.sg1",
+        "create aligned timeseries root.sg1.d1(s1 FLOAT encoding=RLE, s2 INT32 encoding=Gorilla compression=SNAPPY, s3 INT64, s4 BOOLEAN, s5 TEXT)",
+        "insert into root.sg1.d1(time, s1, s2, s3, s4, s5) aligned values(1, 1.0, 1, 1, TRUE, 'aligned_test1')",
+        "insert into root.sg1.d1(time, s1, s2, s3, s5) aligned values(2, 2.0, 2, 2, 'aligned_test2')",
+        "insert into root.sg1.d1(time, s1, s3, s4, s5) aligned values(3, 3.0, 3, FALSE, 'aligned_test3')",
+        "insert into root.sg1.d1(time, s1, s2, s4, s5) aligned values(4, 4.0, 4, TRUE, 'aligned_test4')",
+        "insert into root.sg1.d1(time, s1, s2, s4, s5) aligned values(5, 5.0, 5, TRUE, 'aligned_test5')",
+        "insert into root.sg1.d1(time, s1, s2, s3, s4) aligned values(6, 6.0, 6, 6, TRUE)",
+        "insert into root.sg1.d1(time, s1, s2, s3, s4, s5) aligned values(7, 7.0, 7, 7, FALSE, 'aligned_test7')",
+        "insert into root.sg1.d1(time, s1, s2, s3, s5) aligned values(8, 8.0, 8, 8, 'aligned_test8')",
+        "insert into root.sg1.d1(time, s1, s2, s3, s4, s5) aligned values(9, 9.0, 9, 9, FALSE, 'aligned_test9')",
+        "insert into root.vehicle.d0(timestamp,s0) values(10,10)",
+        "insert into root.vehicle.d0(timestamp,s0,s1) values(12,12,'12')",
+        "insert into root.vehicle.d0(timestamp,s1) values(14,'14')",
+        "insert into root.vehicle.d1(timestamp,s2) values(16,16.0)",
+        "insert into root.vehicle.d1(timestamp,s2,s3) values(18,18.0,true)",
+        "insert into root.vehicle.d1(timestamp,s3) values(20,false)",
+        "flush",
+        "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.sg1.d1(time, s1, s2, s3, s4, s5) aligned values(5, 130000.0, 130000, 130000, FALSE, 'aligned_unseq_test1')",
+        "flush",
+        "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)",
+        "delete from root.sg1.d1.s3 where time<=3",
+        "flush",
+        "delete from root.vehicle.** where time >= 10 and time<=14",
+        "flush",
+        // no flush data
+        "insert into root.sg1.d1(time, s1, s3, s4) aligned values(23, 230000.0, 230000, FALSE)",
+        "insert into root.sg1.d1(time, s2, s5) aligned values(31, 31, 'aligned_test31')",
+        "insert into root.sg1.d1(time, s2, s5) aligned values(32, 32, 'aligned_test32')",
+        "insert into root.sg1.d1(time, s2, s5) aligned values(33, 33, 'aligned_test33')",
+        "insert into root.sg1.d1(time, s2, s5) aligned values(34, 34, 'aligned_test34')",
+        "insert into root.sg1.d1(time, s2, s5) aligned values(35, 35, 'aligned_test35')",
+        "insert into root.sg1.d1(time, s2, s5) aligned values(36, 36, 'aligned_test36')",
+        "insert into root.sg1.d1(time, s2, s5) aligned values(37, 37, 'aligned_test37')",
+        "insert into root.sg1.d1(time, s2, s5) aligned values(38, 38, 'aligned_test38')",
+        "insert into root.sg1.d1(time, s2, s5) aligned values(39, 39, 'aligned_test39')",
+        "insert into root.sg1.d1(time, s2, s5) aligned values(40, 40, 'aligned_test40')",
+      };
+
+  public static void insertData() throws ClassNotFoundException {
+    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()) {
+      // create aligned and non-aligned time series
+      for (String sql : sqls) {
+        statement.execute(sql);
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  /**
+   * scan parentDir and return all TsFile sorted by load sequence
+   *
+   * @param parentDir folder to scan
+   */
+  public static List<File> getTsFilePaths(File parentDir) {
+    List<File> res = new ArrayList<>();
+    if (!parentDir.exists()) {
+      Assert.fail();
+      return res;
+    }
+    scanDir(res, parentDir);
+    Collections.sort(
+        res,
+        new Comparator<File>() {
+          @Override
+          public int compare(File f1, File f2) {
+            int diffSg =
+                f1.getParentFile()
+                    .getParentFile()
+                    .getParentFile()
+                    .getName()
+                    .compareTo(f2.getParentFile().getParentFile().getParentFile().getName());
+            if (diffSg != 0) {
+              return diffSg;
+            } else {
+              return (int)
+                  (FilePathUtils.splitAndGetTsFileVersion(f1.getName())
+                      - FilePathUtils.splitAndGetTsFileVersion(f2.getName()));
+            }
+          }
+        });
+    return res;
+  }
+
+  private static void scanDir(List<File> tsFiles, File parentDir) {
+    if (!parentDir.exists()) {
+      Assert.fail();
+      return;
+    }
+    File fa[] = parentDir.listFiles();
+    for (int i = 0; i < fa.length; i++) {
+      File fs = fa[i];
+      if (fs.isDirectory()) {
+        scanDir(tsFiles, fs);
+      } else if (fs.getName().endsWith(".resource")) {
+        // only add tsfile that has been flushed
+        tsFiles.add(new File(fs.getAbsolutePath().substring(0, fs.getAbsolutePath().length() - 9)));
+      }
+    }
+  }
+
+  public static void checkResult(
+      String sql, String[] columnNames, String[] retArray, boolean hasTimeColumn)
+      throws ClassNotFoundException {
+    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(sql);
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+      Map<String, Integer> map = new HashMap<>();
+      for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
+        map.put(resultSetMetaData.getColumnName(i), i);
+      }
+      assertEquals(
+          hasTimeColumn ? columnNames.length + 1 : columnNames.length,
+          resultSetMetaData.getColumnCount());
+      int cnt = 0;
+      while (resultSet.next()) {
+        StringBuilder builder = new StringBuilder();
+        if (hasTimeColumn) {
+          builder.append(resultSet.getString(1)).append(",");
+        }
+        for (String columnName : columnNames) {
+          int index = map.get(columnName);
+          builder.append(resultSet.getString(index)).append(",");
+        }
+        if (builder.length() > 0) {
+          builder.deleteCharAt(builder.length() - 1);
+        }
+        assertEquals(retArray[cnt], builder.toString());
+        cnt++;
+      }
+      assertEquals(retArray.length, cnt);
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
+  public static void checkResult(String sql, String[] columnNames, String[] retArray)
+      throws ClassNotFoundException {
+    checkResult(sql, columnNames, retArray, true);
+  }
+}
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/sync/TransportClientMock.java b/integration/src/test/java/org/apache/iotdb/db/integration/sync/TransportClientMock.java
new file mode 100644
index 0000000000..702a72f6f9
--- /dev/null
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/sync/TransportClientMock.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.integration.sync;
+
+import org.apache.iotdb.db.exception.SyncConnectionException;
+import org.apache.iotdb.db.sync.pipedata.PipeData;
+import org.apache.iotdb.db.sync.sender.pipe.Pipe;
+import org.apache.iotdb.db.sync.transport.client.ITransportClient;
+import org.apache.iotdb.service.transport.thrift.ResponseType;
+import org.apache.iotdb.service.transport.thrift.SyncRequest;
+import org.apache.iotdb.service.transport.thrift.SyncResponse;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class TransportClientMock implements ITransportClient {
+  private final Pipe pipe;
+  private final String ipAddress;
+  private final int port;
+
+  private List<PipeData> pipeDataList;
+
+  public TransportClientMock(Pipe pipe, String ipAddress, int port) {
+    this.pipe = pipe;
+    this.ipAddress = ipAddress;
+    this.port = port;
+
+    this.pipeDataList = new ArrayList<>();
+  }
+
+  @Override
+  public SyncResponse heartbeat(SyncRequest syncRequest) throws SyncConnectionException {
+    return new SyncResponse(ResponseType.INFO, "");
+  }
+
+  @Override
+  public void run() {
+    try {
+      while (!Thread.currentThread().isInterrupted()) {
+        PipeData pipeData = pipe.take();
+        pipeDataList.add(pipeData);
+        pipe.commit();
+      }
+    } catch (InterruptedException e) {
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  public List<PipeData> getPipeDataList() {
+    return pipeDataList;
+  }
+}
diff --git a/integration/src/test/resources/iotdb-engine.properties b/integration/src/test/resources/iotdb-engine.properties
index f93226c615..ce9be14770 100644
--- a/integration/src/test/resources/iotdb-engine.properties
+++ b/integration/src/test/resources/iotdb-engine.properties
@@ -17,4 +17,5 @@
 
 base_dir=target/tmp
 data_dirs=target/data
-wal_dir=target/wal
\ No newline at end of file
+wal_dir=target/wal
+sync_dir=target/sync
\ No newline at end of file
diff --git a/integration/src/test/resources/logback-test.xml b/integration/src/test/resources/logback-test.xml
index 003c1b363c..a5177a7f55 100644
--- a/integration/src/test/resources/logback-test.xml
+++ b/integration/src/test/resources/logback-test.xml
@@ -40,6 +40,7 @@
     <!-- enable me if you want to monitor when files are opened and closed.
     <logger name="FileMonitor" level="info"/>
     -->
+    <logger name="org.apache.iotdb.db.sync" level="INFO"/>
     <logger name="org.apache.iotdb.db.engine.merge" level="INFO"/>
     <logger name="org.apache.iotdb.commons.service.ThriftServiceThread" level="INFO"/>
     <logger name="org.eclipse.jetty.util.thread.QueuedThreadPool" level="INFO"/>
@@ -50,7 +51,6 @@
     <logger name="org.apache.iotdb.db.service.IoTDB" level="WARN"/>
     <logger name="org.apache.iotdb.db.service.RPCService" level="INFO"/>
     <logger name="org.apache.iotdb.db.service.MQTTService" level="INFO"/>
-    <logger name="org.apache.iotdb.db.sync.receiver.SyncServerManager" level="INFO"/>
     <logger name="DETAILED_FAILURE_QUERY_TRACE" level="ERROR"/>
     <root level="ERROR">
         <appender-ref ref="stdout"/>
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java b/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java
index 6e76e05556..a4e0dc35ad 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java
@@ -47,8 +47,6 @@ public enum ThreadName {
   INDEX_SERVICE("Index"),
   SYNC_CLIENT("Sync-Client"),
   SYNC_SERVER("Sync"),
-  SYNC_MONITOR("Sync-Monitor"),
-  LOAD_TSFILE("Load-TsFile"),
   TIME_COST_STATISTIC("TIME_COST_STATISTIC"),
   QUERY_SERVICE("Query"),
   SUB_RAW_QUERY_SERVICE("Sub_RawQuery"),
@@ -58,6 +56,9 @@ public enum ThreadName {
   TIMED_FlUSH_SEQ_MEMTABLE("Timed-Flush-Seq-Memtable"),
   TIMED_FlUSH_UNSEQ_MEMTABLE("Timed-Flush-Unseq-Memtable"),
   SETTLE_SERVICE("Settle"),
+  SYNC_SENDER_PIPE("Sync-Pipe"),
+  SYNC_SENDER_HEARTBEAT("Sync-Heartbeat"),
+  SYNC_RECEIVER_COLLECTOR("Sync-Collector"),
   CONTINUOUS_QUERY_SERVICE("ContinuousQueryTaskPoolManager"),
   CLUSTER_INFO_SERVICE("ClusterInfoClient"),
   CLUSTER_RPC_SERVICE("ClusterRPC"),
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java b/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java
index b7f1fe4e58..f6bad59128 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java
@@ -126,9 +126,23 @@ public class IoTDBConstant {
   public static final String COLUMN_TRIGGER_STATUS_STARTED = "started";
   public static final String COLUMN_TRIGGER_STATUS_STOPPED = "stopped";
 
+  // sync module
+  public static final String COLUMN_PIPESERVER_STATUS = "enable";
+  public static final String COLUMN_PIPESINK_NAME = "name";
+  public static final String COLUMN_PIPESINK_TYPE = "type";
+  public static final String COLUMN_PIPESINK_ATTRIBUTES = "attributes";
+  public static final String COLUMN_PIPE_NAME = "name";
+  public static final String COLUMN_PIPE_CREATE_TIME = "create time";
+  public static final String COLUMN_PIPE_ROLE = "role";
+  public static final String COLUMN_PIPE_REMOTE = "remote";
+  public static final String COLUMN_PIPE_STATUS = "status";
+  public static final String COLUMN_PIPE_MSG = "message";
+
   public static final String ONE_LEVEL_PATH_WILDCARD = "*";
   public static final String MULTI_LEVEL_PATH_WILDCARD = "**";
   public static final String TIME = "time";
+  public static final String SYNC_SENDER_ROLE = "sender";
+  public static final String SYNC_RECEIVER_ROLE = "receiver";
 
   // sdt parameters
   public static final String LOSS = "loss";
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/service/ServiceType.java b/node-commons/src/main/java/org/apache/iotdb/commons/service/ServiceType.java
index 210c18b07a..0626a0261a 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/service/ServiceType.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/service/ServiceType.java
@@ -36,6 +36,8 @@ public enum ServiceType {
   SYNC_SERVICE("SYNC ServerService", ""),
   UPGRADE_SERVICE("UPGRADE DataService", ""),
   SETTLE_SERVICE("SETTLE DataService", ""),
+  SENDER_SERVICE("Sync Sender service", ""),
+  RECEIVER_SERVICE("Sync Receiver service", ""),
   MERGE_SERVICE("Merge Manager", "Merge Manager"),
   COMPACTION_SERVICE("Compaction Manager", "Compaction Manager"),
   PERFORMANCE_STATISTIC_SERVICE("PERFORMANCE_STATISTIC_SERVICE", "PERFORMANCE_STATISTIC_SERVICE"),
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/utils/StatusUtils.java b/node-commons/src/main/java/org/apache/iotdb/commons/utils/StatusUtils.java
index 290c0c8423..1f202f359a 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/utils/StatusUtils.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/utils/StatusUtils.java
@@ -118,9 +118,6 @@ public class StatusUtils {
       case SYSTEM_CHECK_ERROR:
         status.setMessage("Meet error while system checking. ");
         break;
-      case SYNC_DEVICE_OWNER_CONFLICT_ERROR:
-        status.setMessage("Sync device owners conflict. ");
-        break;
       case SYNC_CONNECTION_EXCEPTION:
         status.setMessage("Meet error while sync connecting. ");
         break;
diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties
index 85a6418304..d9bf4c6ddb 100644
--- a/server/src/assembly/resources/conf/iotdb-engine.properties
+++ b/server/src/assembly/resources/conf/iotdb-engine.properties
@@ -591,18 +591,12 @@ timestamp_precision=ms
 # Datatype: int
 # external_sort_threshold=1000
 
-
 ####################
-### Sync Server Configuration
+### PIPE Server Configuration
 ####################
-
-# Whether to open the sync_server_port for receiving data from sync client, the default is closed
-# Datatype: boolean
-# is_sync_enable=false
-
-# Sync server port to listen
+# PIPE server port to listen
 # Datatype: int
-# sync_server_port=5555
+# pipe_server_port=6670
 
 # White IP list of Sync client.
 # Please use the form of network segment to present the range of IP, for example: 192.168.0.0/16
@@ -611,6 +605,13 @@ timestamp_precision=ms
 # Datatype: String
 # ip_white_list=0.0.0.0/0
 
+####################
+### PIPE Sender Configuration
+####################
+# The maximum number of retry when syncing a file to receiver fails.
+# max_number_of_sync_file_retry=5
+
+
 ####################
 ### 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
deleted file mode 100644
index f975f3bf7c..0000000000
--- a/server/src/assembly/resources/conf/iotdb-sync-client.properties
+++ /dev/null
@@ -1,35 +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.
-#
-
-# Sync receiver server address
-server_ip=127.0.0.1
-
-# Sync receiver server port
-server_port=5555
-
-# The period time of sync process, the time unit is second.
-sync_period_in_second=600
-
-# 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
-
-# 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/tools/start-sync-client.bat b/server/src/assembly/resources/tools/start-sync-client.bat
deleted file mode 100755
index abe85f38c4..0000000000
--- a/server/src/assembly/resources/tools/start-sync-client.bat
+++ /dev/null
@@ -1,71 +0,0 @@
-@REM
-@REM Licensed to the Apache Software Foundation (ASF) under one
-@REM or more contributor license agreements.  See the NOTICE file
-@REM distributed with this work for additional information
-@REM regarding copyright ownership.  The ASF licenses this file
-@REM to you under the Apache License, Version 2.0 (the
-@REM "License"); you may not use this file except in compliance
-@REM with the License.  You may obtain a copy of the License at
-@REM
-@REM     http://www.apache.org/licenses/LICENSE-2.0
-@REM
-@REM Unless required by applicable law or agreed to in writing,
-@REM software distributed under the License is distributed on an
-@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-@REM KIND, either express or implied.  See the License for the
-@REM specific language governing permissions and limitations
-@REM under the License.
-@REM
-
-@echo off
-
-if "%OS%" == "Windows_NT" setlocal
-
-pushd %~dp0..
-if NOT DEFINED IOTDB_HOME set IOTDB_HOME=%CD%
-popd
-
-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.transfer.SyncClient
-if NOT DEFINED JAVA_HOME goto :err
-
-@REM -----------------------------------------------------------------------------
-@REM JVM Opts we'll use in legacy run or installation
-set JAVA_OPTS=-ea^
- -Dlogback.configurationFile="%IOTDB_CONF%\logback.xml"^
- -DIOTDB_HOME=%IOTDB_HOME%^
- -DTSFILE_HOME=%IOTDB_HOME%^
- -DIOTDB_CONF=%IOTDB_CONF%
-
-@REM ***** CLASSPATH library setting *****
-@REM Ensure that any user defined CLASSPATH variables are not used on startup
-set CLASSPATH="%IOTDB_HOME%\lib\*"
-
-goto okClasspath
-
-:append
-set CLASSPATH=%CLASSPATH%;%1
-goto :eof
-
-@REM -----------------------------------------------------------------------------
-:okClasspath
-
-@rem echo CLASSPATH: %CLASSPATH%
-
-"%JAVA_HOME%\bin\java" %JAVA_OPTS% %IOTDB_HEAP_OPTS% -cp %CLASSPATH% %IOTDB_JMX_OPTS% %MAIN_CLASS%
-goto finally
-
-:err
-echo JAVA_HOME environment variable must be set!
-pause
-
-
-@REM -----------------------------------------------------------------------------
-:finally
-
-pause
-
-ENDLOCAL
diff --git a/server/src/assembly/resources/tools/start-sync-client.sh b/server/src/assembly/resources/tools/start-sync-client.sh
deleted file mode 100755
index 505f85c889..0000000000
--- a/server/src/assembly/resources/tools/start-sync-client.sh
+++ /dev/null
@@ -1,54 +0,0 @@
-#!/bin/sh
-#
-# 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.
-#
-
-
-if [ -z "${IOTDB_HOME}" ]; then
-  export IOTDB_HOME="$(cd "`dirname "$0"`"/..; pwd)"
-fi
-
-IOTDB_CONF=${IOTDB_HOME}/conf
-# IOTDB_LOGS=${IOTDB_HOME}/logs
-
-if [ -n "$JAVA_HOME" ]; then
-    for java in "$JAVA_HOME"/bin/amd64/java "$JAVA_HOME"/bin/java; do
-        if [ -x "$java" ]; then
-            JAVA="$java"
-            break
-        fi
-    done
-else
-    JAVA=java
-fi
-
-if [ -z $JAVA ] ; then
-    echo Unable to find java executable. Check JAVA_HOME and PATH environment variables.  > /dev/stderr
-    exit 1;
-fi
-
-CLASSPATH=""
-for f in ${IOTDB_HOME}/lib/*.jar; do
-  CLASSPATH=${CLASSPATH}":"$f
-done
-
-MAIN_CLASS=org.apache.iotdb.db.sync.sender.transfer.SyncClient
-
-"$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"
-
-exit $?
diff --git a/server/src/assembly/resources/tools/stop-sync-client.bat b/server/src/assembly/resources/tools/stop-sync-client.bat
deleted file mode 100755
index 2618bde2ad..0000000000
--- a/server/src/assembly/resources/tools/stop-sync-client.bat
+++ /dev/null
@@ -1,23 +0,0 @@
-@REM
-@REM Licensed to the Apache Software Foundation (ASF) under one
-@REM or more contributor license agreements.  See the NOTICE file
-@REM distributed with this work for additional information
-@REM regarding copyright ownership.  The ASF licenses this file
-@REM to you under the Apache License, Version 2.0 (the
-@REM "License"); you may not use this file except in compliance
-@REM with the License.  You may obtain a copy of the License at
-@REM
-@REM     http://www.apache.org/licenses/LICENSE-2.0
-@REM
-@REM Unless required by applicable law or agreed to in writing,
-@REM software distributed under the License is distributed on an
-@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-@REM KIND, either express or implied.  See the License for the
-@REM specific language governing permissions and limitations
-@REM under the License.
-@REM
-
-@echo off
-
-wmic process where (commandline like "%%SyncClient%%" and not name="wmic.exe") delete
-rem ps ax | grep -i 'SyncClient' | grep -v grep | awk '{print $1}' | xargs kill -SIGTERM
diff --git a/server/src/assembly/resources/tools/stop-sync-client.sh b/server/src/assembly/resources/tools/stop-sync-client.sh
deleted file mode 100755
index 61bbd2384a..0000000000
--- a/server/src/assembly/resources/tools/stop-sync-client.sh
+++ /dev/null
@@ -1,30 +0,0 @@
-#!/bin/sh
-#
-# 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.
-#
-
-
-PIDS=$(ps ax | grep -i 'SyncClient' | grep java | grep -v grep | awk '{print $1}')
-
-if [ -z "$PIDS" ]; then
-  echo "No sync client to stop"
-  exit 1
-else 
-  kill -s TERM $PIDS
-  echo "close sync client"
-fi
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 edee341863..a5f860c194 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
@@ -234,13 +234,8 @@ public class IoTDBConfig {
           + File.separator
           + IoTDBConstant.SCHEMA_FOLDER_NAME;
 
-  /** Sync directory, including the lock file, uuid file, device owner map */
-  private String syncDir =
-      DEFAULT_BASE_DIR
-          + File.separator
-          + IoTDBConstant.SYSTEM_FOLDER_NAME
-          + File.separator
-          + IoTDBConstant.SYNC_FOLDER_NAME;
+  /** Sync directory, including the log and hardlink tsfiles */
+  private String syncDir = DEFAULT_BASE_DIR + File.separator + IoTDBConstant.SYNC_FOLDER_NAME;
 
   /** Performance tracing directory, stores performance tracing files */
   private String tracingDir = DEFAULT_BASE_DIR + File.separator + IoTDBConstant.TRACING_FOLDER_NAME;
@@ -488,19 +483,20 @@ public class IoTDBConfig {
    */
   private int externalSortThreshold = 1000;
 
-  /** Is this IoTDB instance a receiver of sync or not. */
-  private boolean isSyncEnable = false;
-
   /** If this IoTDB instance is a receiver of sync, set the server port. */
-  private int syncServerPort = 5555;
+  private int pipeServerPort = 6670;
+
+  /** White list for sync */
+  private String ipWhiteList = "0.0.0.0/0";
+
+  /** The maximum number of retries when the sender fails to synchronize files to the receiver. */
+  private int maxNumberOfSyncFileRetry = 5;
 
   /**
    * Set the language version when loading file including error information, default value is "EN"
    */
   private String languageVersion = "EN";
 
-  private String ipWhiteList = "0.0.0.0/0";
-
   /** Examining period of cache file reader : 100 seconds. Unit: millisecond */
   private long cacheFileReaderClearPeriod = 100000;
 
@@ -1320,20 +1316,20 @@ public class IoTDBConfig {
     this.mRemoteSchemaCacheSize = mRemoteSchemaCacheSize;
   }
 
-  public boolean isSyncEnable() {
-    return isSyncEnable;
+  public int getPipeServerPort() {
+    return pipeServerPort;
   }
 
-  public void setSyncEnable(boolean syncEnable) {
-    isSyncEnable = syncEnable;
+  public void setPipeServerPort(int pipeServerPort) {
+    this.pipeServerPort = pipeServerPort;
   }
 
-  public int getSyncServerPort() {
-    return syncServerPort;
+  public int getMaxNumberOfSyncFileRetry() {
+    return maxNumberOfSyncFileRetry;
   }
 
-  void setSyncServerPort(int syncServerPort) {
-    this.syncServerPort = syncServerPort;
+  public void setMaxNumberOfSyncFileRetry(int maxNumberOfSyncFileRetry) {
+    this.maxNumberOfSyncFileRetry = maxNumberOfSyncFileRetry;
   }
 
   String getLanguageVersion() {
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 e48f56989c..b34464173f 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
@@ -240,9 +240,6 @@ public class IoTDBDescriptor {
       conf.setSchemaDir(
           FilePathUtils.regularizePath(conf.getSystemDir()) + IoTDBConstant.SCHEMA_FOLDER_NAME);
 
-      conf.setSyncDir(
-          FilePathUtils.regularizePath(conf.getSystemDir()) + IoTDBConstant.SYNC_FOLDER_NAME);
-
       conf.setQueryDir(
           FilePathUtils.regularizePath(conf.getSystemDir() + IoTDBConstant.QUERY_FOLDER_NAME));
 
@@ -250,6 +247,8 @@ public class IoTDBDescriptor {
 
       conf.setDataDirs(properties.getProperty("data_dirs", conf.getDataDirs()[0]).split(","));
 
+      conf.setSyncDir(properties.getProperty("sync_dir", conf.getSyncDir()));
+
       conf.setConsensusDir(properties.getProperty("consensus_dir", conf.getConsensusDir()));
 
       int mlogBufferSize =
@@ -415,15 +414,17 @@ public class IoTDBDescriptor {
               properties.getProperty(
                   "session_timeout_threshold",
                   Integer.toString(conf.getSessionTimeoutThreshold()))));
-
-      conf.setSyncEnable(
-          Boolean.parseBoolean(
-              properties.getProperty("is_sync_enable", Boolean.toString(conf.isSyncEnable()))));
-
-      conf.setSyncServerPort(
+      conf.setPipeServerPort(
           Integer.parseInt(
               properties
-                  .getProperty("sync_server_port", Integer.toString(conf.getSyncServerPort()))
+                  .getProperty("pipe_server_port", Integer.toString(conf.getPipeServerPort()))
+                  .trim()));
+      conf.setMaxNumberOfSyncFileRetry(
+          Integer.parseInt(
+              properties
+                  .getProperty(
+                      "max_number_of_sync_file_retry",
+                      Integer.toString(conf.getMaxNumberOfSyncFileRetry()))
                   .trim()));
 
       conf.setIpWhiteList(properties.getProperty("ip_white_list", conf.getIpWhiteList()));
@@ -1345,6 +1346,20 @@ public class IoTDBDescriptor {
               properties.getProperty(
                   "select_into_insert_tablet_plan_row_limit",
                   String.valueOf(conf.getSelectIntoInsertTabletPlanRowLimit()))));
+
+      // update sync config
+      conf.setPipeServerPort(
+          Integer.parseInt(
+              properties.getProperty(
+                  "pipe_server_port", String.valueOf(conf.getPipeServerPort()))));
+      conf.setMaxNumberOfSyncFileRetry(
+          Integer.parseInt(
+              properties
+                  .getProperty(
+                      "max_number_of_sync_file_retry",
+                      Integer.toString(conf.getMaxNumberOfSyncFileRetry()))
+                  .trim()));
+      conf.setIpWhiteList(properties.getProperty("ip_white_list", conf.getIpWhiteList()));
     } catch (Exception e) {
       throw new QueryProcessException(String.format("Fail to reload configuration because %s", e));
     }
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 f589d8f402..9d4d79fb39 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
@@ -882,12 +882,6 @@ public class StorageEngine implements IService {
     manager.abortCompaction();
   }
 
-  public void loadNewTsFileForSync(TsFileResource newTsFileResource)
-      throws StorageEngineException, LoadFileException, IllegalPathException {
-    getProcessorDirectly(new PartialPath(getSgByEngineFile(newTsFileResource.getTsFile(), false)))
-        .loadNewTsFileForSync(newTsFileResource);
-  }
-
   public void loadNewTsFile(TsFileResource newTsFileResource)
       throws LoadFileException, StorageEngineException, MetadataException {
     Set<String> deviceSet = newTsFileResource.getDevices();
@@ -900,12 +894,6 @@ public class StorageEngine implements IService {
     getProcessorDirectly(storageGroupPath).loadNewTsFile(newTsFileResource);
   }
 
-  public boolean deleteTsfileForSync(File deletedTsfile)
-      throws StorageEngineException, IllegalPathException {
-    return getProcessorDirectly(new PartialPath(getSgByEngineFile(deletedTsfile, false)))
-        .deleteTsfile(deletedTsfile);
-  }
-
   public boolean deleteTsfile(File deletedTsfile)
       throws StorageEngineException, IllegalPathException {
     return getProcessorDirectly(new PartialPath(getSgByEngineFile(deletedTsfile, true)))
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java b/server/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java
index edbc767ae0..edb8d4d76b 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java
@@ -19,8 +19,13 @@
 
 package org.apache.iotdb.db.engine.modification;
 
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
 import java.util.Objects;
 
 /** Deletion is a delete operation on a timeseries. */
@@ -72,6 +77,24 @@ public class Deletion extends Modification {
     this.endTime = timestamp;
   }
 
+  public long serializeWithoutFileOffset(DataOutputStream stream) throws IOException {
+    long serializeSize = 0;
+    stream.writeLong(startTime);
+    serializeSize += Long.BYTES;
+    stream.writeLong(endTime);
+    serializeSize += Long.BYTES;
+    serializeSize += ReadWriteIOUtils.write(getPathString(), stream);
+    return serializeSize;
+  }
+
+  public static Deletion deserializeWithoutFileOffset(DataInputStream stream)
+      throws IOException, IllegalPathException {
+    long startTime = stream.readLong();
+    long endTime = stream.readLong();
+    return new Deletion(
+        new PartialPath(ReadWriteIOUtils.readString(stream)), 0, startTime, endTime);
+  }
+
   @Override
   public boolean equals(Object obj) {
     if (this == obj) {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
index fa94d5be76..674a936c47 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
@@ -80,6 +80,7 @@ import org.apache.iotdb.db.service.SettleService;
 import org.apache.iotdb.db.service.metrics.Metric;
 import org.apache.iotdb.db.service.metrics.MetricsService;
 import org.apache.iotdb.db.service.metrics.Tag;
+import org.apache.iotdb.db.sync.sender.manager.TsFileSyncManager;
 import org.apache.iotdb.db.tools.settle.TsFileAndModSettleTool;
 import org.apache.iotdb.db.utils.CopyOnReadLinkedList;
 import org.apache.iotdb.db.utils.UpgradeUtils;
@@ -259,6 +260,9 @@ public class DataRegion {
 
   private IDTable idTable;
 
+  /** used to collect TsFiles in this virtual storage group */
+  private TsFileSyncManager tsFileSyncManager = TsFileSyncManager.getInstance();
+
   /**
    * constrcut a storage group processor
    *
@@ -2140,6 +2144,8 @@ public class DataRegion {
       if (!tsFileResource.isClosed()) {
         TsFileProcessor tsfileProcessor = tsFileResource.getProcessor();
         tsfileProcessor.deleteDataInMemory(deletion, devicePaths);
+      } else if (tsFileSyncManager.isEnableSync()) {
+        tsFileSyncManager.collectRealTimeDeletion(deletion);
       }
 
       // add a record in case of rollback
@@ -2385,47 +2391,6 @@ public class DataRegion {
     }
   }
 
-  /**
-   * Load a new tsfile to storage group processor. The file may have overlap with other files.
-   *
-   * <p>or unsequence list.
-   *
-   * <p>Secondly, execute the loading process by the type.
-   *
-   * <p>Finally, update the latestTimeForEachDevice and partitionLatestFlushedTimeForEachDevice.
-   *
-   * @param newTsFileResource tsfile resource @UsedBy sync module.
-   */
-  public void loadNewTsFileForSync(TsFileResource newTsFileResource) throws LoadFileException {
-    File tsfileToBeInserted = newTsFileResource.getTsFile();
-    long newFilePartitionId = newTsFileResource.getTimePartitionWithCheck();
-    writeLock("loadNewTsFileForSync");
-    try {
-      if (loadTsFileByType(
-          LoadTsFileType.LOAD_SEQUENCE,
-          tsfileToBeInserted,
-          newTsFileResource,
-          newFilePartitionId,
-          tsFileManager.getSequenceListByTimePartition(newFilePartitionId).size() - 1)) {
-        updateLatestTimeMap(newTsFileResource);
-      }
-      resetLastCacheWhenLoadingTsfile(newTsFileResource);
-    } catch (DiskSpaceInsufficientException e) {
-      logger.error(
-          "Failed to append the tsfile {} to storage group processor {} because the disk space is insufficient.",
-          tsfileToBeInserted.getAbsolutePath(),
-          tsfileToBeInserted.getParentFile().getName());
-      IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
-      throw new LoadFileException(e);
-    } catch (IllegalPathException e) {
-      logger.error(
-          "Failed to reset last cache when loading file {}", newTsFileResource.getTsFilePath());
-      throw new LoadFileException(e);
-    } finally {
-      writeUnlock();
-    }
-  }
-
   private void resetLastCacheWhenLoadingTsfile(TsFileResource newTsFileResource)
       throws IllegalPathException {
     for (String device : newTsFileResource.getDevices()) {
@@ -3416,6 +3381,22 @@ public class DataRegion {
     }
   }
 
+  /**
+   * Used to collect history TsFiles(i.e. the tsfile whose memtable == null).
+   *
+   * @param dataStartTime only collect history TsFiles which contains the data after the
+   *     dataStartTime
+   * @return A list, which contains TsFile path
+   */
+  public List<File> collectHistoryTsFileForSync(long dataStartTime) {
+    writeLock("Collect data for sync");
+    try {
+      return tsFileManager.collectHistoryTsFileForSync(dataStartTime);
+    } finally {
+      writeUnlock();
+    }
+  }
+
   public void setCustomCloseFileListeners(List<CloseFileListener> customCloseFileListeners) {
     this.customCloseFileListeners = customCloseFileListeners;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileManager.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileManager.java
index b816f28c42..e363b6765f 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileManager.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.engine.storagegroup;
 
 import org.apache.iotdb.db.exception.WriteLockFailedException;
 import org.apache.iotdb.db.rescon.TsFileResourceManager;
+import org.apache.iotdb.db.sync.sender.manager.TsFileSyncManager;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -370,6 +371,43 @@ public class TsFileManager {
     return unsequenceRecoverTsFileResources;
   }
 
+  public List<File> collectHistoryTsFileForSync(long dataStartTime) {
+    readLock();
+    try {
+      List<File> historyTsFiles = new ArrayList<>();
+      collectTsFile(historyTsFiles, getTsFileList(true), dataStartTime);
+      collectTsFile(historyTsFiles, getTsFileList(false), dataStartTime);
+      return historyTsFiles;
+    } finally {
+      readUnlock();
+    }
+  }
+
+  private void collectTsFile(
+      List<File> historyTsFiles, List<TsFileResource> tsFileResources, long dataStartTime) {
+    TsFileSyncManager syncManager = TsFileSyncManager.getInstance();
+
+    for (TsFileResource tsFileResource : tsFileResources) {
+      if (tsFileResource.getFileEndTime() < dataStartTime) {
+        continue;
+      }
+      TsFileProcessor tsFileProcessor = tsFileResource.getProcessor();
+      boolean isRealTimeTsFile = false;
+      if (tsFileProcessor != null) {
+        isRealTimeTsFile = tsFileProcessor.isMemtableNotNull();
+      }
+      File tsFile = tsFileResource.getTsFile();
+      if (!isRealTimeTsFile && !syncManager.isTsFileAlreadyBeCollected(tsFile)) {
+        File mods = new File(tsFileResource.getModFile().getFilePath());
+        long modsOffset = mods.exists() ? mods.length() : 0L;
+        File hardlink = syncManager.createHardlink(tsFile, modsOffset);
+        if (hardlink != null) {
+          historyTsFiles.add(hardlink);
+        }
+      }
+    }
+  }
+
   // ({systemTime}-{versionNum}-{innerCompactionNum}-{crossCompactionNum}.tsfile)
   public static int compareFileName(File o1, File o2) {
     String[] items1 = o1.getName().replace(TSFILE_SUFFIX, "").split(FILE_NAME_SEPARATOR);
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
index 43484ae580..9d5e0d323d 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
@@ -55,6 +55,7 @@ import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.rescon.MemTableManager;
 import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
 import org.apache.iotdb.db.rescon.SystemInfo;
+import org.apache.iotdb.db.sync.sender.manager.TsFileSyncManager;
 import org.apache.iotdb.db.utils.MemUtils;
 import org.apache.iotdb.db.utils.datastructure.AlignedTVList;
 import org.apache.iotdb.db.utils.datastructure.TVList;
@@ -161,6 +162,9 @@ public class TsFileProcessor {
   /** flush file listener */
   private List<FlushListener> flushListeners = new ArrayList<>();
 
+  /** used to collct this TsFile for sync */
+  private TsFileSyncManager tsFileSyncManager = TsFileSyncManager.getInstance();
+
   @SuppressWarnings("squid:S107")
   TsFileProcessor(
       String storageGroupName,
@@ -853,6 +857,9 @@ public class TsFileProcessor {
       if (!flushingMemTables.isEmpty()) {
         modsToMemtable.add(new Pair<>(deletion, flushingMemTables.getLast()));
       }
+      if (tsFileSyncManager.isEnableSync()) {
+        tsFileSyncManager.collectRealTimeDeletion(deletion);
+      }
     } finally {
       flushQueryLock.writeLock().unlock();
       if (logger.isDebugEnabled()) {
@@ -1003,6 +1010,9 @@ public class TsFileProcessor {
         // When invoke closing TsFile after insert data to memTable, we shouldn't flush until invoke
         // flushing memTable in System module.
         addAMemtableIntoFlushingList(tmpMemTable);
+        if (tsFileSyncManager.isEnableSync()) {
+          tsFileSyncManager.collectRealTimeTsFile(tsFileResource.getTsFile());
+        }
         logger.info("Memtable {} has been added to flushing list", tmpMemTable);
         shouldClose = true;
       } catch (Exception e) {
@@ -1387,6 +1397,9 @@ public class TsFileProcessor {
     long closeStartTime = System.currentTimeMillis();
     writer.endFile();
     tsFileResource.serialize();
+    if (tsFileSyncManager.isEnableSync()) {
+      tsFileSyncManager.collectRealTimeResource(tsFileResource.getTsFile());
+    }
     logger.info("Ended file {}", tsFileResource);
 
     // remove this processor from Closing list in StorageGroupProcessor,
@@ -1421,6 +1434,16 @@ public class TsFileProcessor {
     this.managedByFlushManager = managedByFlushManager;
   }
 
+  /** sync method */
+  public boolean isMemtableNotNull() {
+    flushQueryLock.writeLock().lock();
+    try {
+      return workMemTable != null;
+    } finally {
+      flushQueryLock.writeLock().unlock();
+    }
+  }
+
   /** close this tsfile */
   public void close() throws TsFileProcessorException {
     try {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/dataregion/StorageGroupManager.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/dataregion/StorageGroupManager.java
index c6b455b365..3127b21213 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/dataregion/StorageGroupManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/dataregion/StorageGroupManager.java
@@ -37,6 +37,7 @@ import org.apache.iotdb.tsfile.utils.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -453,6 +454,15 @@ public class StorageGroupManager {
     }
   }
 
+  /** collect all tsfiles whose memtable == null for sync */
+  public List<File> collectHistoryTsFileForSync(long dataStartTime) {
+    List<File> historyTsFiles = new ArrayList<>();
+    for (DataRegion processor : this.dataRegion) {
+      historyTsFiles.addAll(processor.collectHistoryTsFileForSync(dataStartTime));
+    }
+    return historyTsFiles;
+  }
+
   /** only for test */
   public void reset() {
     Arrays.fill(dataRegion, null);
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java b/server/src/main/java/org/apache/iotdb/db/exception/sync/PipeDataLoadBearableException.java
similarity index 80%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
copy to server/src/main/java/org/apache/iotdb/db/exception/sync/PipeDataLoadBearableException.java
index 24b33dfc95..4638e35709 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/sync/PipeDataLoadBearableException.java
@@ -16,10 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.receiver.load;
+package org.apache.iotdb.db.exception.sync;
 
-public enum LoadType {
-  DELETE,
-  ADD,
-  NONE
+public class PipeDataLoadBearableException extends PipeDataLoadException {
+  public PipeDataLoadBearableException(String message) {
+    super(message);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java b/server/src/main/java/org/apache/iotdb/db/exception/sync/PipeDataLoadException.java
similarity index 81%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
copy to server/src/main/java/org/apache/iotdb/db/exception/sync/PipeDataLoadException.java
index 24b33dfc95..c10d9919e6 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/sync/PipeDataLoadException.java
@@ -16,10 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.receiver.load;
+package org.apache.iotdb.db.exception.sync;
 
-public enum LoadType {
-  DELETE,
-  ADD,
-  NONE
+public class PipeDataLoadException extends PipeServerException {
+  public PipeDataLoadException(String message) {
+    super(message);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java b/server/src/main/java/org/apache/iotdb/db/exception/sync/PipeDataLoadUnbearableException.java
similarity index 79%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
copy to server/src/main/java/org/apache/iotdb/db/exception/sync/PipeDataLoadUnbearableException.java
index 24b33dfc95..8ef4225c3b 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/sync/PipeDataLoadUnbearableException.java
@@ -16,10 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.receiver.load;
+package org.apache.iotdb.db.exception.sync;
 
-public enum LoadType {
-  DELETE,
-  ADD,
-  NONE
+public class PipeDataLoadUnbearableException extends PipeDataLoadException {
+  public PipeDataLoadUnbearableException(String message) {
+    super(message);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/exception/sync/PipeException.java
similarity index 63%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
copy to server/src/main/java/org/apache/iotdb/db/exception/sync/PipeException.java
index df3f78f4a6..a117ed1dfd 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/sync/PipeException.java
@@ -15,19 +15,22 @@
  * 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 org.apache.iotdb.db.sync.receiver.load.IFileLoader;
+package org.apache.iotdb.db.exception.sync;
 
-import java.io.File;
-import java.io.IOException;
+import org.apache.iotdb.commons.exception.IoTDBException;
+import org.apache.iotdb.rpc.TSStatusCode;
 
-public interface ISyncReceiverLogAnalyzer {
+public class PipeException extends IoTDBException {
 
-  void recoverAll() throws IOException;
+  private static final long serialVersionUID = -7312720445194413492L;
 
-  boolean recover(String senderName) throws IOException;
+  public PipeException(String message, int errorCode) {
+    super(message, errorCode);
+  }
 
-  void scanLogger(IFileLoader loader, File syncLog, File loadLog);
+  public PipeException(String message) {
+    super(message, TSStatusCode.PIPE_ERROR.getStatusCode());
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/exception/sync/PipeServerException.java
similarity index 66%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
copy to server/src/main/java/org/apache/iotdb/db/exception/sync/PipeServerException.java
index df3f78f4a6..db20155586 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/sync/PipeServerException.java
@@ -15,19 +15,20 @@
  * 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 org.apache.iotdb.db.sync.receiver.load.IFileLoader;
-
-import java.io.File;
-import java.io.IOException;
+package org.apache.iotdb.db.exception.sync;
 
-public interface ISyncReceiverLogAnalyzer {
+import org.apache.iotdb.commons.exception.IoTDBException;
+import org.apache.iotdb.rpc.TSStatusCode;
 
-  void recoverAll() throws IOException;
+public class PipeServerException extends IoTDBException {
 
-  boolean recover(String senderName) throws IOException;
+  public PipeServerException(String message, int errorCode) {
+    super(message, errorCode);
+  }
 
-  void scanLogger(IFileLoader loader, File syncLog, File loadLog);
+  public PipeServerException(String message) {
+    this(message, TSStatusCode.PIPESINK_ERROR.getStatusCode());
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java b/server/src/main/java/org/apache/iotdb/db/exception/sync/PipeSinkException.java
similarity index 59%
copy from server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
copy to server/src/main/java/org/apache/iotdb/db/exception/sync/PipeSinkException.java
index 271c177c5c..b9f6a905c8 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/sync/PipeSinkException.java
@@ -15,26 +15,28 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
+ *
  */
-package org.apache.iotdb.db.exception;
+package org.apache.iotdb.db.exception.sync;
 
 import org.apache.iotdb.commons.exception.IoTDBException;
 import org.apache.iotdb.rpc.TSStatusCode;
 
-public class SyncDeviceOwnerConflictException extends IoTDBException {
+public class PipeSinkException extends IoTDBException {
+
+  private static final long serialVersionUID = -2355881952697245662L;
 
-  private static final long serialVersionUID = -5037926672199248044L;
+  public PipeSinkException(String message, int errorCode) {
+    super(message, errorCode);
+  }
 
-  public SyncDeviceOwnerConflictException(String message) {
-    super(message, TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+  public PipeSinkException(String message) {
+    super(message, TSStatusCode.PIPESINK_ERROR.getStatusCode());
   }
 
-  public SyncDeviceOwnerConflictException(
-      String device, String correctOwner, String conflictOwner) {
+  public PipeSinkException(String attr, String value, String attrType) {
     super(
-        String.format(
-            "Device: [%s], correct owner: [%s], conflict owner: [%s]",
-            device, correctOwner, conflictOwner),
-        TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+        String.format("%s=%s has wrong format, require for %s.", attr, value, attrType),
+        TSStatusCode.PIPESINK_ERROR.getStatusCode());
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/LocalConfigNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/LocalConfigNode.java
index 34179e6581..4e795d9535 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/LocalConfigNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/LocalConfigNode.java
@@ -43,11 +43,14 @@ import org.apache.iotdb.db.metadata.utils.MetaUtils;
 import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.AppendTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.DropTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.PruneTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
 import org.apache.iotdb.db.rescon.MemTableManager;
+import org.apache.iotdb.db.sync.sender.manager.SchemaSyncManager;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 
@@ -57,6 +60,7 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -220,6 +224,9 @@ public class LocalConfigNode {
 
     schemaEngine.createSchemaRegion(
         storageGroup, partitionTable.allocateSchemaRegionId(storageGroup));
+    if (SchemaSyncManager.getInstance().isEnableSync()) {
+      SchemaSyncManager.getInstance().syncMetadataPlan(new SetStorageGroupPlan(storageGroup));
+    }
 
     if (!config.isEnableMemControl()) {
       MemTableManager.getInstance().addOrDeleteStorageGroup(1);
@@ -233,6 +240,10 @@ public class LocalConfigNode {
     for (Template template : templateManager.getTemplateMap().values()) {
       templateManager.unmarkStorageGroup(template, storageGroup.getFullPath());
     }
+    if (SchemaSyncManager.getInstance().isEnableSync()) {
+      SchemaSyncManager.getInstance()
+          .syncMetadataPlan(new DeleteStorageGroupPlan(Collections.singletonList(storageGroup)));
+    }
 
     if (!config.isEnableMemControl()) {
       MemTableManager.getInstance().addOrDeleteStorageGroup(-1);
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSG.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSG.java
index d0c9cf8e48..89e76793e9 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSG.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTreeBelowSG.java
@@ -627,6 +627,7 @@ public class MTreeBelowSG implements Serializable {
     collector.traverse();
     return result;
   }
+
   // endregion
 
   // region Interfaces for timeseries, measurement and schema info Query
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementCollector.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementCollector.java
index 67df29415f..d87439b758 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementCollector.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MeasurementCollector.java
@@ -86,4 +86,8 @@ public abstract class MeasurementCollector<T> extends CollectorTraverser<T> {
     retPath.setUnderAlignedEntity(par.getAsEntityMNode().isAligned());
     return retPath;
   }
+
+  protected boolean isUnderAlignedEntity() {
+    return traverseContext.peek().getAsEntityMNode().isAligned();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegion.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegion.java
index a067b17e30..47f56a5f12 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegion.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegion.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.db.metadata.schemaregion;
 
+import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.consensus.SchemaRegionId;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -74,6 +75,7 @@ import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
 import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
 import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.sync.sender.manager.SchemaSyncManager;
 import org.apache.iotdb.db.utils.SchemaUtils;
 import org.apache.iotdb.db.utils.TypeInferenceUtils;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
@@ -170,6 +172,7 @@ public class SchemaRegion implements ISchemaRegion {
   // device -> DeviceMNode
   private LoadingCache<PartialPath, IMNode> mNodeCache;
   private TagManager tagManager;
+  private SchemaSyncManager syncManager = SchemaSyncManager.getInstance();
 
   // region Interfaces and Implementation of initialization、snapshot、recover and clear
   public SchemaRegion(
@@ -396,6 +399,14 @@ public class SchemaRegion implements ISchemaRegion {
 
   // region Interfaces for schema region Info query and operation
 
+  public String getStorageGroupFullPath() {
+    return storageGroupFullPath;
+  }
+
+  public ConsensusGroupId getSchemaRegionId() {
+    return schemaRegionId;
+  }
+
   public synchronized void deleteSchemaRegion() throws MetadataException {
     // collect all the LeafMNode in this schema region
     List<IMeasurementMNode> leafMNodes = mtree.getAllMeasurementMNode();
@@ -471,6 +482,9 @@ public class SchemaRegion implements ISchemaRegion {
           }
           plan.setTagOffset(offset);
           logWriter.createTimeseries(plan);
+          if (syncManager.isEnableSync()) {
+            syncManager.syncMetadataPlan(plan);
+          }
         }
         if (offset != -1) {
           leafMNode.setOffset(offset);
@@ -610,6 +624,9 @@ public class SchemaRegion implements ISchemaRegion {
           }
           plan.setTagOffsets(tagOffsets);
           logWriter.createAlignedTimeseries(plan);
+          if (syncManager.isEnableSync()) {
+            syncManager.syncMetadataPlan(plan);
+          }
         }
         tagOffsets = plan.getTagOffsets();
         for (int i = 0; i < measurements.size(); i++) {
@@ -682,6 +699,9 @@ public class SchemaRegion implements ISchemaRegion {
         }
         deleteTimeSeriesPlan.setDeletePathList(Collections.singletonList(p));
         logWriter.deleteTimeseries(deleteTimeSeriesPlan);
+        if (syncManager.isEnableSync()) {
+          syncManager.syncMetadataPlan(deleteTimeSeriesPlan);
+        }
       }
     } catch (DeleteFailedException e) {
       failedNames.add(e.getName());
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
index 9ca03a20b5..b89ed113fa 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
@@ -200,6 +200,19 @@ public class SQLConstant {
 
   public static final int TOK_SHOW_QUERY_RESOURCE = 123;
 
+  public static final int TOK_CREATE_PIPESINK = 200;
+  public static final int TOK_DROP_PIPESINK = 201;
+  public static final int TOK_SHOW_PIPESINK = 202;
+  public static final int TOK_SHOW_PIPESINKTYPE = 203;
+  public static final int TOK_CREATE_PIPE = 204;
+  public static final int TOK_SHOW_PIPE = 205;
+  public static final int TOK_STOP_PIPE = 206;
+  public static final int TOK_START_PIPE = 207;
+  public static final int TOK_DROP_PIPE = 208;
+  public static final int TOK_SHOW_PIPE_SERVER = 209;
+  public static final int TOK_PIPE_SERVER_START = 210;
+  public static final int TOK_PIPE_SERVER_STOP = 211;
+
   public static final Map<Integer, String> tokenNames = new HashMap<>();
 
   public static String[] getSingleRootArray() {
@@ -286,6 +299,19 @@ public class SQLConstant {
     tokenNames.put(TOK_SCHEMA_TEMPLATE_SHOW_PATHS_USING, "TOK_SCHEMA_TEMPLATE_SHOW_PATHS_USING");
 
     tokenNames.put(TOK_SHOW_QUERY_RESOURCE, "TOK_SHOW_QUERY_RESOURCE");
+
+    tokenNames.put(TOK_CREATE_PIPESINK, "TOK_CREATE_PIPESINK");
+    tokenNames.put(TOK_DROP_PIPESINK, "TOK_DROP_PIPESINK");
+    tokenNames.put(TOK_SHOW_PIPESINK, "TOK_SHOW_PIPESINK");
+    tokenNames.put(TOK_SHOW_PIPESINKTYPE, "TOK_SHOW_PIPESINKTYPE");
+    tokenNames.put(TOK_CREATE_PIPE, "TOK_CREATE_PIPE");
+    tokenNames.put(TOK_SHOW_PIPE, "TOK_SHOW_PIPE");
+    tokenNames.put(TOK_STOP_PIPE, "TOK_STOP_PIPE");
+    tokenNames.put(TOK_START_PIPE, "TOK_START_PIPE");
+    tokenNames.put(TOK_DROP_PIPE, "TOK_DROP_PIPE");
+    tokenNames.put(TOK_SHOW_PIPE_SERVER, "TOK_SHOW_PIPE_SERVER");
+    tokenNames.put(TOK_PIPE_SERVER_START, "TOK_PIPE_SERVER_START");
+    tokenNames.put(TOK_PIPE_SERVER_STOP, "TOK_PIPE_SERVER_STOP");
   }
 
   public static boolean isReservedPath(PartialPath pathStr) {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
index 485ea58d8b..3cfb8e7cc7 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
@@ -53,12 +53,16 @@ import org.apache.iotdb.db.exception.metadata.PathNotExistException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.exception.sync.PipeException;
+import org.apache.iotdb.db.exception.sync.PipeServerException;
+import org.apache.iotdb.db.exception.sync.PipeSinkException;
 import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
 import org.apache.iotdb.db.metadata.path.MeasurementPath;
 import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.metadata.utils.MetaUtils;
 import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.logical.sys.AuthorOperator;
 import org.apache.iotdb.db.qp.logical.sys.AuthorOperator.AuthorType;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
@@ -73,6 +77,8 @@ import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateContinuousQueryPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateFunctionPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateMultiTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreatePipePlan;
+import org.apache.iotdb.db.qp.physical.sys.CreatePipeSinkPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTriggerPlan;
@@ -81,12 +87,14 @@ import org.apache.iotdb.db.qp.physical.sys.DeleteStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.DropContinuousQueryPlan;
 import org.apache.iotdb.db.qp.physical.sys.DropFunctionPlan;
+import org.apache.iotdb.db.qp.physical.sys.DropPipeSinkPlan;
 import org.apache.iotdb.db.qp.physical.sys.DropTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.DropTriggerPlan;
 import org.apache.iotdb.db.qp.physical.sys.FlushPlan;
 import org.apache.iotdb.db.qp.physical.sys.KillQueryPlan;
 import org.apache.iotdb.db.qp.physical.sys.LoadConfigurationPlan;
 import org.apache.iotdb.db.qp.physical.sys.OperateFilePlan;
+import org.apache.iotdb.db.qp.physical.sys.OperatePipePlan;
 import org.apache.iotdb.db.qp.physical.sys.PruneTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.SetSystemModePlan;
@@ -101,6 +109,9 @@ import org.apache.iotdb.db.qp.physical.sys.ShowLockInfoPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowNodesInTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowPathsSetTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowPathsUsingTemplatePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPipePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPipeServerPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPipeSinkPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTTLPlan;
@@ -108,6 +119,7 @@ import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.StartTriggerPlan;
 import org.apache.iotdb.db.qp.physical.sys.StopTriggerPlan;
 import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.FileReaderManager;
 import org.apache.iotdb.db.query.control.QueryResourceManager;
@@ -124,6 +136,10 @@ import org.apache.iotdb.db.query.udf.service.UDFRegistrationInformation;
 import org.apache.iotdb.db.query.udf.service.UDFRegistrationService;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.service.SettleService;
+import org.apache.iotdb.db.sync.receiver.ReceiverService;
+import org.apache.iotdb.db.sync.sender.pipe.Pipe;
+import org.apache.iotdb.db.sync.sender.pipe.PipeSink;
+import org.apache.iotdb.db.sync.sender.service.SenderService;
 import org.apache.iotdb.db.tools.TsFileRewriteTool;
 import org.apache.iotdb.db.utils.AuthUtils;
 import org.apache.iotdb.db.utils.FileLoaderUtils;
@@ -188,6 +204,15 @@ import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_FUNCTION_NAME;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_FUNCTION_TYPE;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_ITEM;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_LOCK_INFO;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PIPESINK_ATTRIBUTES;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PIPESINK_NAME;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PIPESINK_TYPE;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PIPE_CREATE_TIME;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PIPE_MSG;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PIPE_NAME;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PIPE_REMOTE;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PIPE_ROLE;
+import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PIPE_STATUS;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PRIVILEGE;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_ROLE;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_SCHEMA_TEMPLATE;
@@ -378,12 +403,48 @@ public class PlanExecutor implements IPlanExecutor {
         return true;
       case SHOW_QUERY_RESOURCE:
         return processShowQueryResource();
+      case CREATE_PIPESINK:
+        createPipeSink((CreatePipeSinkPlan) plan);
+        return true;
+      case DROP_PIPESINK:
+        dropPipeSink((DropPipeSinkPlan) plan);
+        return true;
+      case START_PIPE_SERVER:
+        return operateStartPipeServer();
+      case STOP_PIPE_SERVER:
+        return operateStopPipeServer();
+      case CREATE_PIPE:
+        createPipe((CreatePipePlan) plan);
+        return true;
+      case STOP_PIPE:
+      case START_PIPE:
+      case DROP_PIPE:
+        operatePipe((OperatePipePlan) plan);
+        return true;
       default:
         throw new UnsupportedOperationException(
             String.format("operation %s is not supported", plan.getOperatorName()));
     }
   }
 
+  private boolean operateStopPipeServer() throws QueryProcessException {
+    try {
+      ReceiverService.getInstance().stopPipeServer();
+    } catch (PipeServerException e) {
+      throw new QueryProcessException(e);
+    }
+    return true;
+  }
+
+  private boolean operateStartPipeServer() throws QueryProcessException {
+    try {
+      ReceiverService.getInstance().startPipeServer();
+    } catch (PipeServerException e) {
+      throw new QueryProcessException(e);
+    }
+    return true;
+  }
+
   private boolean createTemplate(CreateTemplatePlan createTemplatePlan)
       throws QueryProcessException {
     try {
@@ -672,11 +733,23 @@ public class PlanExecutor implements IPlanExecutor {
         return processShowPathsSetSchemaTemplate((ShowPathsSetTemplatePlan) showPlan);
       case PATHS_USING_SCHEMA_TEMPLATE:
         return processShowPathsUsingSchemaTemplate((ShowPathsUsingTemplatePlan) showPlan);
+      case PIPESINK:
+        return processShowPipeSink((ShowPipeSinkPlan) showPlan);
+      case PIPESINKTYPE:
+        return processShowPipeSinkType();
+      case PIPESERVER:
+        return processShowPipeServer((ShowPipeServerPlan) showPlan);
+      case PIPE:
+        return processShowPipes((ShowPipePlan) showPlan);
       default:
         throw new QueryProcessException(String.format("Unrecognized show plan %s", showPlan));
     }
   }
 
+  private QueryDataSet processShowPipeServer(ShowPipeServerPlan plan) {
+    return ReceiverService.getInstance().showPipeServer(plan);
+  }
+
   private QueryDataSet processCountNodes(CountPlan countPlan) throws MetadataException {
     int num =
         getNodesNumInGivenLevel(
@@ -1175,6 +1248,74 @@ public class PlanExecutor implements IPlanExecutor {
     listDataSet.putRecord(rowRecord);
   }
 
+  private QueryDataSet processShowPipeSink(ShowPipeSinkPlan plan) {
+    ListDataSet listDataSet =
+        new ListDataSet(
+            Arrays.asList(
+                new PartialPath(COLUMN_PIPESINK_NAME, false),
+                new PartialPath(COLUMN_PIPESINK_TYPE, false),
+                new PartialPath(COLUMN_PIPESINK_ATTRIBUTES, false)),
+            Arrays.asList(TSDataType.TEXT, TSDataType.TEXT, TSDataType.TEXT));
+    boolean showAll = "".equals(plan.getPipeSinkName());
+    for (PipeSink pipeSink : SenderService.getInstance().getAllPipeSink())
+      if (showAll || plan.getPipeSinkName().equals(pipeSink.getName())) {
+        RowRecord record = new RowRecord(0);
+        record.addField(Binary.valueOf(pipeSink.getName()), TSDataType.TEXT);
+        record.addField(Binary.valueOf(pipeSink.getType().name()), TSDataType.TEXT);
+        record.addField(Binary.valueOf(pipeSink.showAllAttributes()), TSDataType.TEXT);
+        listDataSet.putRecord(record);
+      }
+    return listDataSet;
+  }
+
+  private QueryDataSet processShowPipeSinkType() {
+    ListDataSet listDataSet =
+        new ListDataSet(
+            Arrays.asList(new PartialPath(COLUMN_PIPESINK_TYPE, false)),
+            Arrays.asList(TSDataType.TEXT));
+    for (PipeSink.Type type : PipeSink.Type.values()) {
+      RowRecord record = new RowRecord(0);
+      record.addField(Binary.valueOf(type.name()), TSDataType.TEXT);
+      listDataSet.putRecord(record);
+    }
+    return listDataSet;
+  }
+
+  private QueryDataSet processShowPipes(ShowPipePlan plan) {
+    ListDataSet listDataSet =
+        new ListDataSet(
+            Arrays.asList(
+                new PartialPath(COLUMN_PIPE_CREATE_TIME, false),
+                new PartialPath(COLUMN_PIPE_NAME, false),
+                new PartialPath(COLUMN_PIPE_ROLE, false),
+                new PartialPath(COLUMN_PIPE_REMOTE, false),
+                new PartialPath(COLUMN_PIPE_STATUS, false),
+                new PartialPath(COLUMN_PIPE_MSG, false)),
+            Arrays.asList(
+                TSDataType.TEXT,
+                TSDataType.TEXT,
+                TSDataType.TEXT,
+                TSDataType.TEXT,
+                TSDataType.TEXT,
+                TSDataType.TEXT));
+    boolean showAll = "".equals(plan.getPipeName());
+    for (Pipe pipe : SenderService.getInstance().getAllPipes())
+      if (showAll || plan.getPipeName().equals(pipe.getName())) {
+        RowRecord record = new RowRecord(0);
+        record.addField(
+            Binary.valueOf(DatetimeUtils.convertLongToDate(pipe.getCreateTime())), TSDataType.TEXT);
+        record.addField(Binary.valueOf(pipe.getName()), TSDataType.TEXT);
+        record.addField(Binary.valueOf(IoTDBConstant.SYNC_SENDER_ROLE), TSDataType.TEXT);
+        record.addField(Binary.valueOf(pipe.getPipeSink().getName()), TSDataType.TEXT);
+        record.addField(Binary.valueOf(pipe.getStatus().name()), TSDataType.TEXT);
+        record.addField(
+            Binary.valueOf(SenderService.getInstance().getPipeMsg(pipe)), TSDataType.TEXT);
+        listDataSet.putRecord(record);
+      }
+    ReceiverService.getInstance().showPipe(plan, listDataSet);
+    return listDataSet;
+  }
+
   @Override
   public void delete(DeletePlan deletePlan) throws QueryProcessException {
     AUDIT_LOGGER.info(
@@ -1267,7 +1408,7 @@ public class PlanExecutor implements IPlanExecutor {
         throw new QueryProcessException(e.getMessage());
       }
 
-      FileLoaderUtils.checkTsFileResource(tsFileResource);
+      FileLoaderUtils.loadOrGenerateResource(tsFileResource);
       if (UpgradeUtils.isNeedUpgrade(tsFileResource)) {
         throw new QueryProcessException(
             String.format(
@@ -2295,4 +2436,50 @@ public class PlanExecutor implements IPlanExecutor {
       throw new StorageEngineException(e.getMessage());
     }
   }
+
+  private void createPipeSink(CreatePipeSinkPlan plan) throws QueryProcessException {
+    try {
+      SenderService.getInstance().addPipeSink(plan);
+    } catch (PipeSinkException e) {
+      throw new QueryProcessException("Create pipeSink error.", e); // e will override the message
+    } catch (IllegalArgumentException e) {
+      throw new QueryProcessException(
+          "Do not support pipeSink type " + plan.getPipeSinkType() + ".");
+    }
+  }
+
+  private void dropPipeSink(DropPipeSinkPlan plan) throws QueryProcessException {
+    try {
+      SenderService.getInstance().dropPipeSink(plan.getPipeSinkName());
+    } catch (PipeSinkException e) {
+      throw new QueryProcessException("Can not drop pipeSink.", e);
+    }
+  }
+
+  private void createPipe(CreatePipePlan plan) throws QueryProcessException {
+    try {
+      SenderService.getInstance().addPipe(plan);
+    } catch (PipeException e) {
+      throw new QueryProcessException("Create pipe error.", e);
+    }
+  }
+
+  private void operatePipe(OperatePipePlan plan) throws QueryProcessException {
+    try {
+      if (Operator.OperatorType.STOP_PIPE.equals(plan.getOperatorType())) {
+        SenderService.getInstance().stopPipe(plan.getPipeName());
+      } else if (Operator.OperatorType.START_PIPE.equals(plan.getOperatorType())) {
+        SenderService.getInstance().startPipe(plan.getPipeName());
+      } else if (Operator.OperatorType.DROP_PIPE.equals(plan.getOperatorType())) {
+        SenderService.getInstance().dropPipe(plan.getPipeName());
+      } else {
+        throw new QueryProcessException(
+            String.format("Error operator type %s.", plan.getOperatorType()),
+            INTERNAL_SERVER_ERROR.getStatusCode());
+      }
+    } catch (PipeException e) {
+      throw new QueryProcessException(
+          String.format("%s pipe error.", plan.getOperatorType().name()), e);
+    }
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
index de439e989b..4aa910e361 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
@@ -194,6 +194,18 @@ public abstract class Operator {
     APPEND_TEMPLATE,
     DROP_TEMPLATE,
 
-    SHOW_QUERY_RESOURCE
+    SHOW_QUERY_RESOURCE,
+
+    CREATE_PIPESINK,
+    DROP_PIPESINK,
+    SHOW_PIPESINK,
+    SHOW_PIPESINKTYPE,
+    CREATE_PIPE,
+    SHOW_PIPE,
+    STOP_PIPE,
+    START_PIPE,
+    DROP_PIPE,
+    START_PIPE_SERVER,
+    STOP_PIPE_SERVER,
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/CreatePipeOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/CreatePipeOperator.java
new file mode 100644
index 0000000000..44dc6ee3d7
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/CreatePipeOperator.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.iotdb.db.qp.logical.sys;
+
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreatePipePlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+public class CreatePipeOperator extends Operator {
+  private String pipeName;
+  private String pipeSinkName;
+  private long startTime;
+  private Map<String, String> pipeAttributes;
+
+  public CreatePipeOperator(String pipeName, String pipeSinkName) {
+    super(SQLConstant.TOK_CREATE_PIPE);
+    this.operatorType = OperatorType.CREATE_PIPE;
+
+    this.pipeName = pipeName;
+    this.pipeSinkName = pipeSinkName;
+    this.startTime = 0;
+    this.pipeAttributes = new HashMap<>();
+  }
+
+  public void setStartTime(long startTime) {
+    this.startTime = startTime;
+  }
+
+  public void setPipeAttributes(Map<String, String> pipeAttributes) {
+    this.pipeAttributes = pipeAttributes;
+  }
+
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    CreatePipePlan plan = new CreatePipePlan(pipeName, pipeSinkName);
+    plan.setDataStartTimestamp(startTime);
+    Iterator<Map.Entry<String, String>> iterator = pipeAttributes.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<String, String> entry = iterator.next();
+      plan.addPipeAttribute(entry.getKey(), entry.getValue());
+    }
+    return plan;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/CreatePipeSinkOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/CreatePipeSinkOperator.java
new file mode 100644
index 0000000000..d0de62f325
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/CreatePipeSinkOperator.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.iotdb.db.qp.logical.sys;
+
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreatePipeSinkPlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+public class CreatePipeSinkOperator extends Operator {
+  private String pipeSinkName;
+  private String pipeSinkType;
+  private Map<String, String> pipeSinkAttributes;
+
+  public CreatePipeSinkOperator(String pipeSinkName, String pipeSinkType) {
+    super(SQLConstant.TOK_CREATE_PIPESINK);
+    this.pipeSinkName = pipeSinkName;
+    this.pipeSinkType = pipeSinkType;
+    pipeSinkAttributes = new HashMap<>();
+    this.operatorType = OperatorType.CREATE_PIPESINK;
+  }
+
+  public void setPipeSinkAttributes(Map<String, String> pipeSinkAttributes) {
+    this.pipeSinkAttributes = pipeSinkAttributes;
+  }
+
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    CreatePipeSinkPlan plan = new CreatePipeSinkPlan(pipeSinkName, pipeSinkType);
+    Iterator<Map.Entry<String, String>> iterator = pipeSinkAttributes.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<String, String> entry = iterator.next();
+      plan.addPipeSinkAttribute(entry.getKey(), entry.getValue());
+    }
+    return plan;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DropPipeOperator.java
similarity index 50%
copy from server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
copy to server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DropPipeOperator.java
index 271c177c5c..ea98dc017a 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DropPipeOperator.java
@@ -15,26 +15,30 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
+ *
  */
-package org.apache.iotdb.db.exception;
+package org.apache.iotdb.db.qp.logical.sys;
 
-import org.apache.iotdb.commons.exception.IoTDBException;
-import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.OperatePipePlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 
-public class SyncDeviceOwnerConflictException extends IoTDBException {
+public class DropPipeOperator extends Operator {
+  private String pipeName;
 
-  private static final long serialVersionUID = -5037926672199248044L;
+  public DropPipeOperator(String pipeName) {
+    super(SQLConstant.TOK_DROP_PIPE);
+    this.operatorType = OperatorType.DROP_PIPE;
 
-  public SyncDeviceOwnerConflictException(String message) {
-    super(message, TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+    this.pipeName = pipeName;
   }
 
-  public SyncDeviceOwnerConflictException(
-      String device, String correctOwner, String conflictOwner) {
-    super(
-        String.format(
-            "Device: [%s], correct owner: [%s], conflict owner: [%s]",
-            device, correctOwner, conflictOwner),
-        TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    return new OperatePipePlan(pipeName, operatorType);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DropPipeSinkOperator.java
similarity index 50%
copy from server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
copy to server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DropPipeSinkOperator.java
index 271c177c5c..9341e06b56 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DropPipeSinkOperator.java
@@ -15,26 +15,29 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
+ *
  */
-package org.apache.iotdb.db.exception;
-
-import org.apache.iotdb.commons.exception.IoTDBException;
-import org.apache.iotdb.rpc.TSStatusCode;
+package org.apache.iotdb.db.qp.logical.sys;
 
-public class SyncDeviceOwnerConflictException extends IoTDBException {
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.DropPipeSinkPlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 
-  private static final long serialVersionUID = -5037926672199248044L;
+public class DropPipeSinkOperator extends Operator {
+  private String pipeSinkName;
 
-  public SyncDeviceOwnerConflictException(String message) {
-    super(message, TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+  public DropPipeSinkOperator(String pipeSinkName) {
+    super(SQLConstant.TOK_DROP_PIPESINK);
+    this.pipeSinkName = pipeSinkName;
+    this.operatorType = OperatorType.DROP_PIPESINK;
   }
 
-  public SyncDeviceOwnerConflictException(
-      String device, String correctOwner, String conflictOwner) {
-    super(
-        String.format(
-            "Device: [%s], correct owner: [%s], conflict owner: [%s]",
-            device, correctOwner, conflictOwner),
-        TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    return new DropPipeSinkPlan(pipeSinkName);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowPipeOperator.java
similarity index 51%
copy from server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
copy to server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowPipeOperator.java
index 271c177c5c..b6070eea7a 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowPipeOperator.java
@@ -15,26 +15,31 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
+ *
  */
-package org.apache.iotdb.db.exception;
+package org.apache.iotdb.db.qp.logical.sys;
 
-import org.apache.iotdb.commons.exception.IoTDBException;
-import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPipePlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 
-public class SyncDeviceOwnerConflictException extends IoTDBException {
+public class ShowPipeOperator extends ShowOperator {
+  private String pipeName;
 
-  private static final long serialVersionUID = -5037926672199248044L;
+  public ShowPipeOperator() {
+    super(SQLConstant.TOK_SHOW_PIPE, OperatorType.SHOW_PIPE);
+    pipeName = "";
+  }
 
-  public SyncDeviceOwnerConflictException(String message) {
-    super(message, TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+  public void setPipeName(String pipeName) {
+    this.pipeName = pipeName;
   }
 
-  public SyncDeviceOwnerConflictException(
-      String device, String correctOwner, String conflictOwner) {
-    super(
-        String.format(
-            "Device: [%s], correct owner: [%s], conflict owner: [%s]",
-            device, correctOwner, conflictOwner),
-        TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    return new ShowPipePlan(pipeName);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowPipeServerOperator.java
similarity index 58%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java
copy to server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowPipeServerOperator.java
index e82ba05966..310c909692 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowPipeServerOperator.java
@@ -15,20 +15,24 @@
  * 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 org.apache.iotdb.commons.exception.StartupException;
-
-public interface SyncServerManagerMBean {
-
-  String getRPCServiceStatus();
+package org.apache.iotdb.db.qp.logical.sys;
 
-  int getRPCPort();
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPipeServerPlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 
-  void startService() throws StartupException;
+public class ShowPipeServerOperator extends ShowOperator {
 
-  void restartService() throws StartupException;
+  public ShowPipeServerOperator(int tokenIntType) {
+    super(tokenIntType);
+  }
 
-  void stopService();
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    return new ShowPipeServerPlan();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowPipeSinkOperator.java
similarity index 50%
copy from server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
copy to server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowPipeSinkOperator.java
index 271c177c5c..7a528c1c5a 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowPipeSinkOperator.java
@@ -15,26 +15,31 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
+ *
  */
-package org.apache.iotdb.db.exception;
+package org.apache.iotdb.db.qp.logical.sys;
 
-import org.apache.iotdb.commons.exception.IoTDBException;
-import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPipeSinkPlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 
-public class SyncDeviceOwnerConflictException extends IoTDBException {
+public class ShowPipeSinkOperator extends ShowOperator {
+  private String pipeSinkName;
 
-  private static final long serialVersionUID = -5037926672199248044L;
+  public ShowPipeSinkOperator() {
+    super(SQLConstant.TOK_SHOW_PIPESINK, OperatorType.SHOW_PIPESINK);
+    pipeSinkName = "";
+  }
 
-  public SyncDeviceOwnerConflictException(String message) {
-    super(message, TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+  public void setPipeSinkName(String pipeSinkName) {
+    this.pipeSinkName = pipeSinkName;
   }
 
-  public SyncDeviceOwnerConflictException(
-      String device, String correctOwner, String conflictOwner) {
-    super(
-        String.format(
-            "Device: [%s], correct owner: [%s], conflict owner: [%s]",
-            device, correctOwner, conflictOwner),
-        TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    return new ShowPipeSinkPlan(pipeSinkName);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowPipeSinkTypeOperator.java
similarity index 54%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
copy to server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowPipeSinkTypeOperator.java
index df3f78f4a6..0ff83f1d52 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowPipeSinkTypeOperator.java
@@ -15,19 +15,24 @@
  * 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 org.apache.iotdb.db.sync.receiver.load.IFileLoader;
-
-import java.io.File;
-import java.io.IOException;
-
-public interface ISyncReceiverLogAnalyzer {
+package org.apache.iotdb.db.qp.logical.sys;
 
-  void recoverAll() throws IOException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPipeSinkTypePlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 
-  boolean recover(String senderName) throws IOException;
+public class ShowPipeSinkTypeOperator extends ShowOperator {
+  public ShowPipeSinkTypeOperator() {
+    super(SQLConstant.TOK_SHOW_PIPESINKTYPE, OperatorType.SHOW_PIPESINKTYPE);
+  }
 
-  void scanLogger(IFileLoader loader, File syncLog, File loadLog);
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    return new ShowPipeSinkTypePlan();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/StartPipeOperator.java
similarity index 50%
copy from server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
copy to server/src/main/java/org/apache/iotdb/db/qp/logical/sys/StartPipeOperator.java
index 271c177c5c..d471122d41 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/StartPipeOperator.java
@@ -15,26 +15,30 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
+ *
  */
-package org.apache.iotdb.db.exception;
+package org.apache.iotdb.db.qp.logical.sys;
 
-import org.apache.iotdb.commons.exception.IoTDBException;
-import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.OperatePipePlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 
-public class SyncDeviceOwnerConflictException extends IoTDBException {
+public class StartPipeOperator extends Operator {
+  private String pipeName;
 
-  private static final long serialVersionUID = -5037926672199248044L;
+  public StartPipeOperator(String pipeName) {
+    super(SQLConstant.TOK_START_PIPE);
+    this.operatorType = OperatorType.START_PIPE;
 
-  public SyncDeviceOwnerConflictException(String message) {
-    super(message, TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+    this.pipeName = pipeName;
   }
 
-  public SyncDeviceOwnerConflictException(
-      String device, String correctOwner, String conflictOwner) {
-    super(
-        String.format(
-            "Device: [%s], correct owner: [%s], conflict owner: [%s]",
-            device, correctOwner, conflictOwner),
-        TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    return new OperatePipePlan(pipeName, operatorType);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/StartPipeServerOperator.java
similarity index 54%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java
copy to server/src/main/java/org/apache/iotdb/db/qp/logical/sys/StartPipeServerOperator.java
index e82ba05966..e23833a0dd 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/StartPipeServerOperator.java
@@ -16,19 +16,23 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.receiver;
+package org.apache.iotdb.db.qp.logical.sys;
 
-import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.StartPipeServerPlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 
-public interface SyncServerManagerMBean {
+public class StartPipeServerOperator extends Operator {
+  public StartPipeServerOperator(int tokenIntType) {
+    super(tokenIntType);
+    operatorType = OperatorType.START_PIPE_SERVER;
+  }
 
-  String getRPCServiceStatus();
-
-  int getRPCPort();
-
-  void startService() throws StartupException;
-
-  void restartService() throws StartupException;
-
-  void stopService();
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    return new StartPipeServerPlan();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/StopPipeOperator.java
similarity index 50%
rename from server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
rename to server/src/main/java/org/apache/iotdb/db/qp/logical/sys/StopPipeOperator.java
index 271c177c5c..c76500d46c 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/SyncDeviceOwnerConflictException.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/StopPipeOperator.java
@@ -15,26 +15,30 @@
  * KIND, either express or implied.  See the License for the
  * specific language governing permissions and limitations
  * under the License.
+ *
  */
-package org.apache.iotdb.db.exception;
+package org.apache.iotdb.db.qp.logical.sys;
 
-import org.apache.iotdb.commons.exception.IoTDBException;
-import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.OperatePipePlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 
-public class SyncDeviceOwnerConflictException extends IoTDBException {
+public class StopPipeOperator extends Operator {
+  private String pipeName;
 
-  private static final long serialVersionUID = -5037926672199248044L;
+  public StopPipeOperator(String pipeName) {
+    super(SQLConstant.TOK_STOP_PIPE);
+    this.operatorType = OperatorType.STOP_PIPE;
 
-  public SyncDeviceOwnerConflictException(String message) {
-    super(message, TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+    this.pipeName = pipeName;
   }
 
-  public SyncDeviceOwnerConflictException(
-      String device, String correctOwner, String conflictOwner) {
-    super(
-        String.format(
-            "Device: [%s], correct owner: [%s], conflict owner: [%s]",
-            device, correctOwner, conflictOwner),
-        TSStatusCode.SYNC_DEVICE_OWNER_CONFLICT_ERROR.getStatusCode());
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    return new OperatePipePlan(pipeName, operatorType);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/StopPipeServerOperator.java
similarity index 54%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java
copy to server/src/main/java/org/apache/iotdb/db/qp/logical/sys/StopPipeServerOperator.java
index e82ba05966..76746c5fcb 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/StopPipeServerOperator.java
@@ -16,19 +16,23 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.receiver;
+package org.apache.iotdb.db.qp.logical.sys;
 
-import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.StopPipeServerPlan;
+import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 
-public interface SyncServerManagerMBean {
+public class StopPipeServerOperator extends Operator {
+  public StopPipeServerOperator(int tokenIntType) {
+    super(tokenIntType);
+    operatorType = OperatorType.STOP_PIPE_SERVER;
+  }
 
-  String getRPCServiceStatus();
-
-  int getRPCPort();
-
-  void startService() throws StartupException;
-
-  void restartService() throws StartupException;
-
-  void stopService();
+  @Override
+  public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
+      throws QueryProcessException {
+    return new StopPipeServerPlan();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
index cc8f14bfec..5b8fa113f8 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
@@ -63,7 +63,9 @@ import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
 import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.StartPipeServerPlan;
 import org.apache.iotdb.db.qp.physical.sys.StartTriggerPlan;
+import org.apache.iotdb.db.qp.physical.sys.StopPipeServerPlan;
 import org.apache.iotdb.db.qp.physical.sys.StopTriggerPlan;
 import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
 import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
@@ -473,6 +475,12 @@ public abstract class PhysicalPlan implements IConsensusRequest {
         case SET_SYSTEM_MODE:
           plan = new SetSystemModePlan();
           break;
+        case START_PIPE_SERVER:
+          plan = new StartPipeServerPlan();
+          break;
+        case STOP_PIPE_SERVER:
+          plan = new StopPipeServerPlan();
+          break;
         default:
           throw new IOException("unrecognized log type " + type);
       }
@@ -542,6 +550,8 @@ public abstract class PhysicalPlan implements IConsensusRequest {
     UNSET_TEMPLATE,
     APPEND_TEMPLATE,
     PRUNE_TEMPLATE,
+    START_PIPE_SERVER,
+    STOP_PIPE_SERVER,
     DROP_TEMPLATE
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateAlignedTimeSeriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateAlignedTimeSeriesPlan.java
index 35a6fddb30..1ac61f5816 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateAlignedTimeSeriesPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateAlignedTimeSeriesPlan.java
@@ -27,6 +27,7 @@ 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.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -35,6 +36,7 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -80,6 +82,17 @@ public class CreateAlignedTimeSeriesPlan extends PhysicalPlan {
     this.canBeSplit = false;
   }
 
+  public CreateAlignedTimeSeriesPlan(
+      PartialPath prefixPath, String measurement, MeasurementSchema schema) {
+    super(Operator.OperatorType.CREATE_ALIGNED_TIMESERIES);
+    this.prefixPath = prefixPath;
+    this.measurements = Collections.singletonList(measurement);
+    this.dataTypes = Collections.singletonList(schema.getType());
+    this.encodings = Collections.singletonList(schema.getEncodingType());
+    this.compressors = Collections.singletonList(schema.getCompressor());
+    this.canBeSplit = false;
+  }
+
   public PartialPath getPrefixPath() {
     return prefixPath;
   }
@@ -269,7 +282,7 @@ public class CreateAlignedTimeSeriesPlan extends PhysicalPlan {
     for (CompressionType compressor : compressors) {
       buffer.put((byte) compressor.ordinal());
     }
-    for (Long tagOffset : tagOffsets) {
+    for (Long tagOffset : getTagOffsets()) {
       buffer.putLong(tagOffset);
     }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreatePipePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreatePipePlan.java
new file mode 100644
index 0000000000..638b5a93c4
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreatePipePlan.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.iotdb.db.qp.physical.sys;
+
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class CreatePipePlan extends PhysicalPlan {
+  private String pipeName;
+  private String pipeSinkName;
+  private long dataStartTimestamp;
+  private List<Pair<String, String>> pipeAttributes;
+
+  public CreatePipePlan(String pipeName, String pipeSinkName) {
+    super(Operator.OperatorType.CREATE_PIPE);
+    this.pipeName = pipeName;
+    this.pipeSinkName = pipeSinkName;
+    dataStartTimestamp = 0;
+    pipeAttributes = new ArrayList<>();
+  }
+
+  public void setDataStartTimestamp(long dataStartTimestamp) {
+    this.dataStartTimestamp = dataStartTimestamp;
+  }
+
+  public void addPipeAttribute(String attr, String value) {
+    pipeAttributes.add(new Pair<>(attr, value));
+  }
+
+  public String getPipeName() {
+    return pipeName;
+  }
+
+  public String getPipeSinkName() {
+    return pipeSinkName;
+  }
+
+  public long getDataStartTimestamp() {
+    return dataStartTimestamp;
+  }
+
+  public List<Pair<String, String>> getPipeAttributes() {
+    return pipeAttributes;
+  }
+
+  @Override
+  public List<? extends PartialPath> getPaths() {
+    return Collections.emptyList();
+  }
+
+  public static CreatePipePlan parseString(String parsedString) throws IOException {
+    String[] attributes = parsedString.split(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    if (attributes.length < 4) {
+      throw new IOException("Parsing CreatePipePlan error. Attributes is less than expected.");
+    }
+    CreatePipePlan plan = new CreatePipePlan(attributes[0], attributes[1]);
+    plan.setDataStartTimestamp(Long.parseLong(attributes[2]));
+    int size = (Integer.parseInt(attributes[3]) << 1);
+    if (attributes.length != (size + 4)) {
+      throw new IOException("Parsing CreatePipePlan error. Attributes number is wrong.");
+    }
+    for (int i = 0; i < size; i += 2) {
+      plan.addPipeAttribute(attributes[i + 4], attributes[i + 5]);
+    }
+    return plan;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append(pipeName).append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    builder.append(pipeSinkName).append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    builder.append(dataStartTimestamp).append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    builder.append(pipeAttributes.size()).append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    for (int i = 0; i < pipeAttributes.size(); i++) {
+      builder
+          .append(pipeAttributes.get(i).left)
+          .append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+      builder
+          .append(pipeAttributes.get(i).right)
+          .append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    }
+    return builder.toString();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreatePipeSinkPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreatePipeSinkPlan.java
new file mode 100644
index 0000000000..babfd83501
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreatePipeSinkPlan.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.iotdb.db.qp.physical.sys;
+
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class CreatePipeSinkPlan extends PhysicalPlan {
+  private String pipeSinkName;
+  private String pipeSinkType;
+  private List<Pair<String, String>> pipeSinkAttributes;
+
+  public CreatePipeSinkPlan(String pipeSinkName, String pipeSinkType) {
+    super(Operator.OperatorType.CREATE_PIPESINK);
+    this.pipeSinkName = pipeSinkName;
+    this.pipeSinkType = pipeSinkType;
+    pipeSinkAttributes = new ArrayList<>();
+  }
+
+  public void addPipeSinkAttribute(String attr, String value) {
+    pipeSinkAttributes.add(new Pair<>(attr, value));
+  }
+
+  public String getPipeSinkName() {
+    return pipeSinkName;
+  }
+
+  public String getPipeSinkType() {
+    return pipeSinkType;
+  }
+
+  public List<Pair<String, String>> getPipeSinkAttributes() {
+    return pipeSinkAttributes;
+  }
+
+  @Override
+  public List<? extends PartialPath> getPaths() {
+    return Collections.emptyList();
+  }
+
+  public static CreatePipeSinkPlan parseString(String parsedString) throws IOException {
+    String[] attributes = parsedString.split(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    if (attributes.length < 3) {
+      throw new IOException("Parsing CreatePipeSinkPlan error. Attributes is less than expected.");
+    }
+    CreatePipeSinkPlan plan = new CreatePipeSinkPlan(attributes[0], attributes[1]);
+    int size = (Integer.parseInt(attributes[2]) << 1);
+    if (attributes.length != (size + 3)) {
+      throw new IOException("Parsing CreatePipeSinkPlan error. Attributes number is wrong.");
+    }
+    for (int i = 0; i < size; i += 2) {
+      plan.addPipeSinkAttribute(attributes[i + 3], attributes[i + 4]);
+    }
+    return plan;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append(pipeSinkName).append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    builder.append(pipeSinkType).append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    builder.append(pipeSinkAttributes.size()).append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    for (int i = 0; i < pipeSinkAttributes.size(); i++) {
+      builder
+          .append(pipeSinkAttributes.get(i).left)
+          .append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+      builder
+          .append(pipeSinkAttributes.get(i).right)
+          .append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    }
+    return builder.toString();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java
index aceae26e51..a46cb071f3 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java
@@ -26,6 +26,7 @@ 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.utils.ReadWriteIOUtils;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -77,6 +78,15 @@ public class CreateTimeSeriesPlan extends PhysicalPlan {
     }
   }
 
+  public CreateTimeSeriesPlan(PartialPath path, MeasurementSchema schema) {
+    super(Operator.OperatorType.CREATE_TIMESERIES);
+    this.path = path;
+    this.dataType = schema.getType();
+    this.encoding = schema.getEncodingType();
+    this.compressor = schema.getCompressor();
+    canBeSplit = false;
+  }
+
   public PartialPath getPath() {
     return path;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/DropPipeSinkPlan.java
similarity index 55%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
copy to server/src/main/java/org/apache/iotdb/db/qp/physical/sys/DropPipeSinkPlan.java
index df3f78f4a6..9d727d7c53 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/DropPipeSinkPlan.java
@@ -15,19 +15,31 @@
  * 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;
+package org.apache.iotdb.db.qp.physical.sys;
 
-import org.apache.iotdb.db.sync.receiver.load.IFileLoader;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 
-import java.io.File;
-import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
 
-public interface ISyncReceiverLogAnalyzer {
+public class DropPipeSinkPlan extends PhysicalPlan {
+  private String pipeSinkName;
 
-  void recoverAll() throws IOException;
+  public DropPipeSinkPlan(String pipeSinkName) {
+    super(Operator.OperatorType.DROP_PIPESINK);
+    this.pipeSinkName = pipeSinkName;
+  }
 
-  boolean recover(String senderName) throws IOException;
+  public String getPipeSinkName() {
+    return pipeSinkName;
+  }
 
-  void scanLogger(IFileLoader loader, File syncLog, File loadLog);
+  @Override
+  public List<? extends PartialPath> getPaths() {
+    return Collections.emptyList();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/OperatePipePlan.java
similarity index 57%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
copy to server/src/main/java/org/apache/iotdb/db/qp/physical/sys/OperatePipePlan.java
index df3f78f4a6..f2d26fe76f 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/OperatePipePlan.java
@@ -15,19 +15,31 @@
  * 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;
+package org.apache.iotdb.db.qp.physical.sys;
 
-import org.apache.iotdb.db.sync.receiver.load.IFileLoader;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 
-import java.io.File;
-import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
 
-public interface ISyncReceiverLogAnalyzer {
+public class OperatePipePlan extends PhysicalPlan {
+  private String pipeName;
 
-  void recoverAll() throws IOException;
+  public OperatePipePlan(String pipeName, Operator.OperatorType type) {
+    super(type);
+    this.pipeName = pipeName;
+  }
 
-  boolean recover(String senderName) throws IOException;
+  public String getPipeName() {
+    return pipeName;
+  }
 
-  void scanLogger(IFileLoader loader, File syncLog, File loadLog);
+  @Override
+  public List<? extends PartialPath> getPaths() {
+    return Collections.emptyList();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPipePlan.java
similarity index 68%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
copy to server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPipePlan.java
index df3f78f4a6..bbdbdf2043 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPipePlan.java
@@ -15,19 +15,19 @@
  * 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 org.apache.iotdb.db.sync.receiver.load.IFileLoader;
-
-import java.io.File;
-import java.io.IOException;
-
-public interface ISyncReceiverLogAnalyzer {
+package org.apache.iotdb.db.qp.physical.sys;
 
-  void recoverAll() throws IOException;
+public class ShowPipePlan extends ShowPlan {
+  private String pipeName;
 
-  boolean recover(String senderName) throws IOException;
+  public ShowPipePlan(String pipeName) {
+    super(ShowContentType.PIPE);
+    this.pipeName = pipeName;
+  }
 
-  void scanLogger(IFileLoader loader, File syncLog, File loadLog);
+  public String getPipeName() {
+    return pipeName;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPipeServerPlan.java
similarity index 82%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
copy to server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPipeServerPlan.java
index 24b33dfc95..67b664c1b7 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPipeServerPlan.java
@@ -16,10 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.receiver.load;
+package org.apache.iotdb.db.qp.physical.sys;
 
-public enum LoadType {
-  DELETE,
-  ADD,
-  NONE
+public class ShowPipeServerPlan extends ShowPlan {
+
+  public ShowPipeServerPlan() {
+    super(ShowContentType.PIPESERVER);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPipeSinkPlan.java
similarity index 71%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java
copy to server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPipeSinkPlan.java
index e82ba05966..038a681349 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPipeSinkPlan.java
@@ -15,20 +15,19 @@
  * 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 org.apache.iotdb.commons.exception.StartupException;
-
-public interface SyncServerManagerMBean {
-
-  String getRPCServiceStatus();
-
-  int getRPCPort();
+package org.apache.iotdb.db.qp.physical.sys;
 
-  void startService() throws StartupException;
+public class ShowPipeSinkPlan extends ShowPlan {
+  private String pipeSinkName;
 
-  void restartService() throws StartupException;
+  public ShowPipeSinkPlan(String pipeSinkName) {
+    super(ShowContentType.PIPESINK);
+    this.pipeSinkName = pipeSinkName;
+  }
 
-  void stopService();
+  public String getPipeSinkName() {
+    return pipeSinkName;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPipeSinkTypePlan.java
similarity index 81%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
copy to server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPipeSinkTypePlan.java
index 24b33dfc95..fe29622c66 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPipeSinkTypePlan.java
@@ -15,11 +15,12 @@
  * 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;
+package org.apache.iotdb.db.qp.physical.sys;
 
-public enum LoadType {
-  DELETE,
-  ADD,
-  NONE
+public class ShowPipeSinkTypePlan extends ShowPlan {
+  public ShowPipeSinkTypePlan() {
+    super(ShowContentType.PIPESINKTYPE);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPlan.java
index 8cf733739b..bad80321df 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPlan.java
@@ -126,6 +126,10 @@ public class ShowPlan extends PhysicalPlan {
     SCHEMA_TEMPLATE,
     NODES_IN_SCHEMA_TEMPLATE,
     PATHS_SET_SCHEMA_TEMPLATE,
-    PATHS_USING_SCHEMA_TEMPLATE
+    PATHS_USING_SCHEMA_TEMPLATE,
+    PIPESINK,
+    PIPESINKTYPE,
+    PIPE,
+    PIPESERVER
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StartPipeServerPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StartPipeServerPlan.java
new file mode 100644
index 0000000000..a08e5b62bd
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StartPipeServerPlan.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.qp.physical.sys;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+public class StartPipeServerPlan extends PhysicalPlan {
+
+  public StartPipeServerPlan() {
+    super(Operator.OperatorType.START_PIPE_SERVER);
+    canBeSplit = false;
+  }
+
+  @Override
+  public List<? extends PartialPath> getPaths() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public void serialize(DataOutputStream stream) throws IOException {
+    stream.writeByte((byte) PhysicalPlanType.START_PIPE_SERVER.ordinal());
+  }
+
+  @Override
+  public void serializeImpl(ByteBuffer buffer) {
+    buffer.put((byte) PhysicalPlanType.START_PIPE_SERVER.ordinal());
+  }
+
+  @Override
+  public void deserialize(ByteBuffer buffer) throws IllegalPathException {}
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StopPipeServerPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StopPipeServerPlan.java
new file mode 100644
index 0000000000..3365f8e4dc
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/StopPipeServerPlan.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.qp.physical.sys;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+public class StopPipeServerPlan extends PhysicalPlan {
+
+  public StopPipeServerPlan() {
+    super(Operator.OperatorType.STOP_PIPE_SERVER);
+    canBeSplit = false;
+  }
+
+  @Override
+  public List<? extends PartialPath> getPaths() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public void serialize(DataOutputStream stream) throws IOException {
+    stream.writeByte((byte) PhysicalPlanType.STOP_PIPE_SERVER.ordinal());
+  }
+
+  @Override
+  public void serializeImpl(ByteBuffer buffer) {
+    buffer.put((byte) PhysicalPlanType.STOP_PIPE_SERVER.ordinal());
+  }
+
+  @Override
+  public void deserialize(ByteBuffer buffer) throws IllegalPathException {}
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
index 3bcb35e495..029df297a8 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
@@ -55,6 +55,28 @@ import org.apache.iotdb.db.qp.logical.crud.WhereComponent;
 import org.apache.iotdb.db.qp.logical.sys.*;
 import org.apache.iotdb.db.qp.logical.sys.AlterTimeSeriesOperator.AlterType;
 import org.apache.iotdb.db.qp.logical.sys.AuthorOperator.AuthorType;
+import org.apache.iotdb.db.qp.logical.sys.ClearCacheOperator;
+import org.apache.iotdb.db.qp.logical.sys.CountOperator;
+import org.apache.iotdb.db.qp.logical.sys.CreateAlignedTimeSeriesOperator;
+import org.apache.iotdb.db.qp.logical.sys.CreateContinuousQueryOperator;
+import org.apache.iotdb.db.qp.logical.sys.CreateFunctionOperator;
+import org.apache.iotdb.db.qp.logical.sys.CreatePipeOperator;
+import org.apache.iotdb.db.qp.logical.sys.CreatePipeSinkOperator;
+import org.apache.iotdb.db.qp.logical.sys.CreateTemplateOperator;
+import org.apache.iotdb.db.qp.logical.sys.CreateTimeSeriesOperator;
+import org.apache.iotdb.db.qp.logical.sys.CreateTriggerOperator;
+import org.apache.iotdb.db.qp.logical.sys.DataAuthOperator;
+import org.apache.iotdb.db.qp.logical.sys.DeletePartitionOperator;
+import org.apache.iotdb.db.qp.logical.sys.DeleteStorageGroupOperator;
+import org.apache.iotdb.db.qp.logical.sys.DeleteTimeSeriesOperator;
+import org.apache.iotdb.db.qp.logical.sys.DropContinuousQueryOperator;
+import org.apache.iotdb.db.qp.logical.sys.DropFunctionOperator;
+import org.apache.iotdb.db.qp.logical.sys.DropPipeOperator;
+import org.apache.iotdb.db.qp.logical.sys.DropPipeSinkOperator;
+import org.apache.iotdb.db.qp.logical.sys.DropTriggerOperator;
+import org.apache.iotdb.db.qp.logical.sys.FlushOperator;
+import org.apache.iotdb.db.qp.logical.sys.KillQueryOperator;
+import org.apache.iotdb.db.qp.logical.sys.LoadConfigurationOperator;
 import org.apache.iotdb.db.qp.logical.sys.LoadConfigurationOperator.LoadConfigurationOperatorType;
 import org.apache.iotdb.db.qp.logical.sys.LoadDataOperator;
 import org.apache.iotdb.db.qp.logical.sys.LoadFilesOperator;
@@ -72,11 +94,16 @@ import org.apache.iotdb.db.qp.logical.sys.ShowDevicesOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowFunctionsOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowLockInfoOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowOperator;
+import org.apache.iotdb.db.qp.logical.sys.ShowPipeOperator;
+import org.apache.iotdb.db.qp.logical.sys.ShowPipeSinkOperator;
+import org.apache.iotdb.db.qp.logical.sys.ShowPipeSinkTypeOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowStorageGroupOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowTTLOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowTimeSeriesOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowTriggersOperator;
+import org.apache.iotdb.db.qp.logical.sys.StartPipeOperator;
 import org.apache.iotdb.db.qp.logical.sys.StartTriggerOperator;
+import org.apache.iotdb.db.qp.logical.sys.StopPipeOperator;
 import org.apache.iotdb.db.qp.logical.sys.StopTriggerOperator;
 import org.apache.iotdb.db.qp.logical.sys.UnSetTTLOperator;
 import org.apache.iotdb.db.qp.logical.sys.UnloadFileOperator;
@@ -127,6 +154,7 @@ import java.util.Arrays;
 import java.util.EnumMap;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -2162,7 +2190,167 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
         new File(parseFilePath(ctx.dstFileDir.getText())));
   }
 
-  /** 6. Common Clauses */
+  /** 6. sync statement */
+  @Override
+  public Operator visitCreatePipeSink(IoTDBSqlParser.CreatePipeSinkContext ctx) {
+    CreatePipeSinkOperator operator =
+        new CreatePipeSinkOperator(
+            StringEscapeUtils.unescapeJava(ctx.pipeSinkName.getText()),
+            StringEscapeUtils.unescapeJava(ctx.pipeSinkType.getText()));
+    if (ctx.syncAttributeClauses() != null) {
+      operator.setPipeSinkAttributes(parseSyncAttributeClauses(ctx.syncAttributeClauses()));
+    }
+    return operator;
+  }
+
+  private Map<String, String> parseSyncAttributeClauses(
+      IoTDBSqlParser.SyncAttributeClausesContext ctx) {
+    Map<String, String> attributes = new HashMap<>();
+    Iterator<IoTDBSqlParser.PropertyClauseContext> propertyClauseIterator =
+        ctx.propertyClause().iterator();
+    while (propertyClauseIterator.hasNext()) {
+      IoTDBSqlParser.PropertyClauseContext propertyClause = propertyClauseIterator.next();
+      attributes.put(propertyClause.name.getText(), propertyClause.value.getText());
+    }
+    return attributes;
+  }
+
+  @Override
+  public Operator visitDropPipeSink(IoTDBSqlParser.DropPipeSinkContext ctx) {
+    DropPipeSinkOperator operator =
+        new DropPipeSinkOperator(StringEscapeUtils.unescapeJava(ctx.pipeSinkName.getText()));
+    return operator;
+  }
+
+  @Override
+  public Operator visitShowPipeSink(IoTDBSqlParser.ShowPipeSinkContext ctx) {
+    ShowPipeSinkOperator operator = new ShowPipeSinkOperator();
+    if (ctx.pipeSinkName != null) {
+      operator.setPipeSinkName(StringEscapeUtils.unescapeJava(ctx.pipeSinkName.getText()));
+    }
+    return operator;
+  }
+
+  @Override
+  public Operator visitShowPipeSinkType(IoTDBSqlParser.ShowPipeSinkTypeContext ctx) {
+    ShowPipeSinkTypeOperator operator = new ShowPipeSinkTypeOperator();
+    return operator;
+  }
+
+  @Override
+  public Operator visitCreatePipe(IoTDBSqlParser.CreatePipeContext ctx) throws SQLParserException {
+    CreatePipeOperator operator =
+        new CreatePipeOperator(
+            StringEscapeUtils.unescapeJava(ctx.pipeName.getText()),
+            StringEscapeUtils.unescapeJava(ctx.pipeSinkName.getText()));
+
+    if (ctx.selectStatement() != null) {
+      parseSelectStatementForPipe(ctx.selectStatement(), operator);
+    }
+    if (ctx.syncAttributeClauses() != null) {
+      operator.setPipeAttributes(parseSyncAttributeClauses(ctx.syncAttributeClauses()));
+    }
+    return operator;
+  }
+
+  @Override
+  public Operator visitShowPipe(IoTDBSqlParser.ShowPipeContext ctx) {
+    ShowPipeOperator operator = new ShowPipeOperator();
+    if (ctx.pipeName != null) {
+      operator.setPipeName(StringEscapeUtils.unescapeJava(ctx.pipeName.getText()));
+    }
+    return operator;
+  }
+
+  @Override
+  public Operator visitStopPipe(IoTDBSqlParser.StopPipeContext ctx) {
+    return new StopPipeOperator(StringEscapeUtils.unescapeJava(ctx.pipeName.getText()));
+  }
+
+  @Override
+  public Operator visitStartPipe(IoTDBSqlParser.StartPipeContext ctx) {
+    return new StartPipeOperator(StringEscapeUtils.unescapeJava(ctx.pipeName.getText()));
+  }
+
+  @Override
+  public Operator visitDropPipe(IoTDBSqlParser.DropPipeContext ctx) {
+    return new DropPipeOperator(StringEscapeUtils.unescapeJava(ctx.pipeName.getText()));
+  }
+
+  private void parseSelectStatementForPipe(
+      IoTDBSqlParser.SelectStatementContext ctx, CreatePipeOperator operator)
+      throws SQLParserException {
+    if (ctx.TRACING() != null || ctx.intoClause() != null || ctx.specialClause() != null) {
+      throw new SQLParserException("Not support for this sql in pipe.");
+    }
+
+    // parse select
+    IoTDBSqlParser.SelectClauseContext selectCtx = ctx.selectClause();
+    if (selectCtx.LAST() != null
+        || selectCtx.topClause() != null
+        || selectCtx.resultColumn().size() != 1) {
+      throw new SQLParserException("Not support for this sql in pipe.");
+    }
+    IoTDBSqlParser.ResultColumnContext resultColumnCtx = selectCtx.resultColumn(0);
+    if (resultColumnCtx.AS() != null
+        || !IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD.equals(
+            resultColumnCtx.expression().getText())) {
+      throw new SQLParserException("Not support for this sql in pipe.");
+    }
+
+    // parse from
+    IoTDBSqlParser.FromClauseContext fromCtx = ctx.fromClause();
+    if (fromCtx.prefixPath().size() != 1
+        || !IoTDBConstant.PATH_ROOT.equals(fromCtx.prefixPath(0).getText())) {
+      throw new SQLParserException("Not support for this sql in pipe.");
+    }
+
+    // parse where
+    IoTDBSqlParser.WhereClauseContext whereCtx = ctx.whereClause();
+    if (whereCtx != null) {
+      if (whereCtx.orExpression() == null || whereCtx.indexPredicateClause() != null) {
+        throw new SQLParserException("Not support for this sql in pipe.");
+      }
+      IoTDBSqlParser.OrExpressionContext orExpressionCtx = whereCtx.orExpression();
+      if (orExpressionCtx.andExpression().size() != 1) {
+        throw new SQLParserException("Not support for this sql in pipe.");
+      }
+      IoTDBSqlParser.AndExpressionContext andExpressionCtx = orExpressionCtx.andExpression(0);
+      if (andExpressionCtx.predicate().size() != 1) {
+        throw new SQLParserException("Not support for this sql in pipe.");
+      }
+      IoTDBSqlParser.PredicateContext predicateCtx = andExpressionCtx.predicate(0);
+      if (predicateCtx.comparisonOperator() == null
+          || (predicateCtx.comparisonOperator().OPERATOR_GTE() == null
+              && predicateCtx.comparisonOperator().OPERATOR_GT() == null)
+          || predicateCtx.suffixPath() != null
+          || predicateCtx.fullPath() != null) {
+        throw new SQLParserException("Not support for this sql in pipe.");
+      }
+      IoTDBSqlParser.ConstantContext constantCtx = predicateCtx.constant();
+      if (constantCtx.dateExpression() == null) {
+        throw new SQLParserException("data type error for time limit");
+      }
+      operator.setStartTime(parseDateExpression(constantCtx.dateExpression()));
+    }
+  }
+
+  @Override
+  public Operator visitStartPipeServer(IoTDBSqlParser.StartPipeServerContext ctx) {
+    return new StartPipeServerOperator(SQLConstant.TOK_PIPE_SERVER_START);
+  }
+
+  @Override
+  public Operator visitStopPipeServer(IoTDBSqlParser.StopPipeServerContext ctx) {
+    return new StopPipeServerOperator(SQLConstant.TOK_PIPE_SERVER_STOP);
+  }
+
+  @Override
+  public Operator visitShowPipeServer(IoTDBSqlParser.ShowPipeServerContext ctx) {
+    return new ShowPipeServerOperator(SQLConstant.TOK_SHOW_PIPE_SERVER);
+  }
+
+  /** 7. Common Clauses */
 
   // IoTDB Objects
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java b/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java
index fb46eb5ab8..6624fdd3c9 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java
@@ -25,6 +25,7 @@ import org.apache.iotdb.db.query.control.SessionManager;
 
 import java.time.DateTimeException;
 import java.time.Instant;
+import java.time.LocalDateTime;
 import java.time.ZoneId;
 import java.time.ZoneOffset;
 import java.time.ZonedDateTime;
@@ -659,6 +660,20 @@ public class DatetimeUtils {
     }
   }
 
+  public static String convertLongToDate(long timestamp) {
+    String timePrecision = IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision();
+    switch (timePrecision) {
+      case "ns":
+        timestamp /= 1000_000_000;
+        break;
+      case "us":
+        timestamp /= 1000_000;
+        break;
+    }
+    return LocalDateTime.ofInstant(Instant.ofEpochMilli(timestamp), ZoneId.systemDefault())
+        .toString();
+  }
+
   public static ZoneOffset toZoneOffset(ZoneId zoneId) {
     return zoneId.getRules().getOffset(Instant.now());
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/service/DataNode.java b/server/src/main/java/org/apache/iotdb/db/service/DataNode.java
index 57cfdd1354..5d022f9692 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/DataNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/DataNode.java
@@ -88,8 +88,6 @@ public class DataNode implements DataNodeMBean {
   protected void serverCheckAndInit() throws ConfigurationException, IOException {
     IoTDBConfigCheck.getInstance().checkConfig();
     IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
-
-    config.setSyncEnable(false);
     // TODO: check configuration for data node
 
     // if client ip is the default address, set it same with internal ip
diff --git a/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java b/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java
index 5cdc3f88a0..f8c8839444 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java
@@ -48,7 +48,8 @@ import org.apache.iotdb.db.rescon.SystemInfo;
 import org.apache.iotdb.db.service.basic.ServiceProvider;
 import org.apache.iotdb.db.service.basic.StandaloneServiceProvider;
 import org.apache.iotdb.db.service.metrics.MetricsService;
-import org.apache.iotdb.db.sync.receiver.SyncServerManager;
+import org.apache.iotdb.db.sync.receiver.ReceiverService;
+import org.apache.iotdb.db.sync.sender.service.SenderService;
 import org.apache.iotdb.db.wal.WALManager;
 
 import org.slf4j.Logger;
@@ -139,6 +140,8 @@ public class IoTDB implements IoTDBMBean {
     registerManager.register(TemporaryQueryDataFileService.getInstance());
     registerManager.register(UDFClassLoaderManager.getInstance());
     registerManager.register(UDFRegistrationService.getInstance());
+    registerManager.register(ReceiverService.getInstance());
+    registerManager.register(MetricsService.getInstance());
 
     // in cluster mode, RPC service is not enabled.
     if (IoTDBDescriptor.getInstance().getConfig().isEnableRpcService()) {
@@ -165,7 +168,7 @@ public class IoTDB implements IoTDBMBean {
       }
     }
 
-    registerManager.register(SyncServerManager.getInstance());
+    registerManager.register(SenderService.getInstance());
     registerManager.register(UpgradeSevice.getINSTANCE());
     registerManager.register(SettleService.getINSTANCE());
     registerManager.register(TriggerRegistrationService.getInstance());
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncConstant.java b/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncConstant.java
index 5d538ff78f..a0c63590af 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncConstant.java
@@ -21,64 +21,55 @@ package org.apache.iotdb.db.sync.conf;
 import org.apache.iotdb.rpc.RpcUtils;
 
 public class SyncConstant {
-
-  private SyncConstant() {}
-
-  public static final String CONFIG_NAME = "iotdb-sync-client.properties";
-
-  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 = "SHA-256";
+  /** common */
+  public static final String PIPE_LOG_DIR_NAME = "pipe-log";
 
   public static final String SYNC_DIR_NAME_SEPARATOR = "_";
 
-  /** Split data file, block size at each transmission */
-  public static final int DATA_CHUNK_SIZE =
-      Math.min(64 * 1024 * 1024, RpcUtils.THRIFT_FRAME_MAX_SIZE);
-
-  // sender section
-
-  public static final String LOCK_FILE_NAME = "sync_lock";
-
-  public static final String UUID_FILE_NAME = "uuid.txt";
+  /** sender */
+  public static final String DEFAULT_PIPE_SINK_IP = "127.0.0.1";
 
-  public static final String SCHEMA_POS_FILE_NAME = "sync_schema_pos";
+  public static final int DEFAULT_PIPE_SINK_PORT = 6670;
 
-  public static final String LAST_LOCAL_FILE_NAME = "last_local_files.txt";
+  public static final String SENDER_PIPE_DIR_NAME = "sender";
+  public static final String FINISH_COLLECT_LOCK_NAME = "finishCollect.lock";
+  public static final String HISTORY_PIPE_LOG_DIR_NAME = "history-" + PIPE_LOG_DIR_NAME;
+  public static final Long DEFAULT_HEARTBEAT_DELAY_SECONDS = 10 * 60L;
+  public static final Long DEFAULT_WAITING_FOR_TSFILE_CLOSE_MILLISECONDS = 500L;
+  public static final Long DEFAULT_WAITING_FOR_TSFILE_RETRY_NUMBER = 10L;
+  public static final Long DEFAULT_WAITING_FOR_STOP_MILLISECONDS = 100L;
+  public static final String MODS_OFFSET_FILE_SUFFIX = ".offset";
+  public static final String PIPE_LOG_NAME_SUFFIX = "_pipe.log";
+  public static final Long DEFAULT_PIPE_LOG_SIZE_IN_BYTE = 10485760L;
+  public static final String COMMIT_LOG_NAME = "commit.log";
 
-  public static final String CURRENT_LOCAL_FILE_NAME = "current_local_files.txt";
+  public static final String UNKNOWN_IP = "UNKNOWN IP";
 
-  public static final String DATA_SNAPSHOT_NAME = "snapshot";
+  public static final String SENDER_LOG_NAME = "senderService.log";
+  public static final String PLAN_SERIALIZE_SPLIT_CHARACTER = ",";
+  public static final String SENDER_LOG_SPLIT_CHARACTER = " ";
+  public static final int MESSAGE_LENGTH_LIMIT = 200;
 
-  public static final String SYNC_LOG_NAME = "sync.log";
+  public static String getPipeLogName(long serialNumber) {
+    return serialNumber + PIPE_LOG_NAME_SUFFIX;
+  }
 
-  private static final SyncSenderConfig CONFIG = SyncSenderDescriptor.getInstance().getConfig();
+  public static Long getSerialNumberFromPipeLogName(String pipeLogName) {
+    return Long.parseLong(pipeLogName.split("_")[0]);
+  }
 
-  public static final long SYNC_PROCESS_DELAY = 0;
+  /** transport */
 
-  public static final long SYNC_MONITOR_DELAY = CONFIG.getSyncPeriodInSecond();
-
-  public static final long SYNC_PROCESS_PERIOD = CONFIG.getSyncPeriodInSecond();
-
-  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";
-
-  public static final int SUCCESS_CODE = 1;
+  // Split data file, block size at each transmission */
+  public static final int DATA_CHUNK_SIZE =
+      Math.min(64 * 1024 * 1024, RpcUtils.THRIFT_FRAME_MAX_SIZE);
 
-  public static final int ERROR_CODE = -1;
+  /** receiver */
+  public static final String SYNC_SYS_DIR = "sys";
 
-  public static final int CONFLICT_CODE = -2;
+  public static final String RECEIVER_DIR = "receiver";
+  public static final String RECEIVER_LOG_NAME = "receiverService.log";
+  public static final String RECEIVER_MSG_LOG_NAME = "receiverMessage.log";
+  public static final String FILE_DATA_DIR_NAME = "file-data";
+  public static final String IP_SEPARATOR = "\\.";
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncPathUtil.java b/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncPathUtil.java
new file mode 100644
index 0000000000..0889354c7d
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncPathUtil.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.sync.conf;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+
+import java.io.File;
+import java.io.IOException;
+
+/** Util for path generation in sync module */
+public class SyncPathUtil {
+
+  private SyncPathUtil() {
+    // forbidding instantiation
+  }
+
+  /** sender */
+  public static String getSenderPipeDir(String pipeName, long createTime) {
+    return IoTDBDescriptor.getInstance().getConfig().getSyncDir()
+        + File.separator
+        + SyncConstant.SENDER_PIPE_DIR_NAME
+        + String.format("-%s-%d", pipeName, createTime);
+  }
+
+  public static String getSenderHistoryPipeDataDir(String pipeName, long createTime) {
+    return getSenderPipeDir(pipeName, createTime)
+        + File.separator
+        + SyncConstant.HISTORY_PIPE_LOG_DIR_NAME;
+  }
+
+  public static String getSenderRealTimePipeDataDir(String pipeName, long createTime) {
+    return getSenderPipeDir(pipeName, createTime) + File.separator + SyncConstant.PIPE_LOG_DIR_NAME;
+  }
+
+  /** receiver */
+  public static String getReceiverPipeLogDir(String pipeName, String remoteIp, long createTime) {
+    return getReceiverPipeDir(pipeName, remoteIp, createTime)
+        + File.separator
+        + SyncConstant.PIPE_LOG_DIR_NAME;
+  }
+
+  public static String getReceiverFileDataDir(String pipeName, String remoteIp, long createTime) {
+    return getReceiverPipeDir(pipeName, remoteIp, createTime)
+        + File.separator
+        + SyncConstant.FILE_DATA_DIR_NAME;
+  }
+
+  public static String getReceiverPipeDir(String pipeName, String remoteIp, long createTime) {
+    return getReceiverDir()
+        + File.separator
+        + getReceiverPipeFolderName(pipeName, remoteIp, createTime);
+  }
+
+  public static String getReceiverPipeFolderName(
+      String pipeName, String remoteIp, long createTime) {
+    return String.format("%s-%d-%s", pipeName, createTime, remoteIp);
+  }
+
+  public static String getReceiverDir() {
+    return IoTDBDescriptor.getInstance().getConfig().getSyncDir()
+        + File.separator
+        + SyncConstant.RECEIVER_DIR;
+  }
+
+  public static String getSysDir() {
+    return IoTDBDescriptor.getInstance().getConfig().getSyncDir()
+        + File.separator
+        + SyncConstant.SYNC_SYS_DIR;
+  }
+
+  /** common */
+  public static boolean createFile(File file) throws IOException {
+    if (!file.getParentFile().exists()) {
+      file.getParentFile().mkdirs();
+    }
+    return file.createNewFile();
+  }
+
+  public static String createMsg(String msg) {
+    return String.format(
+        "[%s] %s", DatetimeUtils.convertLongToDate(DatetimeUtils.currentTime()), msg);
+  }
+}
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
deleted file mode 100644
index 77c12b88fb..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderConfig.java
+++ /dev/null
@@ -1,127 +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.conf;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
-
-public class SyncSenderConfig {
-
-  private String serverIp = "127.0.0.1";
-
-  private int serverPort = 5555;
-
-  private int syncPeriodInSecond = 600;
-
-  private String senderFolderPath;
-
-  private String lastFileInfoPath;
-
-  private String snapshotPath;
-
-  /** The maximum number of retry when syncing a file to receiver fails. */
-  private int maxNumOfSyncFileRetry = 5;
-
-  /** Storage groups which participate in sync process */
-  private List<String> storageGroupList = new ArrayList<>();
-
-  /** Update paths based on data directory */
-  public void update(String dataDirectory) {
-    senderFolderPath =
-        dataDirectory
-            + File.separatorChar
-            + SyncConstant.SYNC_SENDER
-            + File.separatorChar
-            + getSyncReceiverName();
-    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 getServerIp() {
-    return serverIp;
-  }
-
-  public void setServerIp(String serverIp) {
-    this.serverIp = serverIp;
-  }
-
-  public int getServerPort() {
-    return serverPort;
-  }
-
-  public void setServerPort(int serverPort) {
-    this.serverPort = serverPort;
-  }
-
-  public int getSyncPeriodInSecond() {
-    return syncPeriodInSecond;
-  }
-
-  public void setSyncPeriodInSecond(int syncPeriodInSecond) {
-    this.syncPeriodInSecond = syncPeriodInSecond;
-  }
-
-  public String getSenderFolderPath() {
-    return senderFolderPath;
-  }
-
-  public void setSenderFolderPath(String senderFolderPath) {
-    this.senderFolderPath = senderFolderPath;
-  }
-
-  public String getLastFileInfoPath() {
-    return lastFileInfoPath;
-  }
-
-  public void setLastFileInfoPath(String lastFileInfoPath) {
-    this.lastFileInfoPath = lastFileInfoPath;
-  }
-
-  public String getSnapshotPath() {
-    return snapshotPath;
-  }
-
-  public void setSnapshotPath(String snapshotPath) {
-    this.snapshotPath = snapshotPath;
-  }
-
-  public String getSyncReceiverName() {
-    return serverIp + SyncConstant.SYNC_DIR_NAME_SEPARATOR + serverPort;
-  }
-
-  public List<String> getStorageGroupList() {
-    return new ArrayList<>(storageGroupList);
-  }
-
-  public void setStorageGroupList(List<String> storageGroupList) {
-    this.storageGroupList = storageGroupList;
-  }
-
-  public int getMaxNumOfSyncFileRetry() {
-    return maxNumOfSyncFileRetry;
-  }
-
-  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
deleted file mode 100644
index c39af7fe6d..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderDescriptor.java
+++ /dev/null
@@ -1,124 +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.conf;
-
-import org.apache.iotdb.commons.conf.IoTDBConstant;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-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;
-
-public class SyncSenderDescriptor {
-
-  private static final Logger logger = LoggerFactory.getLogger(SyncSenderDescriptor.class);
-  private SyncSenderConfig conf = new SyncSenderConfig();
-
-  private SyncSenderDescriptor() {
-    loadProps();
-  }
-
-  public static SyncSenderDescriptor getInstance() {
-    return SyncSenderDescriptorHolder.INSTANCE;
-  }
-
-  public SyncSenderConfig getConfig() {
-    return conf;
-  }
-
-  public void setConfig(SyncSenderConfig conf) {
-    this.conf = conf;
-  }
-
-  /** load an properties file and set sync config variables */
-  private void loadProps() {
-    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 + SyncConstant.CONFIG_NAME;
-      } else {
-        logger.warn(
-            "Cannot find IOTDB_HOME or IOTDB_CONF environment variable when loading config file {}, use default configuration",
-            SyncConstant.CONFIG_NAME);
-        return;
-      }
-    } else {
-      url += (File.separatorChar + SyncConstant.CONFIG_NAME);
-    }
-
-    try {
-      inputStream = new FileInputStream(new File(url));
-    } catch (FileNotFoundException e) {
-      logger.warn("Fail to find sync config file {}", url, e);
-      return;
-    }
-
-    logger.info("Start to read sync config file {}", url);
-    Properties properties = new Properties();
-    try {
-      properties.load(inputStream);
-
-      conf.setServerIp(properties.getProperty("server_ip", conf.getServerIp()));
-      conf.setServerPort(
-          Integer.parseInt(
-              properties.getProperty("server_port", Integer.toString(conf.getServerPort()))));
-      conf.setSyncPeriodInSecond(
-          Integer.parseInt(
-              properties.getProperty(
-                  "sync_period_in_second", Integer.toString(conf.getSyncPeriodInSecond()))));
-      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.setMaxNumOfSyncFileRetry(
-          Integer.parseInt(
-              properties.getProperty(
-                  "max_number_of_sync_file_retry",
-                  Integer.toString(conf.getMaxNumOfSyncFileRetry()))));
-    } catch (IOException e) {
-      logger.warn("Cannot load sync config file, use default sync configuration.", e);
-    } catch (Exception 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.", e);
-      }
-    }
-  }
-
-  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
deleted file mode 100644
index 1fad3d5c06..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/package-info.java
+++ /dev/null
@@ -1,38 +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 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.
- *
- * <p>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.
- *
- * <p>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.
- *
- * <p>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;
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/pipedata/DeletionPipeData.java b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/DeletionPipeData.java
new file mode 100644
index 0000000000..a8568bd72e
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/DeletionPipeData.java
@@ -0,0 +1,85 @@
+/*
+ * 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.pipedata;
+
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.sync.receiver.load.DeletionLoader;
+import org.apache.iotdb.db.sync.receiver.load.ILoader;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Objects;
+
+public class DeletionPipeData extends PipeData {
+  private static final Logger logger = LoggerFactory.getLogger(DeletionPipeData.class);
+
+  private Deletion deletion;
+
+  public DeletionPipeData(Deletion deletion, long serialNumber) {
+    super(serialNumber);
+    this.deletion = deletion;
+  }
+
+  @Override
+  public PipeDataType getType() {
+    return PipeDataType.DELETION;
+  }
+
+  @Override
+  public long serialize(DataOutputStream stream) throws IOException {
+    return super.serialize(stream) + deletion.serializeWithoutFileOffset(stream);
+  }
+
+  public static DeletionPipeData deserialize(DataInputStream stream)
+      throws IOException, IllegalPathException {
+    long serialNumber = stream.readLong();
+    Deletion deletion = Deletion.deserializeWithoutFileOffset(stream);
+    return new DeletionPipeData(deletion, serialNumber);
+  }
+
+  @Override
+  public ILoader createLoader() {
+    return new DeletionLoader(deletion);
+  }
+
+  @Override
+  public String toString() {
+    return "DeletionData{" + "deletion=" + deletion + ", serialNumber=" + serialNumber + '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    DeletionPipeData that = (DeletionPipeData) o;
+    return Objects.equals(deletion, that.deletion)
+        && Objects.equals(serialNumber, that.serialNumber);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(deletion, serialNumber);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/pipedata/PipeData.java b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/PipeData.java
new file mode 100644
index 0000000000..ecc3376657
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/PipeData.java
@@ -0,0 +1,96 @@
+/*
+ * 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.pipedata;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.sync.receiver.load.ILoader;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+public abstract class PipeData {
+  private static final Logger logger = LoggerFactory.getLogger(PipeData.class);
+
+  protected long serialNumber;
+
+  public PipeData(long serialNumber) {
+    this.serialNumber = serialNumber;
+  }
+
+  public long getSerialNumber() {
+    return serialNumber;
+  }
+
+  public void setSerialNumber(long serialNumber) {
+    this.serialNumber = serialNumber;
+  }
+
+  public abstract PipeDataType getType();
+
+  public long serialize(DataOutputStream stream) throws IOException {
+    long serializeSize = 0;
+    stream.writeByte((byte) getType().ordinal());
+    serializeSize += Byte.BYTES;
+    stream.writeLong(serialNumber);
+    serializeSize += Long.BYTES;
+    return serializeSize;
+  }
+
+  public byte[] serialize() throws IOException {
+    ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
+    serialize(new DataOutputStream(byteStream));
+    return byteStream.toByteArray();
+  }
+
+  public static PipeData deserialize(DataInputStream stream)
+      throws IOException, IllegalPathException {
+    PipeDataType type = PipeDataType.values()[stream.readByte()];
+    switch (type) {
+      case TSFILE:
+        return TsFilePipeData.deserialize(stream);
+      case DELETION:
+        return DeletionPipeData.deserialize(stream);
+      case SCHEMA:
+        return SchemaPipeData.deserialize(stream);
+      default:
+        logger.error("Deserialize PipeData error because Unknown type {}.", type);
+        throw new UnsupportedOperationException(
+            "Deserialize PipeData error because Unknown type " + type);
+    }
+  }
+
+  public static PipeData deserialize(byte[] bytes) throws IllegalPathException, IOException {
+    return deserialize(new DataInputStream(new ByteArrayInputStream(bytes)));
+  }
+
+  public abstract ILoader createLoader();
+
+  public enum PipeDataType {
+    TSFILE,
+    DELETION,
+    SCHEMA
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/pipedata/SchemaPipeData.java b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/SchemaPipeData.java
new file mode 100644
index 0000000000..61c106a886
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/SchemaPipeData.java
@@ -0,0 +1,98 @@
+/*
+ * 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.pipedata;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.sync.receiver.load.ILoader;
+import org.apache.iotdb.db.sync.receiver.load.SchemaLoader;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+public class SchemaPipeData extends PipeData {
+  private static final int SERIALIZE_BUFFER_SIZE = 1024;
+
+  private PhysicalPlan plan;
+
+  public SchemaPipeData(PhysicalPlan plan, long serialNumber) {
+    super(serialNumber);
+    this.plan = plan;
+  }
+
+  @Override
+  public PipeDataType getType() {
+    return PipeDataType.SCHEMA;
+  }
+
+  @Override
+  public long serialize(DataOutputStream stream) throws IOException {
+    long serializeSize = super.serialize(stream);
+    byte[] bytes = getBytes();
+    stream.writeInt(bytes.length);
+    stream.write(bytes);
+    serializeSize += (Integer.BYTES + bytes.length);
+    return serializeSize;
+  }
+
+  private byte[] getBytes() {
+    ByteBuffer buffer = ByteBuffer.allocate(SERIALIZE_BUFFER_SIZE);
+    plan.serialize(buffer);
+    byte[] bytes = new byte[buffer.position()];
+    buffer.flip();
+    buffer.get(bytes);
+    return bytes;
+  }
+
+  public static SchemaPipeData deserialize(DataInputStream stream)
+      throws IOException, IllegalPathException {
+    long serialNumber = stream.readLong();
+    byte[] bytes = new byte[stream.readInt()];
+    stream.read(bytes);
+    PhysicalPlan plan = PhysicalPlan.Factory.create(ByteBuffer.wrap(bytes));
+    return new SchemaPipeData(plan, serialNumber);
+  }
+
+  @Override
+  public ILoader createLoader() {
+    return new SchemaLoader(plan);
+  }
+
+  @Override
+  public String toString() {
+    return "SchemaPipeData{" + "serialNumber=" + serialNumber + ", plan=" + plan + '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    SchemaPipeData that = (SchemaPipeData) o;
+    return Objects.equals(plan, that.plan) && Objects.equals(serialNumber, that.serialNumber);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(plan, serialNumber);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/pipedata/TsFilePipeData.java b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/TsFilePipeData.java
new file mode 100644
index 0000000000..a548851ef6
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/TsFilePipeData.java
@@ -0,0 +1,177 @@
+/*
+ * 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.pipedata;
+
+import org.apache.iotdb.db.engine.modification.ModificationFile;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.db.sync.receiver.load.ILoader;
+import org.apache.iotdb.db.sync.receiver.load.TsFileLoader;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+public class TsFilePipeData extends PipeData {
+  private static final Logger logger = LoggerFactory.getLogger(TsFilePipeData.class);
+
+  private String parentDirPath;
+  private String tsFileName;
+
+  public TsFilePipeData(String tsFilePath, long serialNumber) {
+    super(serialNumber);
+    String sep = File.separator.equals("\\") ? "\\\\" : File.separator;
+    String[] paths = tsFilePath.split(sep);
+    tsFileName = paths[paths.length - 1];
+    if (paths.length > 1) {
+      parentDirPath =
+          tsFilePath.substring(
+              0, tsFilePath.length() - tsFileName.length() - File.separator.length());
+    } else {
+      parentDirPath = "";
+    }
+  }
+
+  public TsFilePipeData(String parentDirPath, String tsFileName, long serialNumber) {
+    super(serialNumber);
+    this.parentDirPath = parentDirPath;
+    this.tsFileName = tsFileName;
+  }
+
+  public void setParentDirPath(String parentDirPath) {
+    this.parentDirPath = parentDirPath;
+  }
+
+  public String getTsFileName() {
+    return tsFileName;
+  }
+
+  public String getTsFilePath() {
+    return parentDirPath + File.separator + tsFileName;
+  }
+
+  @Override
+  public PipeDataType getType() {
+    return PipeDataType.TSFILE;
+  }
+
+  @Override
+  public long serialize(DataOutputStream stream) throws IOException {
+    return super.serialize(stream)
+        + ReadWriteIOUtils.write(parentDirPath, stream)
+        + ReadWriteIOUtils.write(tsFileName, stream);
+  }
+
+  public static TsFilePipeData deserialize(DataInputStream stream) throws IOException {
+    long serialNumber = stream.readLong();
+    String parentDirPath = ReadWriteIOUtils.readString(stream);
+    String tsFileName = ReadWriteIOUtils.readString(stream);
+    return new TsFilePipeData(parentDirPath == null ? "" : parentDirPath, tsFileName, serialNumber);
+  }
+
+  @Override
+  public ILoader createLoader() {
+    return new TsFileLoader(new File(getTsFilePath()));
+  }
+
+  public List<File> getTsFiles(boolean shouldWaitForTsFileClose) throws FileNotFoundException {
+    File tsFile = new File(getTsFilePath()).getAbsoluteFile();
+    File resource = new File(tsFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);
+    File mods = new File(tsFile.getAbsolutePath() + ModificationFile.FILE_SUFFIX);
+
+    List<File> files = new ArrayList<>();
+    if (!tsFile.exists()) {
+      throw new FileNotFoundException(String.format("Can not find %s.", tsFile.getAbsolutePath()));
+    }
+    files.add(tsFile);
+    if (resource.exists()) {
+      files.add(resource);
+    } else {
+      if (shouldWaitForTsFileClose && !waitForTsFileClose()) {
+        throw new FileNotFoundException(
+            String.format(
+                "Can not find %s, maybe the tsfile is not closed yet", resource.getAbsolutePath()));
+      }
+    }
+    if (mods.exists()) {
+      files.add(mods);
+    }
+    return files;
+  }
+
+  private boolean waitForTsFileClose() {
+    for (int i = 0; i < SyncConstant.DEFAULT_WAITING_FOR_TSFILE_RETRY_NUMBER; i++) {
+      if (isTsFileClosed()) {
+        return true;
+      }
+      try {
+        Thread.sleep(SyncConstant.DEFAULT_WAITING_FOR_TSFILE_CLOSE_MILLISECONDS);
+      } catch (InterruptedException e) {
+        logger.warn(String.format("Be Interrupted when waiting for tsfile %s closed", tsFileName));
+      }
+      logger.info(
+          String.format(
+              "Waiting for tsfile %s close, retry %d / %d.",
+              tsFileName, (i + 1), SyncConstant.DEFAULT_WAITING_FOR_TSFILE_RETRY_NUMBER));
+    }
+    return false;
+  }
+
+  private boolean isTsFileClosed() {
+    File tsFile = new File(getTsFilePath()).getAbsoluteFile();
+    File resource = new File(tsFile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX);
+    return resource.exists();
+  }
+
+  @Override
+  public String toString() {
+    return "TsFilePipeData{"
+        + "serialNumber="
+        + serialNumber
+        + ", tsFilePath='"
+        + getTsFilePath()
+        + '\''
+        + '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    TsFilePipeData pipeData = (TsFilePipeData) o;
+    return Objects.equals(parentDirPath, pipeData.parentDirPath)
+        && Objects.equals(tsFileName, pipeData.tsFileName)
+        && Objects.equals(serialNumber, pipeData.serialNumber);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(parentDirPath, tsFileName, serialNumber);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/pipedata/queue/BufferedPipeDataQueue.java b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/queue/BufferedPipeDataQueue.java
new file mode 100644
index 0000000000..c6e7830613
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/queue/BufferedPipeDataQueue.java
@@ -0,0 +1,438 @@
+/*
+ * 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.pipedata.queue;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.db.sync.conf.SyncPathUtil;
+import org.apache.iotdb.db.sync.pipedata.PipeData;
+import org.apache.iotdb.db.sync.pipedata.TsFilePipeData;
+import org.apache.iotdb.db.utils.FileUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.BlockingDeque;
+import java.util.concurrent.LinkedBlockingDeque;
+
+public class BufferedPipeDataQueue implements PipeDataQueue {
+  private static final Logger logger = LoggerFactory.getLogger(BufferedPipeDataQueue.class);
+
+  private final String pipeLogDir;
+
+  /** input */
+  private long lastMaxSerialNumber;
+
+  private BlockingDeque<PipeData> inputDeque;
+
+  private BlockingDeque<Long> pipeLogStartNumber;
+  private DataOutputStream outputStream;
+  private long currentPipeLogSize;
+
+  /** output */
+  private final Object waitLock = new Object();
+
+  private BlockingDeque<PipeData> outputDeque;
+
+  private long pullSerialNumber;
+  private long commitSerialNumber;
+  private DataOutputStream commitLogWriter;
+  private long currentCommitLogSize;
+
+  public BufferedPipeDataQueue(String pipeLogDir) {
+    this.pipeLogDir = pipeLogDir;
+
+    this.lastMaxSerialNumber = 0;
+    this.pipeLogStartNumber = new LinkedBlockingDeque<>();
+
+    this.outputDeque = new LinkedBlockingDeque<>();
+    this.pullSerialNumber = Long.MIN_VALUE;
+    this.commitSerialNumber = Long.MIN_VALUE;
+
+    recover();
+  }
+
+  /** recover */
+  private void recover() {
+    if (!new File(pipeLogDir).exists()) {
+      return;
+    }
+
+    recoverPipeLogStartNumber();
+    recoverLastMaxSerialNumber();
+    recoverCommitSerialNumber();
+    recoverOutputDeque();
+  }
+
+  private void recoverPipeLogStartNumber() {
+    File logDir = new File(pipeLogDir);
+    List<Long> startNumbers = new ArrayList<>();
+
+    for (File file : logDir.listFiles())
+      if (file.getName().endsWith(SyncConstant.PIPE_LOG_NAME_SUFFIX) && file.length() > 0) {
+        startNumbers.add(SyncConstant.getSerialNumberFromPipeLogName(file.getName()));
+      }
+    if (startNumbers.size() != 0) {
+      Collections.sort(startNumbers);
+      for (Long startTime : startNumbers) {
+        pipeLogStartNumber.offer(startTime);
+      }
+    }
+  }
+
+  private void recoverLastMaxSerialNumber() {
+    if (pipeLogStartNumber.isEmpty()) {
+      return;
+    }
+
+    File writingPipeLog =
+        new File(pipeLogDir, SyncConstant.getPipeLogName(pipeLogStartNumber.peekLast()));
+    try {
+      List<PipeData> recoverPipeData = parsePipeLog(writingPipeLog);
+      int recoverPipeDataSize = recoverPipeData.size();
+      lastMaxSerialNumber =
+          recoverPipeDataSize == 0
+              ? pipeLogStartNumber.peekLast() - 1
+              : recoverPipeData.get(recoverPipeDataSize - 1).getSerialNumber();
+    } catch (IOException e) {
+      logger.error(
+          String.format(
+              "Can not recover inputQueue from %s, because %s.", writingPipeLog.getPath(), e));
+    }
+  }
+
+  private void recoverCommitSerialNumber() {
+    File commitLog = new File(pipeLogDir, SyncConstant.COMMIT_LOG_NAME);
+    if (!commitLog.exists()) {
+      if (!pipeLogStartNumber.isEmpty()) {
+        commitSerialNumber = pipeLogStartNumber.peek() - 1;
+      }
+      return;
+    }
+
+    try (RandomAccessFile raf = new RandomAccessFile(commitLog, "r")) {
+      if (raf.length() >= Long.BYTES) {
+        raf.seek(raf.length() - Long.BYTES);
+        commitSerialNumber = raf.readLong();
+      }
+    } catch (IOException e) {
+      logger.error(
+          String.format(
+              "deserialize remove serial number error, remove serial number has been set to %d, because %s",
+              commitSerialNumber, e));
+    }
+  }
+
+  private void recoverOutputDeque() {
+    if (pipeLogStartNumber.isEmpty()) {
+      return;
+    }
+
+    File readingPipeLog =
+        new File(pipeLogDir, SyncConstant.getPipeLogName(pipeLogStartNumber.peek()));
+    try {
+      List<PipeData> recoverPipeData = parsePipeLog(readingPipeLog);
+      int recoverPipeDataSize = recoverPipeData.size();
+      for (int i = recoverPipeDataSize - 1; i >= 0; --i) {
+        PipeData pipeData = recoverPipeData.get(i);
+        if (pipeData.getSerialNumber() <= commitSerialNumber) {
+          break;
+        }
+        outputDeque.addFirst(pipeData);
+      }
+    } catch (IOException e) {
+      logger.error(
+          String.format(
+              "Recover output deque from pipe log %s error, because %s.",
+              readingPipeLog.getPath(), e));
+    }
+  }
+
+  public long getLastMaxSerialNumber() {
+    return lastMaxSerialNumber;
+  }
+
+  public long getCommitSerialNumber() {
+    return commitSerialNumber;
+  }
+
+  /** input */
+  @Override
+  public boolean offer(PipeData pipeData) {
+    if (outputStream == null || currentPipeLogSize > SyncConstant.DEFAULT_PIPE_LOG_SIZE_IN_BYTE) {
+      try {
+        moveToNextPipeLog(pipeData.getSerialNumber());
+      } catch (IOException e) {
+        logger.error(String.format("Move to next pipe log %s error, because %s.", pipeData, e));
+      }
+    }
+    if (!inputDeque.offer(pipeData)) {
+      return false;
+    }
+    synchronized (waitLock) {
+      waitLock.notifyAll();
+    }
+
+    try {
+      writeToDisk(pipeData);
+    } catch (IOException e) {
+      logger.error(String.format("Record pipe data %s error, because %s.", pipeData, e));
+      return false;
+    }
+    return true;
+  }
+
+  private synchronized void moveToNextPipeLog(long startSerialNumber) throws IOException {
+    if (outputStream != null) {
+      outputStream.close();
+    }
+    File newPipeLog = new File(pipeLogDir, SyncConstant.getPipeLogName(startSerialNumber));
+    SyncPathUtil.createFile(newPipeLog);
+
+    outputStream = new DataOutputStream(new FileOutputStream(newPipeLog));
+    pipeLogStartNumber.offer(startSerialNumber);
+    currentPipeLogSize = 0;
+
+    inputDeque = new LinkedBlockingDeque<>();
+    if (commitSerialNumber == Long.MIN_VALUE) {
+      commitSerialNumber = startSerialNumber - 1;
+    }
+  }
+
+  private void writeToDisk(PipeData pipeData) throws IOException {
+    // skip trick
+
+    currentPipeLogSize += pipeData.serialize(outputStream);
+    outputStream.flush();
+  }
+
+  /** output */
+  private synchronized PipeData pullOnePipeData(long lastSerialNumber) throws IOException {
+    long serialNumber = lastSerialNumber + 1;
+    if (!outputDeque.isEmpty()) {
+      return outputDeque.poll();
+    } else if (outputDeque != inputDeque) {
+      if (pipeLogStartNumber.isEmpty() || lastSerialNumber == Long.MIN_VALUE) {
+        return null;
+      }
+
+      if (serialNumber > pipeLogStartNumber.peekLast()) {
+        return null;
+      } else if (serialNumber == pipeLogStartNumber.peekLast() && inputDeque != null) {
+        outputDeque = inputDeque;
+      } else {
+        List<PipeData> parsePipeData =
+            parsePipeLog(new File(pipeLogDir, SyncConstant.getPipeLogName(serialNumber)));
+        int parsePipeDataSize = parsePipeData.size();
+        outputDeque = new LinkedBlockingDeque<>();
+        for (int i = 0; i < parsePipeDataSize; i++) {
+          outputDeque.offer(parsePipeData.get(i));
+        }
+      }
+      return outputDeque.poll();
+    }
+    return null;
+  }
+
+  @Override
+  public List<PipeData> pull(long serialNumber) {
+    List<PipeData> resPipeData = new ArrayList<>();
+
+    pullSerialNumber = commitSerialNumber;
+    while (pullSerialNumber < serialNumber) {
+      try {
+        PipeData pullPipeData = pullOnePipeData(pullSerialNumber);
+        if (pullPipeData != null) {
+          resPipeData.add(pullPipeData);
+          pullSerialNumber = pullPipeData.getSerialNumber();
+        } else {
+          break;
+        }
+      } catch (IOException e) {
+        logger.error(
+            String.format(
+                "Pull pipe data serial number %s error, because %s.", pullSerialNumber + 1, e));
+        break;
+      }
+    }
+
+    for (int i = resPipeData.size() - 1; i >= 0; --i) {
+      outputDeque.addFirst(resPipeData.get(i));
+    }
+    return resPipeData;
+  }
+
+  @Override
+  public PipeData take() throws InterruptedException {
+    PipeData pipeData = null;
+    try {
+      synchronized (waitLock) {
+        while ((pipeData = pullOnePipeData(commitSerialNumber)) == null) {
+          waitLock.wait();
+          waitLock.notifyAll();
+        }
+      }
+    } catch (IOException e) {
+      logger.error(
+          String.format(
+              "Blocking pull pipe data number %s error, because %s", commitSerialNumber + 1, e));
+    }
+    outputDeque.addFirst(pipeData);
+    pullSerialNumber = pipeData.getSerialNumber();
+    return pipeData;
+  }
+
+  @Override
+  public void commit() {
+    if (pullSerialNumber == Long.MIN_VALUE) {
+      return;
+    }
+    commit(pullSerialNumber);
+  }
+
+  public void commit(long serialNumber) {
+    deletePipeData(serialNumber);
+    deletePipeLog();
+    serializeCommitSerialNumber();
+  }
+
+  private void deletePipeData(long serialNumber) {
+    while (commitSerialNumber < serialNumber) {
+      commitSerialNumber += 1;
+      try {
+        PipeData commitData = pullOnePipeData(commitSerialNumber);
+        if (PipeData.PipeDataType.TSFILE.equals(commitData.getType())) {
+          List<File> tsFiles = ((TsFilePipeData) commitData).getTsFiles(false);
+          for (File file : tsFiles) {
+            Files.deleteIfExists(file.toPath());
+          }
+        }
+      } catch (IOException e) {
+        logger.error(
+            String.format(
+                "Commit pipe data serial number %s error, because %s.", commitSerialNumber, e));
+      }
+    }
+  }
+
+  private void deletePipeLog() {
+    if (pipeLogStartNumber.size() >= 2) {
+      long nowPipeLogStartNumber;
+      while (true) {
+        nowPipeLogStartNumber = pipeLogStartNumber.poll();
+        if (!pipeLogStartNumber.isEmpty() && pipeLogStartNumber.peek() <= commitSerialNumber) {
+          try {
+            Files.deleteIfExists(
+                new File(pipeLogDir, SyncConstant.getPipeLogName(nowPipeLogStartNumber)).toPath());
+          } catch (IOException e) {
+            logger.warn(
+                String.format("Delete %s-pipe.log error, because %s.", nowPipeLogStartNumber, e));
+          }
+        } else {
+          break;
+        }
+      }
+      pipeLogStartNumber.addFirst(nowPipeLogStartNumber);
+    }
+  }
+
+  private void serializeCommitSerialNumber() {
+    try {
+      if (commitLogWriter == null) {
+        commitLogWriter =
+            new DataOutputStream(
+                new FileOutputStream(new File(pipeLogDir, SyncConstant.COMMIT_LOG_NAME)));
+        currentCommitLogSize = 0;
+      }
+      commitLogWriter.writeLong(commitSerialNumber);
+      commitLogWriter.flush();
+      currentCommitLogSize += Long.BYTES;
+      if (currentCommitLogSize >= SyncConstant.DEFAULT_PIPE_LOG_SIZE_IN_BYTE) {
+        commitLogWriter.close();
+        commitLogWriter = null;
+      }
+    } catch (IOException e) {
+      logger.error(
+          String.format(
+              "Serialize commit serial number %s error, because %s.", commitSerialNumber, e));
+    }
+  }
+
+  /** common */
+  @Override
+  public synchronized boolean isEmpty() {
+    if (outputDeque == null || pipeLogStartNumber.isEmpty()) {
+      return true;
+    }
+    return pipeLogStartNumber.size() == 1
+        && outputDeque.isEmpty()
+        && (inputDeque == null || inputDeque.isEmpty());
+  }
+
+  @Override
+  public void clear() {
+    try {
+      if (outputStream != null) {
+        outputStream.close();
+        outputStream = null;
+      }
+      if (commitLogWriter != null) {
+        commitLogWriter.close();
+        commitLogWriter = null;
+      }
+
+      inputDeque = null;
+      pipeLogStartNumber = null;
+      outputDeque = null;
+      File logDir = new File(pipeLogDir);
+      if (logDir.exists()) {
+        FileUtils.deleteDirectory(logDir);
+      }
+    } catch (IOException e) {
+      logger.warn(String.format("Clear pipe log dir %s error, because %s.", pipeLogDir, e));
+    }
+  }
+
+  public static List<PipeData> parsePipeLog(File file) throws IOException {
+    List<PipeData> pipeData = new ArrayList<>();
+    try (DataInputStream inputStream = new DataInputStream(new FileInputStream(file))) {
+      while (true) {
+        pipeData.add(PipeData.deserialize(inputStream));
+      }
+    } catch (EOFException e) {
+      logger.info(String.format("Finish parsing pipeLog %s.", file.getPath()));
+    } catch (IllegalPathException e) {
+      logger.error(String.format("Parsing pipeLog %s error, because %s", file.getPath(), e));
+      throw new IOException(e);
+    }
+    return pipeData;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/queue/PipeDataQueue.java
similarity index 66%
copy from server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java
copy to server/src/main/java/org/apache/iotdb/db/sync/pipedata/queue/PipeDataQueue.java
index e82ba05966..576a06a4c4 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManagerMBean.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/queue/PipeDataQueue.java
@@ -7,7 +7,7 @@
  * "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
+ *      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
@@ -16,19 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.receiver;
+package org.apache.iotdb.db.sync.pipedata.queue;
 
-import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.db.sync.pipedata.PipeData;
 
-public interface SyncServerManagerMBean {
+import java.util.List;
 
-  String getRPCServiceStatus();
+public interface PipeDataQueue {
+  boolean offer(PipeData data);
 
-  int getRPCPort();
+  List<PipeData> pull(long serialNumber);
 
-  void startService() throws StartupException;
+  PipeData take() throws InterruptedException;
 
-  void restartService() throws StartupException;
+  void commit();
 
-  void stopService();
+  boolean isEmpty();
+
+  void clear();
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/pipedata/queue/PipeDataQueueFactory.java b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/queue/PipeDataQueueFactory.java
new file mode 100644
index 0000000000..45071dd745
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/pipedata/queue/PipeDataQueueFactory.java
@@ -0,0 +1,57 @@
+/*
+ * 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.pipedata.queue;
+
+import org.apache.iotdb.commons.utils.TestOnly;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class PipeDataQueueFactory {
+
+  // TODO: try use weakReference to avoid memory leak
+  /** pipe log dir name -> BufferedPipeDataQueue in this dir */
+  private static final Map<String, BufferedPipeDataQueue> bufferedPipeDataQueueMap =
+      new ConcurrentHashMap<>();
+  /**
+   * get or create BufferedPipeDataQueue identified by key
+   *
+   * @param pipeLogDir using path of pipe-log dir as key
+   * @return BufferedPipeDataQueue
+   */
+  public static BufferedPipeDataQueue getBufferedPipeDataQueue(String pipeLogDir) {
+    return bufferedPipeDataQueueMap.computeIfAbsent(
+        pipeLogDir, i -> new BufferedPipeDataQueue(pipeLogDir));
+  }
+
+  public static void removeBufferedPipeDataQueue(String pipeLogDir) {
+    BufferedPipeDataQueue queue = bufferedPipeDataQueueMap.remove(pipeLogDir);
+    if (queue != null) {
+      queue.clear();
+    }
+  }
+
+  @TestOnly
+  public static void clear() {
+    for (PipeDataQueue pipeDataQueue : bufferedPipeDataQueueMap.values()) {
+      pipeDataQueue.clear();
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/ReceiverService.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/ReceiverService.java
new file mode 100644
index 0000000000..04558c4a13
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/ReceiverService.java
@@ -0,0 +1,278 @@
+/*
+ * 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 org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.commons.service.IService;
+import org.apache.iotdb.commons.service.ServiceType;
+import org.apache.iotdb.db.exception.sync.PipeServerException;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.physical.sys.ShowPipePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowPipeServerPlan;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.db.query.dataset.ListDataSet;
+import org.apache.iotdb.db.sync.conf.SyncPathUtil;
+import org.apache.iotdb.db.sync.pipedata.queue.PipeDataQueueFactory;
+import org.apache.iotdb.db.sync.receiver.collector.Collector;
+import org.apache.iotdb.db.sync.receiver.manager.PipeInfo;
+import org.apache.iotdb.db.sync.receiver.manager.PipeMessage;
+import org.apache.iotdb.db.sync.receiver.manager.ReceiverManager;
+import org.apache.iotdb.db.sync.sender.pipe.Pipe.PipeStatus;
+import org.apache.iotdb.db.sync.transport.server.TransportServerManager;
+import org.apache.iotdb.service.transport.thrift.ResponseType;
+import org.apache.iotdb.service.transport.thrift.SyncRequest;
+import org.apache.iotdb.service.transport.thrift.SyncResponse;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Binary;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PIPESERVER_STATUS;
+
+public class ReceiverService implements IService {
+  private static final Logger logger = LoggerFactory.getLogger(ReceiverService.class);
+  private static final ReceiverManager receiverManager = ReceiverManager.getInstance();
+  private Collector collector;
+
+  /** start receiver service */
+  public void startPipeServer() throws PipeServerException {
+    try {
+      TransportServerManager.getInstance().startService();
+      receiverManager.startServer();
+      collector.startCollect();
+      // recover started pipe
+      List<PipeInfo> pipeInfos = receiverManager.getAllPipeInfos();
+      for (PipeInfo pipeInfo : pipeInfos) {
+        if (pipeInfo.getStatus().equals(PipeStatus.RUNNING)) {
+          collector.startPipe(
+              pipeInfo.getPipeName(), pipeInfo.getRemoteIp(), pipeInfo.getCreateTime());
+        }
+      }
+    } catch (IOException | StartupException e) {
+      throw new PipeServerException("Failed to start pipe server because " + e.getMessage());
+    }
+  }
+
+  /** stop receiver service */
+  public void stopPipeServer() throws PipeServerException {
+    try {
+      List<PipeInfo> pipeInfos = receiverManager.getAllPipeInfos();
+      for (PipeInfo pipeInfo : pipeInfos) {
+        if (pipeInfo.getStatus().equals(PipeStatus.RUNNING)) {
+          throw new PipeServerException(
+              "Failed to stop pipe server because there is pipe still running.");
+        }
+      }
+      TransportServerManager.getInstance().stopService();
+      receiverManager.stopServer();
+      collector.stopCollect();
+    } catch (IOException e) {
+      throw new PipeServerException("Failed to start pipe server because " + e.getMessage());
+    }
+  }
+
+  /** heartbeat RPC handle */
+  public SyncResponse receiveMsg(SyncRequest request) {
+    SyncResponse response = new SyncResponse(ResponseType.INFO, "");
+    ;
+    try {
+      switch (request.getType()) {
+        case HEARTBEAT:
+          PipeMessage message =
+              receiverManager.getPipeMessage(
+                  request.getPipeName(), request.getRemoteIp(), request.getCreateTime(), true);
+          switch (message.getType()) {
+            case INFO:
+              break;
+            case WARN:
+              response = new SyncResponse(ResponseType.WARN, "");
+              break;
+            case ERROR:
+              response = new SyncResponse(ResponseType.ERROR, "");
+              break;
+            default:
+              throw new UnsupportedOperationException("Wrong message type " + message.getType());
+          }
+          break;
+        case CREATE:
+          createPipe(request.getPipeName(), request.getRemoteIp(), request.getCreateTime());
+          break;
+        case START:
+          startPipe(request.getPipeName(), request.getRemoteIp(), request.getCreateTime());
+          break;
+        case STOP:
+          stopPipe(request.getPipeName(), request.getRemoteIp(), request.getCreateTime());
+          break;
+        case DROP:
+          dropPipe(request.getPipeName(), request.getRemoteIp(), request.getCreateTime());
+          break;
+      }
+    } catch (IOException e) {
+      logger.warn("Cannot handle message because {}", e.getMessage());
+    }
+    return response;
+  }
+
+  /** create and start a new pipe named pipeName */
+  private void createPipe(String pipeName, String remoteIp, long createTime) throws IOException {
+    logger.info("create Pipe name={}, remoteIp={}, createTime={}", pipeName, remoteIp, createTime);
+    createDir(pipeName, remoteIp, createTime);
+    receiverManager.createPipe(pipeName, remoteIp, createTime);
+  }
+
+  /** start an existed pipe named pipeName */
+  private void startPipe(String pipeName, String remoteIp, long createTime) throws IOException {
+    logger.info("start Pipe name={}, remoteIp={}, createTime={}", pipeName, remoteIp, createTime);
+    receiverManager.startPipe(pipeName, remoteIp);
+    collector.startPipe(pipeName, remoteIp, createTime);
+  }
+
+  /** stop an existed pipe named pipeName */
+  private void stopPipe(String pipeName, String remoteIp, long createTime) throws IOException {
+    logger.info("stop Pipe name={}, remoteIp={}, createTime={}", pipeName, remoteIp, createTime);
+    receiverManager.stopPipe(pipeName, remoteIp);
+    collector.stopPipe(pipeName, remoteIp, createTime);
+  }
+
+  /** drop an existed pipe named pipeName */
+  private void dropPipe(String pipeName, String remoteIp, long createTime) throws IOException {
+    logger.info("drop Pipe name={}, remoteIp={}, createTime={}", pipeName, remoteIp, createTime);
+    receiverManager.dropPipe(pipeName, remoteIp);
+    collector.stopPipe(pipeName, remoteIp, createTime);
+    PipeDataQueueFactory.removeBufferedPipeDataQueue(
+        SyncPathUtil.getReceiverPipeLogDir(pipeName, remoteIp, createTime));
+    File dir = new File(SyncPathUtil.getReceiverPipeDir(pipeName, remoteIp, createTime));
+    FileUtils.deleteDirectory(dir);
+  }
+
+  private void createDir(String pipeName, String remoteIp, long createTime) {
+    File f = new File(SyncPathUtil.getReceiverFileDataDir(pipeName, remoteIp, createTime));
+    if (!f.exists()) {
+      f.mkdirs();
+    }
+    f = new File(SyncPathUtil.getReceiverPipeLogDir(pipeName, remoteIp, createTime));
+    if (!f.exists()) {
+      f.mkdirs();
+    }
+  }
+
+  /**
+   * query by sql SHOW PIPESERVER STATUS
+   *
+   * @return QueryDataSet contained one column: enable
+   */
+  public QueryDataSet showPipeServer(ShowPipeServerPlan plan) {
+    ListDataSet dataSet =
+        new ListDataSet(
+            Collections.singletonList(new PartialPath(COLUMN_PIPESERVER_STATUS, false)),
+            Collections.singletonList(TSDataType.BOOLEAN));
+    RowRecord rowRecord = new RowRecord(0);
+    Field status = new Field(TSDataType.BOOLEAN);
+    status.setBoolV(receiverManager.isPipeServerEnable());
+    rowRecord.addField(status);
+    dataSet.putRecord(rowRecord);
+    return dataSet;
+  }
+
+  /** query by sql SHOW PIPE */
+  public QueryDataSet showPipe(ShowPipePlan plan, ListDataSet dataSet) {
+    List<PipeInfo> pipeInfos;
+    if (!StringUtils.isEmpty(plan.getPipeName())) {
+      pipeInfos = receiverManager.getPipeInfos(plan.getPipeName());
+    } else {
+      pipeInfos = receiverManager.getAllPipeInfos();
+    }
+    for (PipeInfo pipeInfo : pipeInfos) {
+      putPipeRecord(dataSet, pipeInfo);
+    }
+    return dataSet;
+  }
+
+  private void putPipeRecord(ListDataSet dataSet, PipeInfo pipeInfo) {
+    RowRecord record = new RowRecord(0);
+    record.addField(
+        Binary.valueOf(DatetimeUtils.convertLongToDate(pipeInfo.getCreateTime())), TSDataType.TEXT);
+    record.addField(Binary.valueOf(pipeInfo.getPipeName()), TSDataType.TEXT);
+    record.addField(Binary.valueOf(IoTDBConstant.SYNC_RECEIVER_ROLE), TSDataType.TEXT);
+    record.addField(Binary.valueOf(pipeInfo.getRemoteIp()), TSDataType.TEXT);
+    record.addField(Binary.valueOf(pipeInfo.getStatus().name()), TSDataType.TEXT);
+    record.addField(
+        Binary.valueOf(
+            receiverManager
+                .getPipeMessage(
+                    pipeInfo.getPipeName(), pipeInfo.getRemoteIp(), pipeInfo.getCreateTime(), false)
+                .getMsg()),
+        TSDataType.TEXT);
+    dataSet.putRecord(record);
+  }
+
+  private ReceiverService() {
+    collector = new Collector();
+  }
+
+  public static ReceiverService getInstance() {
+    return ReceiverServiceHolder.INSTANCE;
+  }
+
+  /** IService * */
+  @Override
+  public void start() throws StartupException {
+    receiverManager.init();
+    if (receiverManager.isPipeServerEnable()) {
+      try {
+        startPipeServer();
+      } catch (PipeServerException e) {
+        throw new StartupException(e.getMessage());
+      }
+    }
+  }
+
+  @Override
+  public void stop() {
+    try {
+      receiverManager.close();
+      collector.stopCollect();
+    } catch (IOException e) {
+      logger.error(e.getMessage());
+    }
+  }
+
+  @Override
+  public ServiceType getID() {
+    return ServiceType.RECEIVER_SERVICE;
+  }
+
+  private static class ReceiverServiceHolder {
+    private static final ReceiverService INSTANCE = new ReceiverService();
+
+    private ReceiverServiceHolder() {}
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/collector/Collector.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/collector/Collector.java
new file mode 100644
index 0000000000..99829641d7
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/collector/Collector.java
@@ -0,0 +1,170 @@
+/*
+ * 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.collector;
+
+import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.commons.concurrent.ThreadName;
+import org.apache.iotdb.db.exception.sync.PipeDataLoadBearableException;
+import org.apache.iotdb.db.exception.sync.PipeDataLoadException;
+import org.apache.iotdb.db.sync.conf.SyncPathUtil;
+import org.apache.iotdb.db.sync.pipedata.PipeData;
+import org.apache.iotdb.db.sync.pipedata.queue.PipeDataQueue;
+import org.apache.iotdb.db.sync.pipedata.queue.PipeDataQueueFactory;
+import org.apache.iotdb.db.sync.receiver.manager.PipeMessage;
+import org.apache.iotdb.db.sync.receiver.manager.ReceiverManager;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/** scan sync receiver folder and load pipeData into IoTDB */
+public class Collector {
+
+  private static final Logger logger = LoggerFactory.getLogger(Collector.class);
+  private static final int WAIT_TIMEOUT = 2000;
+  private ExecutorService executorService;
+  private Map<String, Future> taskFutures;
+
+  public Collector() {
+    taskFutures = new ConcurrentHashMap<>();
+  }
+
+  public void startCollect() {
+    this.executorService =
+        IoTDBThreadPoolFactory.newCachedThreadPool(ThreadName.SYNC_RECEIVER_COLLECTOR.getName());
+  }
+
+  public void stopCollect() {
+    for (Future f : taskFutures.values()) {
+      f.cancel(true);
+    }
+    if (executorService != null) {
+      executorService.shutdownNow();
+      int totalWaitTime = WAIT_TIMEOUT;
+      while (!executorService.isTerminated()) {
+        try {
+          if (!executorService.awaitTermination(WAIT_TIMEOUT, TimeUnit.MILLISECONDS)) {
+            logger.info(
+                "{} thread pool doesn't exit after {}ms.",
+                ThreadName.SYNC_RECEIVER_COLLECTOR.getName(),
+                totalWaitTime);
+          }
+          totalWaitTime += WAIT_TIMEOUT;
+        } catch (InterruptedException e) {
+          logger.error(
+              "Interrupted while waiting {} thread pool to exit. ",
+              ThreadName.SYNC_RECEIVER_COLLECTOR.getName());
+          Thread.currentThread().interrupt();
+        }
+      }
+      executorService = null;
+    }
+  }
+
+  public void startPipe(String pipeName, String remoteIp, long createTime) {
+    String dir = SyncPathUtil.getReceiverPipeFolderName(pipeName, remoteIp, createTime);
+    synchronized (dir.intern()) {
+      if (!taskFutures.containsKey(dir)) {
+        ScanTask task = new ScanTask(pipeName, remoteIp, createTime);
+        taskFutures.put(dir, executorService.submit(task));
+      }
+    }
+  }
+
+  public void stopPipe(String pipeName, String remoteIp, long createTime) {
+    String dir = SyncPathUtil.getReceiverPipeFolderName(pipeName, remoteIp, createTime);
+    logger.info("try stop task key={}", dir);
+    synchronized (dir.intern()) {
+      if (taskFutures.containsKey(dir)) {
+        taskFutures.get(dir).cancel(true);
+        taskFutures.remove(dir);
+        logger.info("stop task success, key={}", dir);
+      }
+    }
+  }
+
+  private class ScanTask implements Runnable {
+    private final String pipeName;
+    private final String remoteIp;
+    private final long createTime;
+
+    private ScanTask(String pipeName, String remoteIp, long createTime) {
+      this.pipeName = pipeName;
+      this.remoteIp = remoteIp;
+      this.createTime = createTime;
+    }
+
+    @Override
+    public void run() {
+      PipeDataQueue pipeDataQueue =
+          PipeDataQueueFactory.getBufferedPipeDataQueue(
+              SyncPathUtil.getReceiverPipeLogDir(pipeName, remoteIp, createTime));
+      while (!Thread.currentThread().isInterrupted()) {
+        PipeData pipeData = null;
+        try {
+          pipeData = pipeDataQueue.take();
+          logger.info(
+              "Start load pipeData with serialize number {} and type {},value={}",
+              pipeData.getSerialNumber(),
+              pipeData.getType(),
+              pipeData);
+          pipeData.createLoader().load();
+          pipeDataQueue.commit();
+        } catch (InterruptedException e) {
+          logger.warn("Be interrupted when waiting for pipe data, because {}", e.getMessage());
+          Thread.currentThread().interrupt();
+          break;
+        } catch (PipeDataLoadBearableException e) {
+          // bearable exception
+          logger.warn(e.getMessage());
+          ReceiverManager.getInstance()
+              .writePipeMessage(
+                  pipeName,
+                  remoteIp,
+                  createTime,
+                  new PipeMessage(PipeMessage.MsgType.WARN, e.getMessage()));
+          pipeDataQueue.commit();
+        } catch (PipeDataLoadException e) {
+          // unbearable exception
+          // TODO: should drop this pipe?
+          String msg;
+          if (pipeData != null) {
+            msg =
+                String.format(
+                    "Cannot load pipeData with serialize number %d and type %s, because %s",
+                    pipeData.getSerialNumber(), pipeData.getType(), e.getMessage());
+          } else {
+            msg = String.format("Cannot load pipeData because %s", e.getMessage());
+          }
+          logger.error(msg);
+          ReceiverManager.getInstance()
+              .writePipeMessage(
+                  pipeName, remoteIp, createTime, new PipeMessage(PipeMessage.MsgType.ERROR, msg));
+          break;
+        }
+      }
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/DeletionLoader.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/DeletionLoader.java
new file mode 100644
index 0000000000..f67488ebe4
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/DeletionLoader.java
@@ -0,0 +1,48 @@
+/*
+ * 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 org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.exception.sync.PipeDataLoadException;
+import org.apache.iotdb.db.exception.sync.PipeDataLoadUnbearableException;
+
+/** This loader is used to load deletion plan. */
+public class DeletionLoader implements ILoader {
+
+  private Deletion deletion;
+
+  public DeletionLoader(Deletion deletion) {
+    this.deletion = deletion;
+  }
+
+  @Override
+  public void load() throws PipeDataLoadException {
+    if (IoTDBDescriptor.getInstance().getConfig().isReadOnly()) {
+      throw new PipeDataLoadUnbearableException("storage engine readonly");
+    }
+    try {
+      StorageEngine.getInstance()
+          .delete(deletion.getPath(), deletion.getStartTime(), deletion.getEndTime(), 0, null);
+    } catch (Exception e) {
+      throw new PipeDataLoadUnbearableException(e.getMessage());
+    }
+  }
+}
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
deleted file mode 100644
index 3d25f6ab64..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/FileLoader.java
+++ /dev/null
@@ -1,203 +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.load;
-
-import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.exception.LoadFileException;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.SyncDeviceOwnerConflictException;
-import org.apache.iotdb.db.exception.metadata.IllegalPathException;
-import org.apache.iotdb.db.sync.conf.SyncConstant;
-import org.apache.iotdb.db.utils.FileLoaderUtils;
-
-import org.apache.commons.io.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-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);
-          Thread.currentThread().interrupt();
-        }
-      };
-
-  @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);
-    FileLoaderUtils.checkTsFileResource(tsFileResource);
-    try {
-      FileLoaderManager.getInstance().checkAndUpdateDeviceOwner(tsFileResource);
-      StorageEngine.getInstance().loadNewTsFileForSync(tsFileResource);
-    } catch (SyncDeviceOwnerConflictException e) {
-      LOGGER.error("Device owner has conflicts, so skip the loading file", e);
-    } catch (LoadFileException | StorageEngineException | IllegalPathException e) {
-      throw new IOException(
-          String.format("Can not load new tsfile %s", newTsFile.getAbsolutePath()), e);
-    }
-    loadLog.finishLoadTsfile(newTsFile);
-  }
-
-  private void loadDeletedFile(File deletedTsFile) throws IOException {
-    if (curType != LoadType.DELETE) {
-      loadLog.startLoadDeletedFiles();
-      curType = LoadType.DELETE;
-    }
-    try {
-      if (!StorageEngine.getInstance().deleteTsfileForSync(deletedTsFile)) {
-        LOGGER.info("The file {} to be deleted doesn't exist.", deletedTsFile.getAbsolutePath());
-      }
-    } catch (StorageEngineException | IllegalPathException e) {
-      throw new IOException(
-          String.format("Can not load deleted tsfile %s", deletedTsFile.getAbsolutePath()), 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
deleted file mode 100644
index 03e0e1160e..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderManager.java
+++ /dev/null
@@ -1,213 +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.load;
-
-import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
-import org.apache.iotdb.commons.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.apache.commons.io.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-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;
-
-/** 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() {
-    String syncSystemDir = IoTDBDescriptor.getInstance().getConfig().getSyncDir();
-    deviceOwnerFile = new File(syncSystemDir, SyncConstant.DEVICE_OWNER_FILE_NAME);
-    deviceOwnerTmpFile = new File(syncSystemDir, SyncConstant.DEVICE_OWNER_TMP_FILE_NAME);
-    try {
-      recoverDeviceOwnerMap();
-    } catch (IOException | ClassNotFoundException e) {
-      LOGGER.error(
-          "Can not recover device owner map from file {}",
-          new File(syncSystemDir, 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.getTsFile().getParentFile().getParentFile().getParentFile().getName();
-    Set<String> deviceSet = tsFileResource.getDevices();
-    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(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 (FileInputStream fis = new FileInputStream(deviceOwnerFile);
-        ObjectInputStream deviceOwnerInput = new ObjectInputStream(fis)) {
-      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 (FileOutputStream fos = new FileOutputStream(deviceOwnerFile, false);
-        ObjectOutputStream deviceOwnerOutput = new ObjectOutputStream(fos)) {
-      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);
-        Thread.currentThread().interrupt();
-      }
-    }
-    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
deleted file mode 100644
index 549ea657d9..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/IFileLoader.java
+++ /dev/null
@@ -1,50 +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.load;
-
-import org.apache.iotdb.db.sync.receiver.load.FileLoader.LoadTask;
-
-import java.io.File;
-import java.io.IOException;
-
-/**
- * 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
deleted file mode 100644
index 9eb0d17660..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/ILoadLogger.java
+++ /dev/null
@@ -1,57 +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.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/LoadType.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/ILoader.java
similarity index 77%
rename from server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
rename to server/src/main/java/org/apache/iotdb/db/sync/receiver/load/ILoader.java
index 24b33dfc95..b7b50772db 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadType.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/ILoader.java
@@ -18,8 +18,12 @@
  */
 package org.apache.iotdb.db.sync.receiver.load;
 
-public enum LoadType {
-  DELETE,
-  ADD,
-  NONE
+import org.apache.iotdb.db.exception.sync.PipeDataLoadException;
+
+/**
+ * This interface is used to load files, including tsFile, syncTask, schema, modsFile and
+ * deletePlan.
+ */
+public interface ILoader {
+  void load() throws PipeDataLoadException;
 }
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
deleted file mode 100644
index 20a5ccaac6..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/LoadLogger.java
+++ /dev/null
@@ -1,72 +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.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/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/SchemaLoader.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/SchemaLoader.java
new file mode 100644
index 0000000000..4d794924a3
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/SchemaLoader.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 org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.sync.PipeDataLoadBearableException;
+import org.apache.iotdb.db.exception.sync.PipeDataLoadException;
+import org.apache.iotdb.db.exception.sync.PipeDataLoadUnbearableException;
+import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * This loader is used to PhysicalPlan. Support four types of physical plans: CREATE_TIMESERIES |
+ * CREATE_ALIGNED_TIMESERIES | DELETE_TIMESERIES | SET_STORAGE_GROUP
+ */
+public class SchemaLoader implements ILoader {
+  private static final Logger logger = LoggerFactory.getLogger(SchemaLoader.class);
+
+  private PhysicalPlan plan;
+
+  public SchemaLoader(PhysicalPlan plan) {
+    this.plan = plan;
+  }
+
+  @Override
+  public void load() throws PipeDataLoadException {
+    try {
+      LocalSchemaProcessor.getInstance().operation(plan);
+    } catch (StorageGroupAlreadySetException e) {
+      throw new PipeDataLoadBearableException(
+          "Sync receiver try to set storage group "
+              + e.getStorageGroupPath()
+              + " that has already been set");
+    } catch (IOException | MetadataException e) {
+      throw new PipeDataLoadUnbearableException(e.getMessage());
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/TsFileLoader.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/TsFileLoader.java
new file mode 100644
index 0000000000..a90a3fe17a
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/TsFileLoader.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.receiver.load;
+
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
+import org.apache.iotdb.db.exception.sync.PipeDataLoadException;
+import org.apache.iotdb.db.exception.sync.PipeDataLoadUnbearableException;
+import org.apache.iotdb.db.tools.TsFileRewriteTool;
+import org.apache.iotdb.db.utils.FileLoaderUtils;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+/** This loader is used to load tsFiles. If .mods file exists, it will be loaded as well. */
+public class TsFileLoader implements ILoader {
+
+  private File tsFile;
+
+  public TsFileLoader(File tsFile) {
+    this.tsFile = tsFile;
+  }
+
+  @Override
+  public void load() throws PipeDataLoadException {
+    try {
+      TsFileResource tsFileResource = new TsFileResource(tsFile);
+      tsFileResource.setStatus(TsFileResourceStatus.CLOSED);
+      FileLoaderUtils.loadOrGenerateResource(tsFileResource);
+      List<TsFileResource> splitResources = new ArrayList();
+      if (tsFileResource.isSpanMultiTimePartitions()) {
+        TsFileRewriteTool.rewriteTsFile(tsFileResource, splitResources);
+        tsFileResource.writeLock();
+        tsFileResource.removeModFile();
+        tsFileResource.writeUnlock();
+      }
+
+      if (splitResources.isEmpty()) {
+        splitResources.add(tsFileResource);
+      }
+
+      for (TsFileResource resource : splitResources) {
+        StorageEngine.getInstance().loadNewTsFile(resource);
+      }
+    } catch (Exception e) {
+      throw new PipeDataLoadUnbearableException(e.getMessage());
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/manager/PipeInfo.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/manager/PipeInfo.java
new file mode 100644
index 0000000000..bd774ccb4e
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/manager/PipeInfo.java
@@ -0,0 +1,85 @@
+/*
+ * 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.manager;
+
+import org.apache.iotdb.db.sync.sender.pipe.Pipe.PipeStatus;
+
+import java.util.Objects;
+
+public class PipeInfo {
+  private String pipeName;
+  private PipeStatus status;
+  private String remoteIp;
+  private long createTime;
+
+  public PipeInfo(String pipeName, String remoteIp, PipeStatus status, long createTime) {
+    this.pipeName = pipeName;
+    this.remoteIp = remoteIp;
+    this.status = status;
+    this.createTime = createTime;
+  }
+
+  public String getPipeName() {
+    return pipeName;
+  }
+
+  public void setPipeName(String pipeName) {
+    this.pipeName = pipeName;
+  }
+
+  public PipeStatus getStatus() {
+    return status;
+  }
+
+  public void setStatus(PipeStatus status) {
+    this.status = status;
+  }
+
+  public long getCreateTime() {
+    return createTime;
+  }
+
+  public void setCreateTime(long createTime) {
+    this.createTime = createTime;
+  }
+
+  public String getRemoteIp() {
+    return remoteIp;
+  }
+
+  public void setRemoteIp(String remoteIp) {
+    this.remoteIp = remoteIp;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    PipeInfo pipeInfo = (PipeInfo) o;
+    return createTime == pipeInfo.createTime
+        && Objects.equals(pipeName, pipeInfo.pipeName)
+        && status == pipeInfo.status
+        && Objects.equals(remoteIp, pipeInfo.remoteIp);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(pipeName, status, remoteIp, createTime);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/manager/PipeMessage.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/manager/PipeMessage.java
new file mode 100644
index 0000000000..8d81daaaf9
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/manager/PipeMessage.java
@@ -0,0 +1,76 @@
+/*
+ * 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.manager;
+
+import java.util.Objects;
+
+public class PipeMessage {
+  private MsgType type;
+  private String msg;
+
+  public PipeMessage(MsgType msgType, String msg) {
+    this.type = msgType;
+    this.msg = msg;
+  }
+
+  public MsgType getType() {
+    return type;
+  }
+
+  public void setType(MsgType type) {
+    this.type = type;
+  }
+
+  public String getMsg() {
+    return msg;
+  }
+
+  public void setMsg(String msg) {
+    this.msg = msg;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    PipeMessage that = (PipeMessage) o;
+    return type == that.type && Objects.equals(msg, that.msg);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(type, msg);
+  }
+
+  public enum MsgType {
+    INFO(1),
+    WARN(2),
+    ERROR(3);
+
+    private int value;
+
+    MsgType(int value) {
+      this.value = value;
+    }
+
+    public int getValue() {
+      return value;
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/manager/ReceiverManager.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/manager/ReceiverManager.java
new file mode 100644
index 0000000000..a73d064f69
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/manager/ReceiverManager.java
@@ -0,0 +1,216 @@
+/*
+ * 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.manager;
+
+import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.db.sync.conf.SyncPathUtil;
+import org.apache.iotdb.db.sync.receiver.recovery.ReceiverLog;
+import org.apache.iotdb.db.sync.receiver.recovery.ReceiverLogAnalyzer;
+import org.apache.iotdb.db.sync.sender.pipe.Pipe.PipeStatus;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ReceiverManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(ReceiverManager.class);
+
+  private boolean pipeServerEnable;
+  // <pipeName, <remoteIp, pipeInfo>>
+  private Map<String, Map<String, PipeInfo>> pipeInfoMap;
+  // <pipeFolderName, pipeMsg>
+  private Map<String, List<PipeMessage>> pipeMessageMap;
+  private ReceiverLog log;
+
+  public void init() throws StartupException {
+    log = new ReceiverLog();
+    ReceiverLogAnalyzer analyzer = new ReceiverLogAnalyzer();
+    analyzer.scan();
+    pipeInfoMap = analyzer.getPipeInfoMap();
+    pipeServerEnable = analyzer.isPipeServerEnable();
+    pipeMessageMap = analyzer.getPipeMessageMap();
+  }
+
+  public void close() throws IOException {
+    log.close();
+  }
+
+  public void startServer() throws IOException {
+    log.startPipeServer();
+    pipeServerEnable = true;
+  }
+
+  public void stopServer() throws IOException {
+    log.stopPipeServer();
+    pipeServerEnable = false;
+  }
+
+  public void createPipe(String pipeName, String remoteIp, long createTime) throws IOException {
+    log.createPipe(pipeName, remoteIp, createTime);
+    if (!pipeInfoMap.containsKey(pipeName)) {
+      pipeInfoMap.put(pipeName, new HashMap<>());
+    }
+    pipeInfoMap
+        .get(pipeName)
+        .put(remoteIp, new PipeInfo(pipeName, remoteIp, PipeStatus.STOP, createTime));
+  }
+
+  public void startPipe(String pipeName, String remoteIp) throws IOException {
+    log.startPipe(pipeName, remoteIp);
+    pipeInfoMap.get(pipeName).get(remoteIp).setStatus(PipeStatus.RUNNING);
+  }
+
+  public void stopPipe(String pipeName, String remoteIp) throws IOException {
+    log.stopPipe(pipeName, remoteIp);
+    pipeInfoMap.get(pipeName).get(remoteIp).setStatus(PipeStatus.STOP);
+  }
+
+  public void dropPipe(String pipeName, String remoteIp) throws IOException {
+    log.dropPipe(pipeName, remoteIp);
+    pipeInfoMap.get(pipeName).get(remoteIp).setStatus(PipeStatus.DROP);
+  }
+
+  public List<PipeInfo> getPipeInfos(String pipeName) {
+    return new ArrayList<>(pipeInfoMap.get(pipeName).values());
+  }
+
+  public List<PipeInfo> getAllPipeInfos() {
+    List<PipeInfo> res = new ArrayList<>();
+    for (String pipeName : pipeInfoMap.keySet()) {
+      res.addAll(pipeInfoMap.get(pipeName).values());
+    }
+    return res;
+  }
+
+  /**
+   * write a single message and serialize to disk
+   *
+   * @param pipeName name of pipe
+   * @param remoteIp remoteIp of pipe
+   * @param createTime createTime of pipe
+   * @param message pipe message
+   */
+  public void writePipeMessage(
+      String pipeName, String remoteIp, long createTime, PipeMessage message) {
+    if (pipeInfoMap.containsKey(pipeName) && pipeInfoMap.get(pipeName).containsKey(remoteIp)) {
+      synchronized (pipeInfoMap.get(pipeName).get(remoteIp)) {
+        String pipeIdentifier =
+            SyncPathUtil.getReceiverPipeFolderName(pipeName, remoteIp, createTime);
+        try {
+          log.writePipeMsg(pipeIdentifier, message);
+        } catch (IOException e) {
+          logger.error(
+              "Can not write pipe message {} from {} to disk because {}",
+              message,
+              pipeIdentifier,
+              e.getMessage());
+        }
+        pipeMessageMap.computeIfAbsent(pipeIdentifier, i -> new ArrayList<>()).add(message);
+      }
+    }
+  }
+
+  /**
+   * read recent messages about one pipe
+   *
+   * @param pipeName name of pipe
+   * @param remoteIp remoteIp of pipe
+   * @param createTime createTime of pipe
+   * @param consume if consume is true, these messages will not be deleted. Otherwise, these
+   *     messages can be read next time.
+   * @return recent messages
+   */
+  public List<PipeMessage> getPipeMessages(
+      String pipeName, String remoteIp, long createTime, boolean consume) {
+    List<PipeMessage> pipeMessageList = new ArrayList<>();
+    if (pipeInfoMap.containsKey(pipeName) && pipeInfoMap.get(pipeName).containsKey(remoteIp)) {
+      synchronized (pipeInfoMap.get(pipeName).get(remoteIp)) {
+        String pipeIdentifier =
+            SyncPathUtil.getReceiverPipeFolderName(pipeName, remoteIp, createTime);
+        if (consume) {
+          try {
+            log.comsumePipeMsg(pipeIdentifier);
+          } catch (IOException e) {
+            logger.error(
+                "Can not read pipe message about {} from disk because {}",
+                pipeIdentifier,
+                e.getMessage());
+          }
+        }
+        if (pipeMessageMap.containsKey(pipeIdentifier)) {
+          pipeMessageList = pipeMessageMap.get(pipeIdentifier);
+          if (consume) {
+            pipeMessageMap.remove(pipeIdentifier);
+          }
+        }
+      }
+    }
+    return pipeMessageList;
+  }
+
+  /**
+   * read the most important message about one pipe. ERROR > WARN > INFO.
+   *
+   * @param pipeName name of pipe
+   * @param remoteIp remoteIp of pipe
+   * @param createTime createTime of pipe
+   * @param consume if consume is true, recent messages will not be deleted. Otherwise, these
+   *     messages can be read next time.
+   * @return the most important message
+   */
+  public PipeMessage getPipeMessage(
+      String pipeName, String remoteIp, long createTime, boolean consume) {
+    List<PipeMessage> pipeMessageList = getPipeMessages(pipeName, remoteIp, createTime, consume);
+    PipeMessage message = new PipeMessage(PipeMessage.MsgType.INFO, "");
+    if (!pipeMessageList.isEmpty()) {
+      for (PipeMessage pipeMessage : pipeMessageList) {
+        if (pipeMessage.getType().getValue() > message.getType().getValue()) {
+          message = pipeMessage;
+        }
+      }
+    }
+    return message;
+  }
+
+  public boolean isPipeServerEnable() {
+    return pipeServerEnable;
+  }
+
+  public void setPipeServerEnable(boolean pipeServerEnable) {
+    this.pipeServerEnable = pipeServerEnable;
+  }
+
+  public static ReceiverManager getInstance() {
+    return ReceiverManagerHolder.INSTANCE;
+  }
+
+  private ReceiverManager() {}
+
+  private static class ReceiverManagerHolder {
+    private static final ReceiverManager INSTANCE = new ReceiverManager();
+
+    private ReceiverManagerHolder() {}
+  }
+}
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
deleted file mode 100644
index 9307bef137..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogger.java
+++ /dev/null
@@ -1,50 +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.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
deleted file mode 100644
index a976cc98eb..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogAnalyzer.java
+++ /dev/null
@@ -1,154 +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.recover;
-
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-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.load.LoadLogger;
-import org.apache.iotdb.db.sync.receiver.load.LoadType;
-import org.apache.iotdb.tsfile.utils.FilePathUtils;
-
-import org.apache.commons.io.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-
-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.");
-        Thread.currentThread().interrupt();
-      }
-    } 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
deleted file mode 100644
index 8630ab59ac..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/SyncReceiverLogger.java
+++ /dev/null
@@ -1,72 +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.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/recovery/ReceiverLog.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recovery/ReceiverLog.java
new file mode 100644
index 0000000000..12c7d0281c
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recovery/ReceiverLog.java
@@ -0,0 +1,127 @@
+/*
+ * 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.recovery;
+
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.db.sync.conf.SyncPathUtil;
+import org.apache.iotdb.db.sync.receiver.manager.PipeMessage;
+import org.apache.iotdb.db.sync.sender.pipe.Pipe.PipeStatus;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+
+public class ReceiverLog {
+  private BufferedWriter pipeServerWriter;
+  private BufferedWriter msgWriter;
+
+  public void init() throws IOException {
+    File logFile = new File(SyncPathUtil.getSysDir(), SyncConstant.RECEIVER_LOG_NAME);
+    File msgFile = new File(SyncPathUtil.getSysDir(), SyncConstant.RECEIVER_MSG_LOG_NAME);
+    if (!logFile.getParentFile().exists()) {
+      logFile.getParentFile().mkdirs();
+    }
+    pipeServerWriter = new BufferedWriter(new FileWriter(logFile, true));
+    msgWriter = new BufferedWriter(new FileWriter(msgFile, true));
+  }
+
+  public void startPipeServer() throws IOException {
+    if (pipeServerWriter == null) {
+      init();
+    }
+    pipeServerWriter.write("on");
+    pipeServerWriter.newLine();
+    pipeServerWriter.flush();
+  }
+
+  public void stopPipeServer() throws IOException {
+    if (pipeServerWriter == null) {
+      init();
+    }
+    pipeServerWriter.write("off");
+    pipeServerWriter.newLine();
+    pipeServerWriter.flush();
+  }
+
+  public void createPipe(String pipeName, String remoteIp, long time) throws IOException {
+    writeLog(pipeName, remoteIp, PipeStatus.RUNNING, time);
+  }
+
+  public void startPipe(String pipeName, String remoteIp) throws IOException {
+    writeLog(pipeName, remoteIp, PipeStatus.RUNNING);
+  }
+
+  public void stopPipe(String pipeName, String remoteIp) throws IOException {
+    writeLog(pipeName, remoteIp, PipeStatus.STOP);
+  }
+
+  public void dropPipe(String pipeName, String remoteIp) throws IOException {
+    writeLog(pipeName, remoteIp, PipeStatus.DROP);
+  }
+
+  public void writePipeMsg(String pipeIdentifier, PipeMessage pipeMessage) throws IOException {
+    if (msgWriter == null) {
+      init();
+    }
+    msgWriter.write(
+        String.format("%s,%s,%s", pipeIdentifier, pipeMessage.getType(), pipeMessage.getMsg()));
+    msgWriter.newLine();
+    msgWriter.flush();
+  }
+
+  public void comsumePipeMsg(String pipeIdentifier) throws IOException {
+    if (msgWriter == null) {
+      init();
+    }
+    msgWriter.write(String.format("%s,read", pipeIdentifier));
+    msgWriter.newLine();
+    msgWriter.flush();
+  }
+
+  private void writeLog(String pipeName, String remoteIp, PipeStatus status, long time)
+      throws IOException {
+    if (pipeServerWriter == null) {
+      init();
+    }
+    pipeServerWriter.write(String.format("%s,%s,%s,%d", pipeName, remoteIp, status, time));
+    pipeServerWriter.newLine();
+    pipeServerWriter.flush();
+  }
+
+  private void writeLog(String pipeName, String remoteIp, PipeStatus status) throws IOException {
+    if (pipeServerWriter == null) {
+      init();
+    }
+    pipeServerWriter.write(String.format("%s,%s,%s", pipeName, remoteIp, status));
+    pipeServerWriter.newLine();
+    pipeServerWriter.flush();
+  }
+
+  public void close() throws IOException {
+    if (pipeServerWriter != null) {
+      pipeServerWriter.close();
+      pipeServerWriter = null;
+    }
+    if (msgWriter != null) {
+      msgWriter.close();
+      msgWriter = null;
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recovery/ReceiverLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recovery/ReceiverLogAnalyzer.java
new file mode 100644
index 0000000000..02dc2229fe
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recovery/ReceiverLogAnalyzer.java
@@ -0,0 +1,157 @@
+/*
+ * 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.recovery;
+
+import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.commons.service.ServiceType;
+import org.apache.iotdb.db.sync.conf.SyncConstant;
+import org.apache.iotdb.db.sync.conf.SyncPathUtil;
+import org.apache.iotdb.db.sync.receiver.manager.PipeInfo;
+import org.apache.iotdb.db.sync.receiver.manager.PipeMessage;
+import org.apache.iotdb.db.sync.sender.pipe.Pipe.PipeStatus;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class ReceiverLogAnalyzer {
+  private static final Logger logger = LoggerFactory.getLogger(ReceiverLogAnalyzer.class);
+  // record recovery result of receiver server status
+  private boolean pipeServerEnable = false;
+  private Map<String, Map<String, PipeInfo>> pipeInfoMap = new ConcurrentHashMap<>();
+  private Map<String, List<PipeMessage>> pipeMessageMap = new ConcurrentHashMap<>();
+
+  public void scan() throws StartupException {
+    logger.info("Start to recover all sync state for sync receiver.");
+    pipeInfoMap = new ConcurrentHashMap<>();
+    pipeMessageMap = new ConcurrentHashMap<>();
+    pipeServerEnable = false;
+    File serviceLogFile = new File(SyncPathUtil.getSysDir(), SyncConstant.RECEIVER_LOG_NAME);
+    try (BufferedReader loadReader = new BufferedReader(new FileReader(serviceLogFile))) {
+      String line;
+      int lineNum = 0;
+      while ((line = loadReader.readLine()) != null) {
+        lineNum++;
+        try {
+          analyzeServiceLog(line);
+        } catch (Exception e) {
+          logger.error(
+              "Receiver service log recovery error: log file parse error at line " + lineNum);
+          logger.error(e.getMessage());
+          throw new StartupException(
+              ServiceType.RECEIVER_SERVICE.getName(),
+              "Receiver service log file recover error at line " + lineNum);
+        }
+      }
+    } catch (IOException e) {
+      logger.info("Receiver service log file not found");
+    }
+    File msgLogFile = new File(SyncPathUtil.getSysDir(), SyncConstant.RECEIVER_MSG_LOG_NAME);
+    try (BufferedReader loadReader = new BufferedReader(new FileReader(msgLogFile))) {
+      String line;
+      int lineNum = 0;
+      while ((line = loadReader.readLine()) != null) {
+        lineNum++;
+        try {
+          analyzeMsgLog(line);
+        } catch (Exception e) {
+          logger.error("Receiver msg log recovery error: log file parse error at line " + lineNum);
+          logger.error(e.getMessage());
... 8427 lines suppressed ...