Merge branch 'master' of https://github.com/apache/iotdb into rel/0.14.0-preview2
diff --git a/RELEASE_NOTES.md b/RELEASE_NOTES.md
index 5f1c1e8..35df72d 100644
--- a/RELEASE_NOTES.md
+++ b/RELEASE_NOTES.md
@@ -29,10 +29,8 @@
 * Support Between expression
 * Support order by timeseries in last query
 * Support hot configuration of data_dirs
-* Support set system mode to readonly, running and error
 * Support schema template
 * Support executeBatchStatement in JDBC
-* Support clear cache
 * Support new UDF 'change_points'
 
 ## Improvements
diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4
index 03b9219..6a649b8 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IdentifierParser.g4
@@ -75,6 +75,7 @@
     | EVERY
     | EXPLAIN
     | FILL
+    | FILE
     | FLUSH
     | FOR
     | FROM
@@ -169,6 +170,7 @@
     | UNSET
     | UPDATE
     | UPSERT
+    | URI
     | USER
     | USING
     | VALUES
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 8855e97..6a141ce 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
@@ -126,7 +126,7 @@
 
 // Create Trigger
 createTrigger
-    : CREATE triggerType? TRIGGER triggerName=identifier triggerEventClause ON prefixPath AS className=STRING_LITERAL triggerAttributeClause?
+    : CREATE triggerType? TRIGGER triggerName=identifier triggerEventClause ON prefixPath AS className=STRING_LITERAL jarLocation? triggerAttributeClause?
     ;
 
 triggerType
@@ -137,6 +137,10 @@
     : (BEFORE | AFTER) (INSERT | DELETE)
     ;
 
+jarLocation
+    : USING ((FILE fileName=STRING_LITERAL) | URI uri)
+    ;
+
 triggerAttributeClause
     : WITH LR_BRACKET triggerAttribute (COMMA triggerAttribute)* RR_BRACKET
     ;
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 4753170..10bb58e 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
@@ -215,6 +215,10 @@
     : F I L L
     ;
 
+FILE
+    : F I L E
+    ;
+
 FLUSH
     : F L U S H
     ;
@@ -603,6 +607,10 @@
     : U P S E R T
     ;
 
+URI
+    : U R I
+    ;
+
 USER
     : U S E R
     ;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/CreateRegionGroupsPlan.java b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/CreateRegionGroupsPlan.java
index ca7e3fd..57bd998 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/CreateRegionGroupsPlan.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/CreateRegionGroupsPlan.java
@@ -60,6 +60,16 @@
         .add(regionReplicaSet);
   }
 
+  public void serializeForProcedure(DataOutputStream stream) throws IOException {
+    this.serializeImpl(stream);
+  }
+
+  public void deserializeForProcedure(ByteBuffer buffer) throws IOException {
+    // to remove the ordinal of ConfigPhysicalPlanType
+    buffer.getInt();
+    this.deserializeImpl(buffer);
+  }
+
   @Override
   protected void serializeImpl(DataOutputStream stream) throws IOException {
     stream.writeInt(ConfigPhysicalPlanType.CreateRegionGroups.ordinal());
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/UDFInfo.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/UDFInfo.java
index e3a7568..3665b7d 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/UDFInfo.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/UDFInfo.java
@@ -20,10 +20,10 @@
 package org.apache.iotdb.confignode.persistence;
 
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.executable.ExecutableResource;
 import org.apache.iotdb.commons.snapshot.SnapshotProcessor;
 import org.apache.iotdb.commons.udf.service.UDFClassLoader;
 import org.apache.iotdb.commons.udf.service.UDFExecutableManager;
-import org.apache.iotdb.commons.udf.service.UDFExecutableResource;
 import org.apache.iotdb.commons.udf.service.UDFRegistrationService;
 import org.apache.iotdb.confignode.conf.ConfigNodeConfig;
 import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
@@ -78,7 +78,7 @@
 
   private void fetchExecutablesAndCheckInstantiation(String className, List<String> uris)
       throws Exception {
-    final UDFExecutableResource resource = udfExecutableManager.request(uris);
+    final ExecutableResource resource = udfExecutableManager.request(uris);
     try (UDFClassLoader temporaryUdfClassLoader = new UDFClassLoader(resource.getResourceDir())) {
       Class.forName(className, true, temporaryUdfClassLoader)
           .getDeclaredConstructor()
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/CreateRegionGroupsProcedure.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/CreateRegionGroupsProcedure.java
index 3287757..db9a308 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/CreateRegionGroupsProcedure.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/CreateRegionGroupsProcedure.java
@@ -20,26 +20,50 @@
 
 import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
 import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
 import org.apache.iotdb.confignode.consensus.request.write.CreateRegionGroupsPlan;
 import org.apache.iotdb.confignode.consensus.request.write.DeleteRegionGroupsPlan;
 import org.apache.iotdb.confignode.procedure.StateMachineProcedure;
 import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
 import org.apache.iotdb.confignode.procedure.exception.ProcedureException;
 import org.apache.iotdb.confignode.procedure.state.CreateRegionGroupsState;
+import org.apache.iotdb.confignode.procedure.store.ProcedureFactory;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
 
 public class CreateRegionGroupsProcedure
     extends StateMachineProcedure<ConfigNodeProcedureEnv, CreateRegionGroupsState> {
 
-  private final CreateRegionGroupsPlan createRegionGroupsPlan;
-  // Map<TConsensusGroupId, Failed RegionReplicas>
-  private Map<TConsensusGroupId, TRegionReplicaSet> failedRegions;
+  private static final Logger LOGGER = LoggerFactory.getLogger(CreateRegionGroupsProcedure.class);
+
+  private CreateRegionGroupsPlan createRegionGroupsPlan = new CreateRegionGroupsPlan();
+
+  /** key: TConsensusGroupId value: Failed RegionReplicas */
+  private Map<TConsensusGroupId, TRegionReplicaSet> failedRegions = new HashMap<>();
+
+  public CreateRegionGroupsProcedure() {
+    super();
+  }
 
   public CreateRegionGroupsProcedure(CreateRegionGroupsPlan createRegionGroupsPlan) {
     this.createRegionGroupsPlan = createRegionGroupsPlan;
   }
 
+  public CreateRegionGroupsProcedure(
+      CreateRegionGroupsPlan createRegionGroupsPlan,
+      Map<TConsensusGroupId, TRegionReplicaSet> failedRegions) {
+    this.createRegionGroupsPlan = createRegionGroupsPlan;
+    this.failedRegions = failedRegions;
+  }
+
   @Override
   protected Flow executeFromState(ConfigNodeProcedureEnv env, CreateRegionGroupsState state) {
     switch (state) {
@@ -130,6 +154,59 @@
 
   @Override
   protected CreateRegionGroupsState getInitialState() {
-    return CreateRegionGroupsState.CREATE_REGION_GROUPS;
+    return CreateRegionGroupsState.CREATE_REGION_GROUPS_PREPARE;
+  }
+
+  @Override
+  public void serialize(DataOutputStream stream) throws IOException {
+    // must serialize CREATE_REGION_GROUPS.ordinal() firstly
+    stream.writeInt(ProcedureFactory.ProcedureType.CREATE_REGION_GROUPS.ordinal());
+    super.serialize(stream);
+    createRegionGroupsPlan.serializeForProcedure(stream);
+    stream.writeInt(failedRegions.size());
+    failedRegions.forEach(
+        (groupId, replica) -> {
+          ThriftCommonsSerDeUtils.serializeTConsensusGroupId(groupId, stream);
+          ThriftCommonsSerDeUtils.serializeTRegionReplicaSet(replica, stream);
+        });
+  }
+
+  @Override
+  public void deserialize(ByteBuffer byteBuffer) {
+    super.deserialize(byteBuffer);
+    try {
+      createRegionGroupsPlan.deserializeForProcedure(byteBuffer);
+      failedRegions.clear();
+      int failedRegionsSize = byteBuffer.getInt();
+      while (failedRegionsSize-- > 0) {
+        TConsensusGroupId groupId =
+            ThriftCommonsSerDeUtils.deserializeTConsensusGroupId(byteBuffer);
+        TRegionReplicaSet replica =
+            ThriftCommonsSerDeUtils.deserializeTRegionReplicaSet(byteBuffer);
+        failedRegions.put(groupId, replica);
+      }
+    } catch (Exception e) {
+      LOGGER.error("Deserialize meets error in CreateRegionGroupsProcedure", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that instanceof CreateRegionGroupsProcedure) {
+      CreateRegionGroupsProcedure thatProc = (CreateRegionGroupsProcedure) that;
+      return thatProc.getProcId() == this.getProcId()
+          && thatProc.getState() == this.getState()
+          && thatProc.createRegionGroupsPlan.equals(this.createRegionGroupsPlan)
+          && thatProc.failedRegions.equals(this.failedRegions);
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = createRegionGroupsPlan.hashCode();
+    result = 31 * result + Objects.hash(failedRegions);
+    return result;
   }
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/RemoveConfigNodeState.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/RemoveConfigNodeState.java
index 9c94ac1..dc7acbd 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/RemoveConfigNodeState.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/state/RemoveConfigNodeState.java
@@ -20,8 +20,7 @@
 package org.apache.iotdb.confignode.procedure.state;
 
 public enum RemoveConfigNodeState {
-  REMOVE_CONSENSUS_GROUP,
   REMOVE_PEER,
-
+  REMOVE_CONSENSUS_GROUP,
   STOP_CONFIG_NODE
 }
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java
index 59f0eed..5711940 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java
@@ -27,15 +27,21 @@
 import org.apache.iotdb.confignode.procedure.impl.RemoveConfigNodeProcedure;
 import org.apache.iotdb.confignode.procedure.impl.RemoveDataNodeProcedure;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
 public class ProcedureFactory implements IProcedureFactory {
 
+  private static final Logger LOGGER = LoggerFactory.getLogger(ProcedureFactory.class);
+
   @Override
   public Procedure create(ByteBuffer buffer) throws IOException {
     int typeNum = buffer.getInt();
     if (typeNum >= ProcedureType.values().length) {
+      LOGGER.error("unrecognized log type " + typeNum);
       throw new IOException("unrecognized log type " + typeNum);
     }
     ProcedureType type = ProcedureType.values()[typeNum];
@@ -56,7 +62,11 @@
       case REGION_MIGRATE_PROCEDURE:
         procedure = new RegionMigrateProcedure();
         break;
+      case CREATE_REGION_GROUPS:
+        procedure = new CreateRegionGroupsProcedure();
+        break;
       default:
+        LOGGER.error("unknown Procedure type: " + typeNum);
         throw new IOException("unknown Procedure type: " + typeNum);
     }
     procedure.deserialize(buffer);
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCService.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCService.java
index 7407749..134a528 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCService.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCService.java
@@ -72,7 +72,7 @@
           new ThriftServiceThread(
               processor,
               getID().getName(),
-              ThreadName.CONFIG_NODE_RPC_CLIENT.getName(),
+              ThreadName.CONFIGNODE_RPC_PROCESSOR.getName(),
               getBindIP(),
               getBindPort(),
               configConf.getRpcMaxConcurrentClientNum(),
@@ -82,7 +82,7 @@
     } catch (RPCServiceException e) {
       throw new IllegalAccessException(e.getMessage());
     }
-    thriftServiceThread.setName(ThreadName.CONFIG_NODE_RPC_SERVER.getName());
+    thriftServiceThread.setName(ThreadName.CONFIGNODE_RPC_SERVICE.getName());
     MetricService.getInstance()
         .getOrCreateAutoGauge(
             Metric.THRIFT_ACTIVE_THREADS.toString(),
@@ -90,7 +90,7 @@
             thriftServiceThread,
             AbstractThriftServiceThread::getActiveThreadCount,
             Tag.NAME.toString(),
-            ThreadName.CONFIG_NODE_RPC_SERVER.getName());
+            ThreadName.CONFIGNODE_RPC_SERVICE.getName());
   }
 
   @Override
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
index 0064e6b..1da0ac6 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
@@ -77,6 +77,7 @@
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteStorageGroupReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteStorageGroupsReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDropFunctionReq;
+import org.apache.iotdb.confignode.rpc.thrift.TDropTriggerReq;
 import org.apache.iotdb.confignode.rpc.thrift.TGetAllTemplatesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TGetPathsSetTemplatesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TGetTemplateResp;
@@ -477,6 +478,12 @@
   }
 
   @Override
+  public TSStatus dropTrigger(TDropTriggerReq req) throws TException {
+    // todo : implementation
+    return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
+  }
+
+  @Override
   public TSStatus merge() throws TException {
     return configManager.merge();
   }
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateRegionGroupsProcedureTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateRegionGroupsProcedureTest.java
new file mode 100644
index 0000000..e050ded
--- /dev/null
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/CreateRegionGroupsProcedureTest.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.confignode.procedure.impl;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.common.rpc.thrift.TEndPoint;
+import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
+import org.apache.iotdb.confignode.consensus.request.write.CreateRegionGroupsPlan;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+
+import org.junit.Test;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.DataRegion;
+import static org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.SchemaRegion;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class CreateRegionGroupsProcedureTest {
+
+  @Test
+  public void serializeDeserializeTest() {
+    TDataNodeLocation dataNodeLocation0 = new TDataNodeLocation();
+    dataNodeLocation0.setDataNodeId(5);
+    dataNodeLocation0.setClientRpcEndPoint(new TEndPoint("0.0.0.0", 6667));
+    dataNodeLocation0.setInternalEndPoint(new TEndPoint("0.0.0.0", 9003));
+    dataNodeLocation0.setMPPDataExchangeEndPoint(new TEndPoint("0.0.0.0", 8777));
+    dataNodeLocation0.setDataRegionConsensusEndPoint(new TEndPoint("0.0.0.0", 40010));
+    dataNodeLocation0.setSchemaRegionConsensusEndPoint(new TEndPoint("0.0.0.0", 50010));
+
+    TDataNodeLocation dataNodeLocation1 = new TDataNodeLocation();
+    dataNodeLocation1.setDataNodeId(6);
+    dataNodeLocation1.setClientRpcEndPoint(new TEndPoint("0.0.0.1", 6667));
+    dataNodeLocation1.setInternalEndPoint(new TEndPoint("0.0.0.1", 9003));
+    dataNodeLocation1.setMPPDataExchangeEndPoint(new TEndPoint("0.0.0.1", 8777));
+    dataNodeLocation1.setDataRegionConsensusEndPoint(new TEndPoint("0.0.0.1", 40010));
+    dataNodeLocation1.setSchemaRegionConsensusEndPoint(new TEndPoint("0.0.0.1", 50010));
+
+    TConsensusGroupId schemaRegionGroupId = new TConsensusGroupId(SchemaRegion, 1);
+    TConsensusGroupId dataRegionGroupId = new TConsensusGroupId(DataRegion, 0);
+
+    TRegionReplicaSet schemaRegionSet =
+        new TRegionReplicaSet(schemaRegionGroupId, Collections.singletonList(dataNodeLocation0));
+    TRegionReplicaSet dataRegionSet =
+        new TRegionReplicaSet(dataRegionGroupId, Collections.singletonList(dataNodeLocation1));
+
+    // to test the equals method of Map<TConsensusGroupId, TRegionReplicaSet>
+    Map<TConsensusGroupId, TRegionReplicaSet> failedRegions0 =
+        new HashMap<TConsensusGroupId, TRegionReplicaSet>() {
+          {
+            put(dataRegionGroupId, dataRegionSet);
+            put(schemaRegionGroupId, schemaRegionSet);
+          }
+        };
+    Map<TConsensusGroupId, TRegionReplicaSet> failedRegions1 =
+        new HashMap<TConsensusGroupId, TRegionReplicaSet>() {
+          {
+            put(schemaRegionGroupId, schemaRegionSet);
+            put(dataRegionGroupId, dataRegionSet);
+          }
+        };
+    assertEquals(failedRegions0, failedRegions1);
+
+    CreateRegionGroupsPlan createRegionGroupsPlan = new CreateRegionGroupsPlan();
+    createRegionGroupsPlan.addRegionGroup("root.sg0", dataRegionSet);
+    createRegionGroupsPlan.addRegionGroup("root.sg1", schemaRegionSet);
+
+    CreateRegionGroupsProcedure procedure0 =
+        new CreateRegionGroupsProcedure(createRegionGroupsPlan, failedRegions0);
+    PublicBAOS byteArrayOutputStream = new PublicBAOS();
+    DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream);
+
+    try {
+      procedure0.serialize(outputStream);
+      CreateRegionGroupsProcedure procedure1 = new CreateRegionGroupsProcedure();
+      ByteBuffer buffer =
+          ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size());
+      buffer.getInt();
+      procedure1.deserialize(buffer);
+      assertEquals(procedure0, procedure1);
+      assertEquals(procedure0.hashCode(), procedure1.hashCode());
+    } catch (IOException e) {
+      fail();
+    }
+  }
+}
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/config/RatisConfig.java b/consensus/src/main/java/org/apache/iotdb/consensus/config/RatisConfig.java
index e976137..877a361 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/config/RatisConfig.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/config/RatisConfig.java
@@ -34,6 +34,7 @@
   private final ThreadPool threadPool;
   private final Log log;
   private final Grpc grpc;
+  private final RatisConsensus ratisConsensus;
 
   private RatisConfig(
       Rpc rpc,
@@ -41,13 +42,15 @@
       Snapshot snapshot,
       ThreadPool threadPool,
       Log log,
-      Grpc grpc) {
+      Grpc grpc,
+      RatisConsensus ratisConsensus) {
     this.rpc = rpc;
     this.leaderElection = leaderElection;
     this.snapshot = snapshot;
     this.threadPool = threadPool;
     this.log = log;
     this.grpc = grpc;
+    this.ratisConsensus = ratisConsensus;
   }
 
   public Rpc getRpc() {
@@ -74,6 +77,10 @@
     return grpc;
   }
 
+  public RatisConsensus getRatisConsensus() {
+    return ratisConsensus;
+  }
+
   public static Builder newBuilder() {
     return new Builder();
   }
@@ -85,6 +92,7 @@
     private ThreadPool threadPool;
     private Log log;
     private Grpc grpc;
+    private RatisConsensus ratisConsensus;
 
     public RatisConfig build() {
       return new RatisConfig(
@@ -93,7 +101,8 @@
           snapshot != null ? snapshot : Snapshot.newBuilder().build(),
           threadPool != null ? threadPool : ThreadPool.newBuilder().build(),
           log != null ? log : Log.newBuilder().build(),
-          grpc != null ? grpc : Grpc.newBuilder().build());
+          grpc != null ? grpc : Grpc.newBuilder().build(),
+          ratisConsensus != null ? ratisConsensus : RatisConsensus.newBuilder().build());
     }
 
     public Builder setRpc(Rpc rpc) {
@@ -125,6 +134,11 @@
       this.grpc = grpc;
       return this;
     }
+
+    public Builder setRatisConsensus(RatisConsensus ratisConsensus) {
+      this.ratisConsensus = ratisConsensus;
+      return this;
+    }
   }
 
   /** server rpc timeout related */
@@ -692,4 +706,45 @@
       }
     }
   }
+
+  public static class RatisConsensus {
+    private final int retryTimesMax;
+    private final long retryWaitMillis;
+
+    private RatisConsensus(int retryTimesMax, long retryWaitMillis) {
+      this.retryTimesMax = retryTimesMax;
+      this.retryWaitMillis = retryWaitMillis;
+    }
+
+    public int getRetryTimesMax() {
+      return retryTimesMax;
+    }
+
+    public long getRetryWaitMillis() {
+      return retryWaitMillis;
+    }
+
+    public static RatisConsensus.Builder newBuilder() {
+      return new Builder();
+    }
+
+    public static class Builder {
+      private int retryTimesMax = 3;
+      private long retryWaitMillis = 500;
+
+      public RatisConsensus build() {
+        return new RatisConsensus(retryTimesMax, retryWaitMillis);
+      }
+
+      public RatisConsensus.Builder setRetryTimesMax(int retryTimesMax) {
+        this.retryTimesMax = retryTimesMax;
+        return this;
+      }
+
+      public RatisConsensus.Builder setRetryWaitMillis(long retryWaitMillis) {
+        this.retryWaitMillis = retryWaitMillis;
+        return this;
+      }
+    }
+  }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java b/consensus/src/main/java/org/apache/iotdb/consensus/exception/NodeReadOnlyException.java
similarity index 65%
copy from node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java
copy to consensus/src/main/java/org/apache/iotdb/consensus/exception/NodeReadOnlyException.java
index ec0c375..cd733bc 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/exception/NodeReadOnlyException.java
@@ -16,24 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.consensus.exception;
 
-package org.apache.iotdb.commons.udf.service;
+import org.apache.ratis.protocol.RaftPeer;
 
-public class UDFExecutableResource {
-
-  private final long requestId;
-  private final String resourceDir;
-
-  public UDFExecutableResource(long requestId, String resourceDir) {
-    this.requestId = requestId;
-    this.resourceDir = resourceDir;
-  }
-
-  public long getRequestId() {
-    return requestId;
-  }
-
-  public String getResourceDir() {
-    return resourceDir;
+public class NodeReadOnlyException extends ConsensusException {
+  public NodeReadOnlyException(RaftPeer peer) {
+    super(
+        String.format(
+            "Current Peer %s in Address %s is in Read Only State",
+            peer.getId(), peer.getAddress()));
   }
 }
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/multileader/service/MultiLeaderRPCService.java b/consensus/src/main/java/org/apache/iotdb/consensus/multileader/service/MultiLeaderRPCService.java
index c94892a..d7b41f7 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/multileader/service/MultiLeaderRPCService.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/multileader/service/MultiLeaderRPCService.java
@@ -74,7 +74,7 @@
           new ThriftServiceThread(
               (TBaseAsyncProcessor) processor,
               getID().getName(),
-              ThreadName.MULTI_LEADER_CONSENSUS_RPC_CLIENT.getName(),
+              ThreadName.MULTI_LEADER_CONSENSUS_RPC_PROCESSOR.getName(),
               getBindIP(),
               getBindPort(),
               config.getRpc().getRpcSelectorThreadNum(),
@@ -89,7 +89,7 @@
     } catch (RPCServiceException e) {
       throw new IllegalAccessException(e.getMessage());
     }
-    thriftServiceThread.setName(ThreadName.MULTI_LEADER_CONSENSUS_RPC_SERVER.getName());
+    thriftServiceThread.setName(ThreadName.MULTI_LEADER_CONSENSUS_RPC_SERVICE.getName());
   }
 
   @Override
diff --git a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java
index 0b57fdc..1abf6a4 100644
--- a/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java
+++ b/consensus/src/main/java/org/apache/iotdb/consensus/ratis/RatisConsensus.java
@@ -26,6 +26,7 @@
 import org.apache.iotdb.commons.client.ClientPoolProperty;
 import org.apache.iotdb.commons.client.IClientManager;
 import org.apache.iotdb.commons.client.IClientPoolFactory;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.consensus.ConsensusGroupId;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.consensus.IConsensus;
@@ -37,8 +38,10 @@
 import org.apache.iotdb.consensus.common.response.ConsensusReadResponse;
 import org.apache.iotdb.consensus.common.response.ConsensusWriteResponse;
 import org.apache.iotdb.consensus.config.ConsensusConfig;
+import org.apache.iotdb.consensus.config.RatisConfig;
 import org.apache.iotdb.consensus.exception.ConsensusException;
 import org.apache.iotdb.consensus.exception.ConsensusGroupNotExistException;
+import org.apache.iotdb.consensus.exception.NodeReadOnlyException;
 import org.apache.iotdb.consensus.exception.PeerAlreadyInConsensusGroupException;
 import org.apache.iotdb.consensus.exception.PeerNotInConsensusGroupException;
 import org.apache.iotdb.consensus.exception.RatisRequestFailedException;
@@ -60,9 +63,11 @@
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.protocol.SnapshotManagementRequest;
 import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.apache.ratis.protocol.exceptions.ResourceUnavailableException;
 import org.apache.ratis.server.DivisionInfo;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.util.function.CheckedSupplier;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -109,6 +114,8 @@
   // TODO make it configurable
   private static final int DEFAULT_WAIT_LEADER_READY_TIMEOUT = (int) TimeUnit.SECONDS.toMillis(20);
 
+  private final RatisConfig config;
+
   public RatisConsensus(ConsensusConfig config, IStateMachine.Registry registry)
       throws IOException {
     myself = Utils.fromTEndPointAndPriorityToRaftPeer(config.getThisNode(), DEFAULT_PRIORITY);
@@ -120,6 +127,7 @@
     GrpcConfigKeys.Server.setPort(properties, config.getThisNode().getPort());
 
     Utils.initRatisConfig(properties, config.getRatisConfig());
+    this.config = config.getRatisConfig();
 
     clientRpc = new GrpcFactory(new Parameters()).newRaftClientRpc(ClientId.randomId(), properties);
 
@@ -146,6 +154,49 @@
     server.close();
   }
 
+  private boolean shouldRetry(RaftClientReply reply) {
+    // currently, we only retry when ResourceUnavailableException is caught
+    return !reply.isSuccess()
+        && (reply.getException() != null
+            && reply.getException() instanceof ResourceUnavailableException);
+  }
+  /** launch a consensus write with retry mechanism */
+  private RaftClientReply writeWithRetry(CheckedSupplier<RaftClientReply, IOException> caller)
+      throws IOException {
+
+    final int maxRetryTimes = config.getRatisConsensus().getRetryTimesMax();
+    final long waitMillis = config.getRatisConsensus().getRetryWaitMillis();
+
+    int retry = 0;
+    RaftClientReply reply = null;
+    while (retry < maxRetryTimes) {
+      retry++;
+
+      reply = caller.get();
+      if (!shouldRetry(reply)) {
+        return reply;
+      }
+      logger.debug("{} sending write request with retry = {} and reply = {}", this, retry, reply);
+
+      try {
+        Thread.sleep(waitMillis);
+      } catch (InterruptedException e) {
+        logger.warn("{} retry write sleep is interrupted: {}", this, e);
+        Thread.currentThread().interrupt();
+      }
+    }
+    return reply;
+  }
+
+  private RaftClientReply writeLocallyWithRetry(RaftClientRequest request) throws IOException {
+    return writeWithRetry(() -> server.submitClientRequest(request));
+  }
+
+  private RaftClientReply writeRemotelyWithRetry(RatisClient client, Message message)
+      throws IOException {
+    return writeWithRetry(() -> client.getRaftClient().io().send(message));
+  }
+
   /**
    * write will first send request to local server use method call if local server is not leader, it
    * will use RaftClient to send RPC to read leader
@@ -161,6 +212,16 @@
       return failedWrite(new ConsensusGroupNotExistException(consensusGroupId));
     }
 
+    // current Peer is group leader and in ReadOnly State
+    if (isLeader(consensusGroupId) && CommonDescriptor.getInstance().getConfig().isReadOnly()) {
+      try {
+        forceStepDownLeader(raftGroup);
+      } catch (IOException e) {
+        logger.warn("leader {} read only, force step down failed due to {}", myself, e);
+      }
+      return failedWrite(new NodeReadOnlyException(myself));
+    }
+
     // serialize request into Message
     Message message = new RequestMessage(IConsensusRequest);
 
@@ -171,7 +232,7 @@
     RaftPeer suggestedLeader = null;
     if (isLeader(consensusGroupId) && waitUntilLeaderReady(raftGroupId)) {
       try {
-        localServerReply = server.submitClientRequest(clientRequest);
+        localServerReply = writeLocallyWithRetry(clientRequest);
         if (localServerReply.isSuccess()) {
           ResponseMessage responseMessage = (ResponseMessage) localServerReply.getMessage();
           TSStatus writeStatus = (TSStatus) responseMessage.getContentHolder();
@@ -191,7 +252,7 @@
     RatisClient client = null;
     try {
       client = getRaftClient(raftGroup);
-      RaftClientReply reply = client.getRaftClient().io().send(message);
+      RaftClientReply reply = writeRemotelyWithRetry(client, message);
       if (!reply.isSuccess()) {
         return failedWrite(new RatisRequestFailedException(reply.getException()));
       }
@@ -456,10 +517,8 @@
       if (!configChangeReply.isSuccess()) {
         return failed(new RatisRequestFailedException(configChangeReply.getException()));
       }
-      // TODO tuning for timeoutMs
-      // when newLeaderPeerId == null, ratis forces current leader to step down and raise new
-      // election
-      reply = client.getRaftClient().admin().transferLeadership(null, 5000);
+
+      reply = forceStepDownLeader(raftGroup);
       if (!reply.isSuccess()) {
         return failed(new RatisRequestFailedException(reply.getException()));
       }
@@ -473,6 +532,22 @@
     return ConsensusGenericResponse.newBuilder().setSuccess(reply.isSuccess()).build();
   }
 
+  // TODO when Ratis implements read leader transfer mechanism, change this implementation
+  private RaftClientReply forceStepDownLeader(RaftGroup group) throws IOException {
+    RatisClient client = null;
+    try {
+      client = getRaftClient(group);
+      // TODO tuning for timeoutMs
+      // when newLeaderPeerId == null, ratis forces current leader to step down and raise new
+      // election
+      return client.getRaftClient().admin().transferLeadership(null, 5000);
+    } finally {
+      if (client != null) {
+        client.returnSelf();
+      }
+    }
+  }
+
   @Override
   public boolean isLeader(ConsensusGroupId groupId) {
     RaftGroupId raftGroupId = Utils.fromConsensusGroupIdToRaftGroupId(groupId);
diff --git a/docs/Download/README.md b/docs/Download/README.md
index c42b2ea..0753d51 100644
--- a/docs/Download/README.md
+++ b/docs/Download/README.md
@@ -39,24 +39,24 @@
             <td rowspan="1"><a href="https://raw.githubusercontent.com/apache/iotdb/master/RELEASE_NOTES.md">release notes</a></td>
       </tr>
       <tr>
-            <td rowspan="3">0.13.1</td>
-            <td><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.1/apache-iotdb-0.13.1-all-bin.zip">All-in-one</a></td>
-            <td><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-all-bin.zip.sha512">SHA512</a></td>
-            <td><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-all-bin.zip.asc">ASC</a></td>
-            <td rowspan="3"><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.1/apache-iotdb-0.13.1-source-release.zip">Sources</a></td>
-            <td rowspan="3"><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-source-release.zip.sha512">SHA512</a></td>
-            <td rowspan="3"><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-source-release.zip.asc">ASC</a></td>
-            <td rowspan="3"><a href="https://raw.githubusercontent.com/apache/iotdb/v0.13.1/RELEASE_NOTES.md">release notes</a></td>
+            <td rowspan="3">0.13.2</td>
+            <td><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.2/apache-iotdb-0.13.2-all-bin.zip">All-in-one</a></td>
+            <td><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-all-bin.zip.sha512">SHA512</a></td>
+            <td><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-all-bin.zip.asc">ASC</a></td>
+            <td rowspan="3"><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.2/apache-iotdb-0.13.2-source-release.zip">Sources</a></td>
+            <td rowspan="3"><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-source-release.zip.sha512">SHA512</a></td>
+            <td rowspan="3"><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-source-release.zip.asc">ASC</a></td>
+            <td rowspan="3"><a href="https://raw.githubusercontent.com/apache/iotdb/v0.13.2/RELEASE_NOTES.md">release notes</a></td>
       </tr>
       <tr>
-            <td><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.1/apache-iotdb-0.13.1-grafana-plugin-bin.zip">Grafana-plugin</a></td>
-            <td><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-grafana-plugin-bin.zip.sha512">SHA512</a></td>
-            <td><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-grafana-plugin-bin.zip.asc">ASC</a></td>
+            <td><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.2/apache-iotdb-0.13.2-grafana-plugin-bin.zip">Grafana-plugin</a></td>
+            <td><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-grafana-plugin-bin.zip.sha512">SHA512</a></td>
+            <td><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-grafana-plugin-bin.zip.asc">ASC</a></td>
       </tr>
       <tr>
-            <td><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.1/apache-iotdb-0.13.1-grafana-connector-bin.zip">Grafana-connector</a></td>
-            <td><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-grafana-connector-bin.zip.sha512">SHA512</a></td>
-            <td><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-grafana-connector-bin.zip.asc">ASC</a></td>
+            <td><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.2/apache-iotdb-0.13.2-grafana-connector-bin.zip">Grafana-connector</a></td>
+            <td><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-grafana-connector-bin.zip.sha512">SHA512</a></td>
+            <td><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-grafana-connector-bin.zip.asc">ASC</a></td>
       </tr>
       <tr>
             <td rowspan="4">0.12.6</td>
diff --git a/docs/UserGuide/API/Status-Codes.md b/docs/UserGuide/API/Status-Codes.md
index f3bdeb6..19cacaa 100644
--- a/docs/UserGuide/API/Status-Codes.md
+++ b/docs/UserGuide/API/Status-Codes.md
@@ -43,7 +43,7 @@
 Here is a list of Status Code and related message:
 
 |Status Code|Status Type|Meanings|
-|:---|:---|:---|
+|:--|:---|:---|
 |200|SUCCESS_STATUS||
 |201|STILL_EXECUTING_STATUS||
 |202|INVALID_HANDLE_STATUS||
@@ -89,6 +89,7 @@
 |504|START_UP_ERROR|Meet error while starting up|
 |505|SHUT_DOWN_ERROR|Meet error while shutdown|
 |506|MULTIPLE_ERROR|Meet error when executing multiple statements|
+|507|SESSION_EXPIRED|Session expired|
 |600|WRONG_LOGIN_PASSWORD_ERROR|Username or password is wrong|
 |601|NOT_LOGIN_ERROR|Has not logged in|
 |602|NO_PERMISSION_ERROR|No permissions for this operation, please add privilege|
diff --git a/docs/UserGuide/Process-Data/Triggers.md b/docs/UserGuide/Process-Data/Triggers.md
index 950aa50..0dc5407 100644
--- a/docs/UserGuide/Process-Data/Triggers.md
+++ b/docs/UserGuide/Process-Data/Triggers.md
@@ -804,7 +804,7 @@
 package org.apache.iotdb.trigger;
 
 import org.apache.iotdb.db.engine.trigger.api.Trigger;
-import org.apache.iotdb.db.engine.trigger.api.TriggerAttributes;
+import org.apache.iotdb.trigger.api.TriggerAttributes;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.engine.trigger.sink.mqtt.MQTTConfiguration;
 import org.apache.iotdb.db.engine.trigger.sink.mqtt.MQTTEvent;
diff --git a/docs/UserGuide/Query-Data/Last-Query.md b/docs/UserGuide/Query-Data/Last-Query.md
index d22fd52..80134b4 100644
--- a/docs/UserGuide/Query-Data/Last-Query.md
+++ b/docs/UserGuide/Query-Data/Last-Query.md
@@ -28,7 +28,7 @@
 The SQL syntax is defined as:
 
 ```sql
-select last <Path> [COMMA <Path>]* from < PrefixPath > [COMMA < PrefixPath >]* <WhereClause>
+select last <Path> [COMMA <Path>]* from < PrefixPath > [COMMA < PrefixPath >]* <WhereClause> [ORDER BY TIMESERIES (DESC | ASC)?]
 ```
 
 which means: Query and return the last data points of timeseries prefixPath.path.
@@ -68,4 +68,20 @@
 +-----------------------------+-----------------------------+---------+--------+
 Total line number = 2
 It costs 0.002s
+```
+
+## Order By timeseries
+
+**Example 3:** get the last points of all sensor in root.ln.wf01.wt01, and order the result by the timeseries column desc
+
+```
+IoTDB> select last * from root.ln.wf01.wt01 order by timeseries desc;
++-----------------------------+-----------------------------+---------+--------+
+|                         Time|                   timeseries|    value|dataType|
++-----------------------------+-----------------------------+---------+--------+
+|2017-11-07T23:59:00.000+08:00|root.ln.wf01.wt01.temperature|21.067368|  DOUBLE|
+|2017-11-07T23:59:00.000+08:00|     root.ln.wf01.wt01.status|    false| BOOLEAN|
++-----------------------------+-----------------------------+---------+--------+
+Total line number = 2
+It costs 0.002s
 ```
\ No newline at end of file
diff --git a/docs/zh/Download/README.md b/docs/zh/Download/README.md
index 0272a1a..98d4875 100644
--- a/docs/zh/Download/README.md
+++ b/docs/zh/Download/README.md
@@ -38,24 +38,24 @@
             <td rowspan="1"><a href="https://raw.githubusercontent.com/apache/iotdb/master/RELEASE_NOTES.md">release notes</a></td>
       </tr>
       <tr>
-            <td rowspan="3">0.13.1</td>
-            <td><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.1/apache-iotdb-0.13.1-all-bin.zip">All-in-one</a></td>
-            <td><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-all-bin.zip.sha512">SHA512</a></td>
-            <td><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-all-bin.zip.asc">ASC</a></td>
-            <td rowspan="3"><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.1/apache-iotdb-0.13.1-source-release.zip">源代码</a></td>
-            <td rowspan="3"><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-source-release.zip.sha512">SHA512</a></td>
-            <td rowspan="3"><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-source-release.zip.asc">ASC</a></td>
-            <td rowspan="3"><a href="https://raw.githubusercontent.com/apache/iotdb/v0.13.1/RELEASE_NOTES.md">release notes</a></td>
+            <td rowspan="3">0.13.2</td>
+            <td><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.2/apache-iotdb-0.13.2-all-bin.zip">All-in-one</a></td>
+            <td><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-all-bin.zip.sha512">SHA512</a></td>
+            <td><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-all-bin.zip.asc">ASC</a></td>
+            <td rowspan="3"><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.2/apache-iotdb-0.13.2-source-release.zip">源代码</a></td>
+            <td rowspan="3"><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-source-release.zip.sha512">SHA512</a></td>
+            <td rowspan="3"><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-source-release.zip.asc">ASC</a></td>
+            <td rowspan="3"><a href="https://raw.githubusercontent.com/apache/iotdb/v0.13.2/RELEASE_NOTES.md">release notes</a></td>
       </tr>
       <tr>
-            <td><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.1/apache-iotdb-0.13.1-grafana-plugin-bin.zip">Grafana 插件</a></td>
-            <td><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-grafana-plugin-bin.zip.sha512">SHA512</a></td>
-            <td><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-grafana-plugin-bin.zip.asc">ASC</a></td>
+            <td><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.2/apache-iotdb-0.13.2-grafana-plugin-bin.zip">Grafana 插件</a></td>
+            <td><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-grafana-plugin-bin.zip.sha512">SHA512</a></td>
+            <td><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-grafana-plugin-bin.zip.asc">ASC</a></td>
       </tr>
       <tr>
-            <td><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.1/apache-iotdb-0.13.1-grafana-connector-bin.zip">Grafana 连接器</a></td>
-            <td><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-grafana-connector-bin.zip.sha512">SHA512</a></td>
-            <td><a href="https://downloads.apache.org/iotdb/0.13.1/apache-iotdb-0.13.1-grafana-connector-bin.zip.asc">ASC</a></td>
+            <td><a href="https://www.apache.org/dyn/closer.cgi/iotdb/0.13.2/apache-iotdb-0.13.2-grafana-connector-bin.zip">Grafana 连接器</a></td>
+            <td><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-grafana-connector-bin.zip.sha512">SHA512</a></td>
+            <td><a href="https://downloads.apache.org/iotdb/0.13.2/apache-iotdb-0.13.2-grafana-connector-bin.zip.asc">ASC</a></td>
       </tr>
       <tr>
             <td rowspan="4">0.12.6</td>
diff --git a/docs/zh/UserGuide/API/Status-Codes.md b/docs/zh/UserGuide/API/Status-Codes.md
index 0f001f3..438adf7 100644
--- a/docs/zh/UserGuide/API/Status-Codes.md
+++ b/docs/zh/UserGuide/API/Status-Codes.md
@@ -43,7 +43,7 @@
 这里是状态码和相对应信息的列表:
 
 |状态码|状态类型|状态信息|
-|:---|:---|:---|
+|:--|:---|:---|
 |200|SUCCESS_STATUS|成功状态|
 |201|STILL_EXECUTING_STATUS|仍在执行状态|
 |202|INVALID_HANDLE_STATUS|无效句柄状态|
@@ -89,6 +89,7 @@
 |504|START_UP_ERROR|启动错误|
 |505|SHUT_DOWN_ERROR|关机错误|
 |506|MULTIPLE_ERROR|多行语句执行错误|
+|507|SESSION_EXPIRED|会话过期|
 |600|WRONG_LOGIN_PASSWORD_ERROR|用户名或密码错误|
 |601|NOT_LOGIN_ERROR|没有登录|
 |602|NO_PERMISSION_ERROR|没有操作权限|
diff --git a/docs/zh/UserGuide/Ecosystem-Integration/Flink TsFile.md b/docs/zh/UserGuide/Ecosystem-Integration/Flink-TsFile.md
similarity index 100%
rename from docs/zh/UserGuide/Ecosystem-Integration/Flink TsFile.md
rename to docs/zh/UserGuide/Ecosystem-Integration/Flink-TsFile.md
diff --git a/docs/zh/UserGuide/Ecosystem-Integration/Grafana Plugin.md b/docs/zh/UserGuide/Ecosystem-Integration/Grafana-Plugin.md
similarity index 100%
rename from docs/zh/UserGuide/Ecosystem-Integration/Grafana Plugin.md
rename to docs/zh/UserGuide/Ecosystem-Integration/Grafana-Plugin.md
diff --git a/docs/zh/UserGuide/Ecosystem-Integration/Spark TsFile.md b/docs/zh/UserGuide/Ecosystem-Integration/Spark-TsFile.md
similarity index 100%
rename from docs/zh/UserGuide/Ecosystem-Integration/Spark TsFile.md
rename to docs/zh/UserGuide/Ecosystem-Integration/Spark-TsFile.md
diff --git a/docs/zh/UserGuide/Process-Data/Triggers.md b/docs/zh/UserGuide/Process-Data/Triggers.md
index 5413f81..1ef4ffd 100644
--- a/docs/zh/UserGuide/Process-Data/Triggers.md
+++ b/docs/zh/UserGuide/Process-Data/Triggers.md
@@ -743,7 +743,7 @@
 package org.apache.iotdb.trigger;
 
 import org.apache.iotdb.db.engine.trigger.api.Trigger;
-import org.apache.iotdb.db.engine.trigger.api.TriggerAttributes;
+import org.apache.iotdb.trigger.api.TriggerAttributes;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.engine.trigger.sink.mqtt.MQTTConfiguration;
 import org.apache.iotdb.db.engine.trigger.sink.mqtt.MQTTEvent;
diff --git a/docs/zh/UserGuide/Query-Data/Last-Query.md b/docs/zh/UserGuide/Query-Data/Last-Query.md
index 6c657b0..2084cc0 100644
--- a/docs/zh/UserGuide/Query-Data/Last-Query.md
+++ b/docs/zh/UserGuide/Query-Data/Last-Query.md
@@ -26,7 +26,7 @@
 SQL 语法:
 
 ```sql
-select last <Path> [COMMA <Path>]* from < PrefixPath > [COMMA < PrefixPath >]* <whereClause>
+select last <Path> [COMMA <Path>]* from < PrefixPath > [COMMA < PrefixPath >]* <whereClause> [ORDER BY TIMESERIES (DESC | ASC)?]
 ```
 
 其含义是:查询时间序列 prefixPath.path 中最近时间戳的数据。
@@ -66,4 +66,20 @@
 +-----------------------------+-----------------------------+---------+--------+
 Total line number = 2
 It costs 0.002s
+```
+
+## 将结果集根据序列名进行排序
+
+**示例 3:** 查询 root.ln.wf01.wt01 下所有序列的最新数据点,并按照序列名降序排列。
+
+```
+IoTDB> select last * from root.ln.wf01.wt01 order by timeseries desc;
++-----------------------------+-----------------------------+---------+--------+
+|                         Time|                   timeseries|    value|dataType|
++-----------------------------+-----------------------------+---------+--------+
+|2017-11-07T23:59:00.000+08:00|root.ln.wf01.wt01.temperature|21.067368|  DOUBLE|
+|2017-11-07T23:59:00.000+08:00|     root.ln.wf01.wt01.status|    false| BOOLEAN|
++-----------------------------+-----------------------------+---------+--------+
+Total line number = 2
+It costs 0.002s
 ```
\ No newline at end of file
diff --git a/example/trigger/src/main/java/org/apache/iotdb/trigger/AlertingExample.java b/example/trigger/src/main/java/org/apache/iotdb/trigger/AlertingExample.java
index 48975a1..c9bb2b6 100644
--- a/example/trigger/src/main/java/org/apache/iotdb/trigger/AlertingExample.java
+++ b/example/trigger/src/main/java/org/apache/iotdb/trigger/AlertingExample.java
@@ -21,10 +21,10 @@
 
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.engine.trigger.api.Trigger;
-import org.apache.iotdb.db.engine.trigger.api.TriggerAttributes;
 import org.apache.iotdb.db.engine.trigger.sink.alertmanager.AlertManagerConfiguration;
 import org.apache.iotdb.db.engine.trigger.sink.alertmanager.AlertManagerEvent;
 import org.apache.iotdb.db.engine.trigger.sink.alertmanager.AlertManagerHandler;
+import org.apache.iotdb.trigger.api.TriggerAttributes;
 
 import java.io.IOException;
 import java.util.HashMap;
diff --git a/example/trigger/src/main/java/org/apache/iotdb/trigger/TriggerExample.java b/example/trigger/src/main/java/org/apache/iotdb/trigger/TriggerExample.java
index 7e1a615..a41346e 100644
--- a/example/trigger/src/main/java/org/apache/iotdb/trigger/TriggerExample.java
+++ b/example/trigger/src/main/java/org/apache/iotdb/trigger/TriggerExample.java
@@ -21,7 +21,6 @@
 
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.engine.trigger.api.Trigger;
-import org.apache.iotdb.db.engine.trigger.api.TriggerAttributes;
 import org.apache.iotdb.db.engine.trigger.sink.local.LocalIoTDBConfiguration;
 import org.apache.iotdb.db.engine.trigger.sink.local.LocalIoTDBEvent;
 import org.apache.iotdb.db.engine.trigger.sink.local.LocalIoTDBHandler;
@@ -30,6 +29,7 @@
 import org.apache.iotdb.db.engine.trigger.sink.mqtt.MQTTHandler;
 import org.apache.iotdb.db.utils.windowing.configuration.SlidingSizeWindowConfiguration;
 import org.apache.iotdb.db.utils.windowing.handler.SlidingSizeWindowEvaluationHandler;
+import org.apache.iotdb.trigger.api.TriggerAttributes;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
 import org.fusesource.mqtt.client.QoS;
diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFilterIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFilterIT.java
new file mode 100644
index 0000000..76662fb
--- /dev/null
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBFilterIT.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.it;
+
+import org.apache.iotdb.it.env.ConfigFactory;
+import org.apache.iotdb.it.env.EnvFactory;
+import org.apache.iotdb.it.framework.IoTDBTestRunner;
+import org.apache.iotdb.itbase.category.ClusterIT;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+@RunWith(IoTDBTestRunner.class)
+@Category({ClusterIT.class})
+public class IoTDBFilterIT {
+  protected static final int ITERATION_TIMES = 10;
+
+  protected static boolean enableSeqSpaceCompaction;
+  protected static boolean enableUnseqSpaceCompaction;
+  protected static boolean enableCrossSpaceCompaction;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    enableSeqSpaceCompaction = ConfigFactory.getConfig().isEnableSeqSpaceCompaction();
+    enableUnseqSpaceCompaction = ConfigFactory.getConfig().isEnableUnseqSpaceCompaction();
+    enableCrossSpaceCompaction = ConfigFactory.getConfig().isEnableCrossSpaceCompaction();
+    ConfigFactory.getConfig().setEnableSeqSpaceCompaction(false);
+    ConfigFactory.getConfig().setEnableUnseqSpaceCompaction(false);
+    ConfigFactory.getConfig().setEnableCrossSpaceCompaction(false);
+    ConfigFactory.getConfig()
+        .setUdfCollectorMemoryBudgetInMB(5)
+        .setUdfTransformerMemoryBudgetInMB(5)
+        .setUdfReaderMemoryBudgetInMB(5);
+    EnvFactory.getEnv().initBeforeClass();
+    createTimeSeries();
+    generateData();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    EnvFactory.getEnv().cleanAfterClass();
+    ConfigFactory.getConfig().setEnableSeqSpaceCompaction(enableSeqSpaceCompaction);
+    ConfigFactory.getConfig().setEnableUnseqSpaceCompaction(enableUnseqSpaceCompaction);
+    ConfigFactory.getConfig().setEnableCrossSpaceCompaction(enableCrossSpaceCompaction);
+  }
+
+  private static void createTimeSeries() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      statement.execute("SET STORAGE GROUP TO root.vehicle");
+      statement.execute(
+          "create TIMESERIES root.vehicle.testNaN.d1 with datatype=DOUBLE,encoding=PLAIN");
+      statement.execute(
+          "create TIMESERIES root.vehicle.testNaN.d2 with datatype=DOUBLE,encoding=PLAIN");
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+  }
+
+  private static void generateData() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      for (int i = 0; i < ITERATION_TIMES; i++) {
+        statement.execute(
+            String.format(
+                "insert into root.vehicle.testNaN(timestamp,d1,d2) values(%d,%d,%d)", i, i, i));
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+  }
+
+  @Test
+  public void testFilterNaN() {
+    String sqlStr = "select d1 from root.vehicle.testNaN where d1/d2 > 0";
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet = statement.executeQuery(sqlStr);
+
+      int count = 0;
+      while (resultSet.next()) {
+        ++count;
+      }
+
+      // 0.0/0.0 is NaN which should not be kept.
+      assertEquals(ITERATION_TIMES - 1, count);
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+  }
+}
diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBNestedQueryIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBNestedQueryIT.java
index 3acd717..9eb5f56 100644
--- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBNestedQueryIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBNestedQueryIT.java
@@ -108,7 +108,7 @@
                 "insert into root.vehicle.d1(timestamp,s1,s2,s3) values(%d,%d,%d,%s)", i, i, i, i));
         statement.execute(
             (String.format(
-                "insert into root.vehicle.d2(timestamp,s1,s2) values(%d,%d,%d)", i, i, i, i)));
+                "insert into root.vehicle.d2(timestamp,s1,s2) values(%d,%d,%d)", i, i, i)));
       }
     } catch (SQLException throwable) {
       fail(throwable.getMessage());
diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/sync/IoTDBPipeIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/sync/IoTDBPipeIT.java
new file mode 100644
index 0000000..96907ef
--- /dev/null
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/sync/IoTDBPipeIT.java
@@ -0,0 +1,125 @@
+/*
+ * 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.it.sync;
+
+import org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant;
+import org.apache.iotdb.it.env.EnvFactory;
+import org.apache.iotdb.it.framework.IoTDBTestRunner;
+import org.apache.iotdb.itbase.category.LocalStandaloneIT;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+import static org.apache.iotdb.db.it.utils.TestUtils.assertResultSetEqual;
+
+// TODO: this test only support for new standalone now
+@Ignore
+@RunWith(IoTDBTestRunner.class)
+@Category({LocalStandaloneIT.class})
+public class IoTDBPipeIT {
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    EnvFactory.getEnv().initBeforeClass();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    EnvFactory.getEnv().cleanAfterClass();
+  }
+
+  @Test
+  public void testOperatePipe() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      String ip = EnvFactory.getEnv().getDataNodeWrapperList().get(0).getIp();
+      int port = EnvFactory.getEnv().getDataNodeWrapperList().get(0).getPort();
+      statement.execute(
+          String.format("CREATE PIPESINK demo AS IoTDB (ip='%s',port='%d');", ip, port));
+      statement.execute("CREATE PIPE p to demo;");
+      String expectedHeader =
+          ColumnHeaderConstant.COLUMN_PIPE_CREATE_TIME
+              + ","
+              + ColumnHeaderConstant.COLUMN_PIPE_NAME
+              + ","
+              + ColumnHeaderConstant.COLUMN_PIPE_ROLE
+              + ","
+              + ColumnHeaderConstant.COLUMN_PIPE_REMOTE
+              + ","
+              + ColumnHeaderConstant.COLUMN_PIPE_STATUS
+              + ","
+              + ColumnHeaderConstant.COLUMN_PIPE_MESSAGE
+              + ",";
+
+      String createTime = getCreateTime("p");
+      try (ResultSet resultSet = statement.executeQuery("SHOW PIPE")) {
+        String[] expectedRetSet =
+            new String[] {String.format("%s,p,sender,demo,STOP,,", createTime)};
+        assertResultSetEqual(resultSet, expectedHeader, expectedRetSet);
+      }
+      statement.execute("START PIPE p;");
+      Thread.sleep(1000); // wait 1000 ms to start thread
+      try (ResultSet resultSet = statement.executeQuery("SHOW PIPE")) {
+        String[] expectedRetSet =
+            new String[] {
+              String.format("%s,p,sender,demo,RUNNING,,", createTime),
+              String.format("%s,p,receiver,0.0.0.0,RUNNING,,", createTime),
+            };
+        assertResultSetEqual(resultSet, expectedHeader, expectedRetSet);
+      }
+      statement.execute("STOP PIPE p;");
+      try (ResultSet resultSet = statement.executeQuery("SHOW PIPE")) {
+        String[] expectedRetSet =
+            new String[] {String.format("%s,p,sender,demo,STOP,,", createTime)};
+        assertResultSetEqual(resultSet, expectedHeader, expectedRetSet);
+      }
+      statement.execute("DROP PIPE p;");
+      try (ResultSet resultSet = statement.executeQuery("SHOW PIPE")) {
+        String[] expectedRetSet =
+            new String[] {String.format("%s,p,sender,demo,DROP,,", createTime)};
+        assertResultSetEqual(resultSet, expectedHeader, expectedRetSet);
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.fail();
+    }
+  }
+
+  private String getCreateTime(String pipeName) throws Exception {
+    String createTime = "";
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      try (ResultSet resultSet = statement.executeQuery("SHOW PIPE " + pipeName)) {
+        Assert.assertTrue(resultSet.next());
+        createTime = resultSet.getString(1);
+      }
+    }
+    Assert.assertNotEquals("", createTime);
+    return createTime;
+  }
+}
diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/sync/IoTDBPipeSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/sync/IoTDBPipeSinkIT.java
index 51dbcf9..573c2da 100644
--- a/integration-test/src/test/java/org/apache/iotdb/db/it/sync/IoTDBPipeSinkIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/db/it/sync/IoTDBPipeSinkIT.java
@@ -83,12 +83,12 @@
       try (ResultSet resultSet = statement.executeQuery("SHOW PIPESINK")) {
         String[] expectedRetSet =
             new String[] {
-              "demo3,IoTDB,ip='127.0.0.1',port=6670,", "demo1,IoTDB,ip='192.168.0.1',port=6677,"
+              "demo3,IoTDB,ip='127.0.0.1',port=6667,", "demo1,IoTDB,ip='192.168.0.1',port=6677,"
             };
         assertResultSetEqual(resultSet, expectedHeader, expectedRetSet);
       }
       try (ResultSet resultSet = statement.executeQuery("SHOW PIPESINK demo3")) {
-        String[] expectedRetSet = new String[] {"demo3,IoTDB,ip='127.0.0.1',port=6670,"};
+        String[] expectedRetSet = new String[] {"demo3,IoTDB,ip='127.0.0.1',port=6667,"};
         assertResultSetEqual(resultSet, expectedHeader, expectedRetSet);
       }
     } catch (Exception e) {
diff --git a/integration/src/main/java/org/apache/iotdb/db/engine/trigger/example/Accumulator.java b/integration/src/main/java/org/apache/iotdb/db/engine/trigger/example/Accumulator.java
index c2d662c..d1243aa 100644
--- a/integration/src/main/java/org/apache/iotdb/db/engine/trigger/example/Accumulator.java
+++ b/integration/src/main/java/org/apache/iotdb/db/engine/trigger/example/Accumulator.java
@@ -21,7 +21,7 @@
 
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.engine.trigger.api.Trigger;
-import org.apache.iotdb.db.engine.trigger.api.TriggerAttributes;
+import org.apache.iotdb.trigger.api.TriggerAttributes;
 import org.apache.iotdb.tsfile.exception.NotImplementedException;
 import org.apache.iotdb.tsfile.utils.Binary;
 
diff --git a/integration/src/main/java/org/apache/iotdb/db/engine/trigger/example/Counter.java b/integration/src/main/java/org/apache/iotdb/db/engine/trigger/example/Counter.java
index 7f52620..5d87e8c 100644
--- a/integration/src/main/java/org/apache/iotdb/db/engine/trigger/example/Counter.java
+++ b/integration/src/main/java/org/apache/iotdb/db/engine/trigger/example/Counter.java
@@ -21,7 +21,7 @@
 
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.engine.trigger.api.Trigger;
-import org.apache.iotdb.db.engine.trigger.api.TriggerAttributes;
+import org.apache.iotdb.trigger.api.TriggerAttributes;
 import org.apache.iotdb.tsfile.utils.Binary;
 
 public class Counter implements Trigger {
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
index a5be8af..588db66 100644
--- 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
@@ -51,6 +51,7 @@
 import java.util.List;
 import java.util.Map;
 
+@Ignore
 @Category({LocalStandaloneTest.class})
 public class IoTDBSyncSenderIT {
   private boolean enableSeqSpaceCompaction;
diff --git a/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionInsertNullT.java b/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionInsertNullT.java
new file mode 100644
index 0000000..62d9ee3
--- /dev/null
+++ b/integration/src/test/java/org/apache/iotdb/session/IoTDBSessionInsertNullT.java
@@ -0,0 +1,298 @@
+/*
+ * 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.session;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.StatementExecutionException;
+import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+@Category({LocalStandaloneTest.class})
+public class IoTDBSessionInsertNullT {
+  private Session session;
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty(IoTDBConstant.IOTDB_CONF, "src/test/resources/");
+    EnvironmentUtils.envSetUp();
+    prepareData();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    session.close();
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void testInsertRecordNull() throws StatementExecutionException, IoTDBConnectionException {
+    String deviceId = "root.sg1.clsu.d1";
+    session.insertRecord(deviceId, 100, Arrays.asList("s1"), Arrays.asList("true"));
+    List<String> t = new ArrayList<>();
+    t.add(null);
+    session.insertRecord(deviceId, 200, Arrays.asList("s1"), t);
+    session.insertRecord(
+        deviceId,
+        300,
+        Arrays.asList("s1", "s2"),
+        Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+        Arrays.asList(true, 30));
+    session.insertRecord(
+        deviceId,
+        400,
+        Arrays.asList("s1", "s2"),
+        Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+        Arrays.asList(true, null));
+    session.insertRecord(
+        deviceId,
+        500,
+        Arrays.asList("s1", "s2"),
+        Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+        Arrays.asList(null, null));
+    long nums = queryCountRecords("select count(s1) from " + deviceId);
+    assertEquals(3, nums);
+  }
+
+  @Test
+  public void testInsertAlignedRecordNull()
+      throws StatementExecutionException, IoTDBConnectionException {
+    String deviceId = "root.sg1.clsu.aligned_d1";
+    session.insertAlignedRecord(deviceId, 100, Arrays.asList("s1"), Arrays.asList("true"));
+    List<String> t = new ArrayList<>();
+    t.add(null);
+    session.insertAlignedRecord(deviceId, 200, Arrays.asList("s1"), t);
+    session.insertAlignedRecord(
+        deviceId,
+        300,
+        Arrays.asList("s1", "s2"),
+        Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+        Arrays.asList(true, 30));
+    session.insertAlignedRecord(
+        deviceId,
+        400,
+        Arrays.asList("s1", "s2"),
+        Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+        Arrays.asList(true, null));
+    session.insertAlignedRecord(
+        deviceId,
+        500,
+        Arrays.asList("s1", "s2"),
+        Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+        Arrays.asList(null, null));
+    long nums = queryCountRecords("select count(s1) from " + deviceId);
+    assertEquals(3, nums);
+  }
+
+  @Test
+  public void testInsertRecordsNull() throws StatementExecutionException, IoTDBConnectionException {
+    String deviceId1 = "root.sg1.clsu.d2";
+    String deviceId2 = "root.sg1.clsu.d3";
+    session.insertRecords(
+        Arrays.asList(deviceId1, deviceId2),
+        Arrays.asList(300L, 300L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32)),
+        Arrays.asList(Arrays.asList(true, 101), Arrays.asList(false, 201)));
+    session.insertRecords(
+        Arrays.asList(deviceId1, deviceId2),
+        Arrays.asList(200L, 200L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(Arrays.asList("false", "101"), Arrays.asList("true", "201")));
+    session.insertRecords(
+        Arrays.asList(deviceId1, deviceId2),
+        Arrays.asList(400L, 400L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(Arrays.asList(null, "102"), Arrays.asList("false", "202")));
+    session.insertRecords(
+        Arrays.asList(deviceId1, deviceId2),
+        Arrays.asList(500L, 500L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32)),
+        Arrays.asList(Arrays.asList(true, null), Arrays.asList(null, null)));
+    long nums = queryCountRecords("select count(s1) from " + deviceId1);
+    assertEquals(3, nums);
+    nums = queryCountRecords("select count(s2) from " + deviceId2);
+    assertEquals(3, nums);
+  }
+
+  @Test
+  public void testInsertAlignedRecordsNull()
+      throws StatementExecutionException, IoTDBConnectionException {
+    String deviceId1 = "root.sg1.clsu.aligned_d2";
+    String deviceId2 = "root.sg1.clsu.aligned_d3";
+    session.insertAlignedRecords(
+        Arrays.asList(deviceId1, deviceId2),
+        Arrays.asList(300L, 300L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32)),
+        Arrays.asList(Arrays.asList(true, 101), Arrays.asList(false, 201)));
+    session.insertAlignedRecords(
+        Arrays.asList(deviceId1, deviceId2),
+        Arrays.asList(200L, 200L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(Arrays.asList("false", "101"), Arrays.asList("true", "201")));
+    session.insertAlignedRecords(
+        Arrays.asList(deviceId1, deviceId2),
+        Arrays.asList(400L, 400L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(Arrays.asList(null, "102"), Arrays.asList("false", "202")));
+    session.insertAlignedRecords(
+        Arrays.asList(deviceId1, deviceId2),
+        Arrays.asList(500L, 500L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32)),
+        Arrays.asList(Arrays.asList(true, null), Arrays.asList(null, null)));
+    long nums = queryCountRecords("select count(s1) from " + deviceId1);
+    assertEquals(3, nums);
+    nums = queryCountRecords("select count(s2) from " + deviceId2);
+    assertEquals(3, nums);
+  }
+
+  @Test
+  public void testInsertRecordsOfOneDeviceNull()
+      throws StatementExecutionException, IoTDBConnectionException {
+    String deviceId1 = "root.sg1.clsu.InsertRecordsOfOneDevice";
+    session.insertRecordsOfOneDevice(
+        deviceId1,
+        Arrays.asList(300L, 301L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32)),
+        Arrays.asList(Arrays.asList(true, 101), Arrays.asList(false, 201)));
+    session.insertStringRecordsOfOneDevice(
+        deviceId1,
+        Arrays.asList(200L, 201L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(Arrays.asList("false", "101"), Arrays.asList("true", "201")));
+    session.insertStringRecordsOfOneDevice(
+        deviceId1,
+        Arrays.asList(400L, 401L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(Arrays.asList(null, "102"), Arrays.asList("false", "202")));
+    session.insertRecordsOfOneDevice(
+        deviceId1,
+        Arrays.asList(500L, 501L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32)),
+        Arrays.asList(Arrays.asList(true, null), Arrays.asList(null, null)));
+    long nums = queryCountRecords("select count(s1) from " + deviceId1);
+    assertEquals(6, nums);
+  }
+
+  @Test
+  public void testInsertAlignedRecordsOfOneDeviceNull()
+      throws StatementExecutionException, IoTDBConnectionException {
+    String deviceId1 = "root.sg1.clsu.InsertAlignedRecordsOfOneDevice";
+    session.insertAlignedRecordsOfOneDevice(
+        deviceId1,
+        Arrays.asList(300L, 301L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32)),
+        Arrays.asList(Arrays.asList(true, 101), Arrays.asList(false, 201)));
+    session.insertAlignedStringRecordsOfOneDevice(
+        deviceId1,
+        Arrays.asList(200L, 201L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(Arrays.asList("false", "101"), Arrays.asList("true", "201")));
+    session.insertAlignedStringRecordsOfOneDevice(
+        deviceId1,
+        Arrays.asList(400L, 401L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(Arrays.asList(null, "102"), Arrays.asList("false", "202")));
+    session.insertAlignedRecordsOfOneDevice(
+        deviceId1,
+        Arrays.asList(500L, 501L),
+        Arrays.asList(Arrays.asList("s1", "s2"), Arrays.asList("s1", "s2")),
+        Arrays.asList(
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32),
+            Arrays.asList(TSDataType.BOOLEAN, TSDataType.INT32)),
+        Arrays.asList(Arrays.asList(true, null), Arrays.asList(null, null)));
+    long nums = queryCountRecords("select count(s1) from " + deviceId1);
+    assertEquals(6, nums);
+  }
+
+  private void prepareData() throws IoTDBConnectionException, StatementExecutionException {
+    session = new Session("127.0.0.1", 6667, "root", "root");
+    session.open();
+    session.setStorageGroup("root.sg1");
+    session.createTimeseries(
+        "root.sg1.clsu.d1.s1", TSDataType.BOOLEAN, TSEncoding.PLAIN, CompressionType.SNAPPY);
+    session.createTimeseries(
+        "root.sg1.clsu.d1.s2", TSDataType.INT32, TSEncoding.PLAIN, CompressionType.SNAPPY);
+    session.createTimeseries(
+        "root.sg1.clsu.d1.s3", TSDataType.INT64, TSEncoding.PLAIN, CompressionType.SNAPPY);
+    session.createTimeseries(
+        "root.sg1.clsu.d1.s4", TSDataType.FLOAT, TSEncoding.PLAIN, CompressionType.SNAPPY);
+    session.createTimeseries(
+        "root.sg1.clsu.d1.s5", TSDataType.DOUBLE, TSEncoding.PLAIN, CompressionType.SNAPPY);
+    session.createTimeseries(
+        "root.sg1.clsu.d1.s6", TSDataType.TEXT, TSEncoding.PLAIN, CompressionType.SNAPPY);
+    session.createTimeseries(
+        "root.sg1.clsu.d2.s1", TSDataType.BOOLEAN, TSEncoding.PLAIN, CompressionType.SNAPPY);
+  }
+
+  public long queryCountRecords(String sql)
+      throws StatementExecutionException, IoTDBConnectionException {
+    SessionDataSet dataSetWrapper = session.executeQueryStatement(sql, 1000);
+    long count = 0;
+    while (dataSetWrapper.hasNext()) {
+      RowRecord record = dataSetWrapper.next();
+      Field field = record.getFields().get(0);
+      switch (field.getDataType()) {
+        case INT32:
+          count = field.getIntV();
+          break;
+        case INT64:
+          count = field.getLongV();
+          break;
+      }
+    }
+    return count;
+  }
+}
diff --git a/node-commons/pom.xml b/node-commons/pom.xml
index 2ce550d..3b2e804 100644
--- a/node-commons/pom.xml
+++ b/node-commons/pom.xml
@@ -57,6 +57,11 @@
             <version>${project.version}</version>
         </dependency>
         <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>trigger-api</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
             <groupId>commons-io</groupId>
             <artifactId>commons-io</artifactId>
         </dependency>
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java b/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java
index 0361f45..a95da8a 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/client/ClientPoolFactory.java
@@ -67,7 +67,7 @@
                   .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled())
                   .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager())
                   .build(),
-              ThreadName.DATA_NODE_CLIENT_POOL_THREAD_NAME.getName()),
+              ThreadName.ASYNC_DATANODE_CLIENT_POOL.getName()),
           new ClientPoolProperty.Builder<AsyncDataNodeInternalServiceClient>().build().getConfig());
     }
   }
@@ -85,7 +85,7 @@
                   .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled())
                   .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager())
                   .build(),
-              ThreadName.CONFIG_NODE_HEARTBEAT_CLIENT_POOL_THREAD_NAME.getName()),
+              ThreadName.ASYNC_CONFIGNODE_HEARTBEAT_CLIENT_POOL.getName()),
           new ClientPoolProperty.Builder<AsyncConfigNodeHeartbeatServiceClient>()
               .build()
               .getConfig());
@@ -105,7 +105,7 @@
                   .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnabled())
                   .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager())
                   .build(),
-              ThreadName.DATA_NODE_HEARTBEAT_CLIENT_POOL_THREAD_NAME.getName()),
+              ThreadName.ASYNC_DATANODE_HEARTBEAT_CLIENT_POOL.getName()),
           new ClientPoolProperty.Builder<AsyncDataNodeHeartbeatServiceClient>()
               .build()
               .getConfig());
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/client/sync/SyncThriftClientWithErrorHandler.java b/node-commons/src/main/java/org/apache/iotdb/commons/client/sync/SyncThriftClientWithErrorHandler.java
index 0601ebe..e2b4762 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/client/sync/SyncThriftClientWithErrorHandler.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/client/sync/SyncThriftClientWithErrorHandler.java
@@ -62,7 +62,7 @@
       if (cur instanceof TException) {
         int level = 0;
         while (cur != null) {
-          LOGGER.error(
+          LOGGER.debug(
               "level-{} Exception class {}, message {}",
               level,
               cur.getClass().getName(),
@@ -77,13 +77,13 @@
       if (rootCause != null) {
         // if the exception is SocketException and its error message is Broken pipe, it means that
         // the remote node may restart and all the connection we cached before should be cleared.
-        LOGGER.error(
+        LOGGER.debug(
             "root cause message {}, LocalizedMessage {}, ",
             rootCause.getMessage(),
             rootCause.getLocalizedMessage(),
             rootCause);
         if (isConnectionBroken(rootCause)) {
-          LOGGER.error(
+          LOGGER.debug(
               "Broken pipe error happened in calling method {}, we need to clear all previous cached connection, err: {}",
               method.getName(),
               t);
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 971f04a..79ea759 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
@@ -19,21 +19,18 @@
 package org.apache.iotdb.commons.concurrent;
 
 public enum ThreadName {
-  METRICS_SERVICE("Metrics-ServerServiceImpl"),
-  RPC_SERVICE("RPC"),
-  RPC_CLIENT("RPC-Client"),
-  INFLUXDB_SERVICE("Influxdb-Service"),
-  INFLUXDB_CLIENT("Influxdb-Client"),
-  MERGE_SERVICE("Merge"),
-  CLOSE_MERGE_SERVICE("Close-Merge"),
-  CLOSE_MERGE_DAEMON("Close-Merge-Daemon"),
-  CLOSE_DAEMON("Close-Daemon"),
-  MERGE_DAEMON("Merge-Daemon"),
-  MEMORY_MONITOR("MemMonitor"),
-  MEMORY_STATISTICS("MemStatistic"),
-  FLUSH_PARTIAL_POLICY("FlushPartialPolicy"),
-  FORCE_FLUSH_ALL_POLICY("ForceFlushAllPolicy"),
-  STAT_MONITOR("StatMonitor"),
+  CLIENT_RPC_SERVICE("ClientRPC-Service"),
+  CLIENT_RPC_PROCESSOR("ClientRPC-Processor"),
+  CONFIGNODE_RPC_SERVICE("ConfigNodeRPC-Service"),
+  CONFIGNODE_RPC_PROCESSOR("ConfigNodeRPC-Processor"),
+  MULTI_LEADER_CONSENSUS_RPC_SERVICE("MultiLeaderConsensusRPC-Service"),
+  MULTI_LEADER_CONSENSUS_RPC_PROCESSOR("MultiLeaderConsensusRPC-Processor"),
+  MPP_DATA_EXCHANGE_RPC_SERVICE("MPPDataExchangeRPC-Service"),
+  MPP_DATA_EXCHANGE_RPC_PROCESSOR("MPPDataExchangeRPC-Processor"),
+  DATANODE_INTERNAL_RPC_SERVICE("DataNodeInternalRPC-Service"),
+  DATANODE_INTERNAL_RPC_PROCESSOR("DataNodeInternalRPC-Processor"),
+  INFLUXDB_RPC_SERVICE("InfluxdbRPC-Service"),
+  INFLUXDB_RPC_PROCESSOR("InfluxdbRPC-Processor"),
   DATA_REGION_RECOVER_SERVICE("Data-Region-Recover"),
   FLUSH_SERVICE("Flush"),
   FLUSH_SUB_TASK_SERVICE("Flush-SubTask"),
@@ -44,10 +41,8 @@
   WAL_SYNC("WAL-Sync"),
   WAL_DELETE("WAL-Delete"),
   WAL_RECOVER("WAL-Recover"),
-  INDEX_SERVICE("Index"),
   SYNC_CLIENT("Sync-Client"),
   SYNC_SERVER("Sync"),
-  TIME_COST_STATISTIC("TIME_COST_STATISTIC"),
   QUERY_SERVICE("Query"),
   SUB_RAW_QUERY_SERVICE("Sub_RawQuery"),
   INSERTION_SERVICE("MultithreadingInsertionPool"),
@@ -60,35 +55,12 @@
   SYNC_SENDER_HEARTBEAT("Sync-Heartbeat"),
   SYNC_RECEIVER_COLLECTOR("Sync-Collector"),
   CONTINUOUS_QUERY_SERVICE("ContinuousQueryTaskPoolManager"),
-  CLUSTER_INFO_SERVICE("ClusterInfoClient"),
-  CLUSTER_RPC_SERVICE("ClusterRPC"),
-  CLUSTER_RPC_CLIENT("Cluster-RPC-Client"),
-  CLUSTER_META_RPC_SERVICE("ClusterMetaRPC"),
-  CLUSTER_META_RPC_CLIENT("ClusterMetaRPC-Client"),
-  CLUSTER_META_HEARTBEAT_RPC_SERVICE("ClusterMetaHeartbeatRPC"),
-  CLUSTER_META_HEARTBEAT_RPC_CLIENT("ClusterMetaHeartbeatRPC-Client"),
-  CLUSTER_DATA_RPC_SERVICE("ClusterDataRPC"),
-  CLUSTER_DATA_RPC_CLIENT("ClusterDataRPC-Client"),
-  CLUSTER_DATA_HEARTBEAT_RPC_SERVICE("ClusterDataHeartbeatRPC"),
-  CLUSTER_DATA_HEARTBEAT_RPC_CLIENT("ClusterDataHeartbeatRPC-Client"),
-  CLUSTER_MONITOR("ClusterMonitor"),
-  CONFIG_NODE_RPC_SERVER("ConfigNodeRpcServer"),
-  CONFIG_NODE_RPC_CLIENT("ConfigNodeRPC-Client"),
-  MULTI_LEADER_CONSENSUS_RPC_CLIENT("MultiLeaderConsensusRPC-Client"),
-  MULTI_LEADER_CONSENSUS_RPC_SERVER("MultiLeaderConsensusRPC-Server"),
-  DATA_NODE_MANAGEMENT_RPC_SERVER("DataNodeManagementRPC"),
-  DATA_NODE_MANAGEMENT_RPC_CLIENT("DataNodeManagementRPC-Client"),
-  MPP_DATA_EXCHANGE_RPC_SERVER("MPPDataExchangeRPC"),
-  MPP_DATA_EXCHANGE_RPC_CLIENT("MPPDataExchangeRPC-Client"),
-  INTERNAL_SERVICE_RPC_SERVER("InternalServiceRPC"),
-  INTERNAL_SERVICE_RPC_CLIENT("InternalServiceRPC-Client"),
   EXT_PIPE_PLUGIN_WORKER("ExtPipePlugin-Worker"),
-  DATA_NODE_CLIENT_POOL_THREAD_NAME("AsyncDataNodeInternalServiceClientPool"),
-  CONFIG_NODE_HEARTBEAT_CLIENT_POOL_THREAD_NAME("AsyncConfigNodeHeartbeatServiceClientPool"),
-  DATA_NODE_HEARTBEAT_CLIENT_POOL_THREAD_NAME("AsyncDataNodeHeartbeatServiceClientPool"),
-  CONFIG_NODE_CLIENT_POOL_THREAD_NAME("AsyncConfigNodeIServiceClientPool"),
-  DATA_NODE_MPP_DATA_EXCHANGE_CLIENT_POOL_THREAD_NAME(
-      "AsyncDataNodeMPPDataExchangeServiceClientPool");
+  ASYNC_DATANODE_CLIENT_POOL("AsyncDataNodeInternalServiceClientPool"),
+  ASYNC_CONFIGNODE_HEARTBEAT_CLIENT_POOL("AsyncConfigNodeHeartbeatServiceClientPool"),
+  ASYNC_DATANODE_HEARTBEAT_CLIENT_POOL("AsyncDataNodeHeartbeatServiceClientPool"),
+  ASYNC_CONFIGNODE_CLIENT_POOL("AsyncConfigNodeIServiceClientPool"),
+  ASYNC_DATANODE_MPP_DATA_EXCHANGE_CLIENT_POOL("AsyncDataNodeMPPDataExchangeServiceClientPool");
 
   private final String name;
 
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/executable/ExecutableManager.java b/node-commons/src/main/java/org/apache/iotdb/commons/executable/ExecutableManager.java
new file mode 100644
index 0000000..7022416
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/executable/ExecutableManager.java
@@ -0,0 +1,115 @@
+/*
+ * 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.commons.executable;
+
+import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
+
+import org.apache.commons.io.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ExecutableManager {
+
+  protected final String temporaryLibRoot;
+  protected final String libRoot;
+
+  protected final AtomicLong requestCounter;
+
+  public ExecutableManager(String temporaryLibRoot, String libRoot) {
+    this.temporaryLibRoot = temporaryLibRoot;
+    this.libRoot = libRoot;
+
+    requestCounter = new AtomicLong(0);
+  }
+
+  public ExecutableResource request(List<String> uris) throws URISyntaxException, IOException {
+    final long requestId = generateNextRequestId();
+    downloadExecutables(uris, requestId);
+    return new ExecutableResource(requestId, getDirStringByRequestId(requestId));
+  }
+
+  public void moveToExtLibDir(ExecutableResource resource, String name) throws IOException {
+    FileUtils.moveDirectory(getDirByRequestId(resource.getRequestId()), getDirByName(name));
+  }
+
+  public void removeFromTemporaryLibRoot(ExecutableResource resource) {
+    removeFromTemporaryLibRoot(resource.getRequestId());
+  }
+
+  public void removeFromExtLibDir(String functionName) {
+    FileUtils.deleteQuietly(getDirByName(functionName));
+  }
+
+  private synchronized long generateNextRequestId() throws IOException {
+    long requestId = requestCounter.getAndIncrement();
+    while (FileUtils.isDirectory(getDirByRequestId(requestId))) {
+      requestId = requestCounter.getAndIncrement();
+    }
+    FileUtils.forceMkdir(getDirByRequestId(requestId));
+    return requestId;
+  }
+
+  private void downloadExecutables(List<String> uris, long requestId)
+      throws IOException, URISyntaxException {
+    // TODO: para download
+    try {
+      for (String uriString : uris) {
+        final URL url = new URI(uriString).toURL();
+        final String fileName = uriString.substring(uriString.lastIndexOf("/") + 1);
+        final String destination =
+            temporaryLibRoot + File.separator + requestId + File.separator + fileName;
+        FileUtils.copyURLToFile(url, FSFactoryProducer.getFSFactory().getFile(destination));
+      }
+    } catch (Exception e) {
+      removeFromTemporaryLibRoot(requestId);
+      throw e;
+    }
+  }
+
+  private void removeFromTemporaryLibRoot(long requestId) {
+    FileUtils.deleteQuietly(getDirByRequestId(requestId));
+  }
+
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+  // dir string and dir file generation
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+
+  public File getDirByRequestId(long requestId) {
+    return FSFactoryProducer.getFSFactory().getFile(getDirStringByRequestId(requestId));
+  }
+
+  public String getDirStringByRequestId(long requestId) {
+    return temporaryLibRoot + File.separator + requestId + File.separator;
+  }
+
+  public File getDirByName(String name) {
+    return FSFactoryProducer.getFSFactory().getFile(getDirStringByName(name));
+  }
+
+  public String getDirStringByName(String name) {
+    return libRoot + File.separator + name + File.separator;
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java b/node-commons/src/main/java/org/apache/iotdb/commons/executable/ExecutableResource.java
similarity index 87%
rename from node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/executable/ExecutableResource.java
index ec0c375..d763445 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/executable/ExecutableResource.java
@@ -17,14 +17,14 @@
  * under the License.
  */
 
-package org.apache.iotdb.commons.udf.service;
+package org.apache.iotdb.commons.executable;
 
-public class UDFExecutableResource {
+public class ExecutableResource {
 
   private final long requestId;
   private final String resourceDir;
 
-  public UDFExecutableResource(long requestId, String resourceDir) {
+  public ExecutableResource(long requestId, String resourceDir) {
     this.requestId = requestId;
     this.resourceDir = resourceDir;
   }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/file/SystemFileFactory.java b/node-commons/src/main/java/org/apache/iotdb/commons/file/SystemFileFactory.java
index bbed056..54f2e13 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/file/SystemFileFactory.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/file/SystemFileFactory.java
@@ -22,7 +22,10 @@
 import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.tsfile.fileSystem.FSType;
 
+import org.apache.commons.io.FileUtils;
+
 import java.io.File;
+import java.io.IOException;
 import java.net.URI;
 
 public enum SystemFileFactory {
@@ -67,4 +70,12 @@
       return new File(uri);
     }
   }
+
+  public void makeDirIfNecessary(String dir) throws IOException {
+    File file = getFile(dir);
+    if (file.exists() && file.isDirectory()) {
+      return;
+    }
+    FileUtils.forceMkdir(file);
+  }
 }
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 ec38df7..f9bdc39 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
@@ -44,8 +44,10 @@
   UDF_CLASSLOADER_MANAGER_SERVICE("UDF Classloader Manager Service", ""),
   UDF_REGISTRATION_SERVICE("UDF Registration Service", ""),
   UDF_EXECUTABLE_MANAGER_SERVICE("UDF Executable Manager Service", ""),
-  TEMPORARY_QUERY_DATA_FILE_SERVICE("Temporary Query Data File Service", ""),
+  TRIGGER_CLASSLOADER_MANAGER_SERVICE("Trigger ClassLoader Manager Service", ""),
   TRIGGER_REGISTRATION_SERVICE("Trigger Registration Service", ""),
+  TEMPORARY_QUERY_DATA_FILE_SERVICE("Temporary Query Data File Service", ""),
+  TRIGGER_REGISTRATION_SERVICE_OLD("Old Standalone Trigger Registration Service", ""),
   CACHE_HIT_RATIO_DISPLAY_SERVICE(
       "CACHE_HIT_RATIO_DISPLAY_SERVICE",
       generateJmxName("org.apache.iotdb.service", "Cache Hit Ratio")),
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/sync/SyncConstant.java b/node-commons/src/main/java/org/apache/iotdb/commons/sync/SyncConstant.java
index 81f54e7..8e17263 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/sync/SyncConstant.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/sync/SyncConstant.java
@@ -26,6 +26,8 @@
 
   public static final String SYNC_SYS_DIR = "sys";
   public static final String FILE_DATA_DIR_NAME = "file-data";
+  public static final String ROLE_SENDER = "sender";
+  public static final String ROLE_RECEIVER = "receiver";
 
   // pipe log: serialNumber + SEPARATOR + SUFFIX
   public static final String PIPE_LOG_DIR_NAME = "pipe-log";
@@ -54,7 +56,7 @@
 
   // data config
   public static final String DEFAULT_PIPE_SINK_IP = "127.0.0.1";
-  public static final int DEFAULT_PIPE_SINK_PORT = 6670;
+  public static final int DEFAULT_PIPE_SINK_PORT = 6667;
 
   public static final Long HEARTBEAT_DELAY_SECONDS = 30L;
   public static final int CONNECT_TIMEOUT_MILLISECONDS = 1_000;
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/trigger/TriggerInformation.java b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/TriggerInformation.java
new file mode 100644
index 0000000..3b31fb4
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/TriggerInformation.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.commons.trigger;
+
+import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils;
+import org.apache.iotdb.confignode.rpc.thrift.TTriggerState;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/** This Class used to save the specific information of one Trigger. */
+public class TriggerInformation {
+  private PartialPath pathPattern;
+  private String triggerName;
+  private String className;
+  private String jarName;
+  private TTriggerState triggerState;
+
+  /** indicate this Trigger is Stateful or Stateless */
+  private boolean isStateful;
+
+  /** only used for Stateful Trigger */
+  private TDataNodeLocation dataNodeLocation;
+
+  public TriggerInformation() {};
+
+  public TriggerInformation(
+      PartialPath pathPattern,
+      String triggerName,
+      String className,
+      String jarName,
+      TTriggerState triggerState,
+      boolean isStateful,
+      TDataNodeLocation dataNodeLocation) {
+    this.pathPattern = pathPattern;
+    this.triggerName = triggerName;
+    this.className = className;
+    this.jarName = jarName;
+    this.triggerState = triggerState;
+    this.isStateful = isStateful;
+    this.dataNodeLocation = dataNodeLocation;
+  }
+
+  public ByteBuffer serialize() throws IOException {
+    PublicBAOS byteArrayOutputStream = new PublicBAOS();
+    DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream);
+    serialize(outputStream);
+    return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size());
+  }
+
+  public void serialize(DataOutputStream outputStream) throws IOException {
+    pathPattern.serialize(outputStream);
+    ReadWriteIOUtils.write(triggerName, outputStream);
+    ReadWriteIOUtils.write(className, outputStream);
+    ReadWriteIOUtils.write(jarName, outputStream);
+    ReadWriteIOUtils.write(triggerState.getValue(), outputStream);
+    ReadWriteIOUtils.write(isStateful, outputStream);
+    if (isStateful) {
+      ThriftCommonsSerDeUtils.serializeTDataNodeLocation(dataNodeLocation, outputStream);
+    }
+  }
+
+  public static TriggerInformation deserialize(ByteBuffer byteBuffer) {
+    TriggerInformation triggerInformation = new TriggerInformation();
+    triggerInformation.pathPattern = PartialPath.deserialize(byteBuffer);
+    triggerInformation.triggerName = ReadWriteIOUtils.readString(byteBuffer);
+    triggerInformation.className = ReadWriteIOUtils.readString(byteBuffer);
+    triggerInformation.jarName = ReadWriteIOUtils.readString(byteBuffer);
+    triggerInformation.triggerState =
+        TTriggerState.findByValue(ReadWriteIOUtils.readInt(byteBuffer));
+    boolean isStateful = ReadWriteIOUtils.readBool(byteBuffer);
+    triggerInformation.isStateful = isStateful;
+    if (isStateful) {
+      triggerInformation.dataNodeLocation =
+          ThriftCommonsSerDeUtils.deserializeTDataNodeLocation(byteBuffer);
+    }
+    return triggerInformation;
+  }
+
+  public PartialPath getPathPattern() {
+    return pathPattern;
+  }
+
+  public void setPathPattern(PartialPath pathPattern) {
+    this.pathPattern = pathPattern;
+  }
+
+  public String getTriggerName() {
+    return triggerName;
+  }
+
+  public void setTriggerName(String triggerName) {
+    this.triggerName = triggerName;
+  }
+
+  public String getClassName() {
+    return className;
+  }
+
+  public void setClassName(String className) {
+    this.className = className;
+  }
+
+  public String getJarName() {
+    return jarName;
+  }
+
+  public void setJarName(String jarName) {
+    this.jarName = jarName;
+  }
+
+  public TTriggerState getTriggerState() {
+    return triggerState;
+  }
+
+  public void setTriggerState(TTriggerState triggerState) {
+    this.triggerState = triggerState;
+  }
+
+  public boolean isStateful() {
+    return isStateful;
+  }
+
+  public void setStateful(boolean stateful) {
+    isStateful = stateful;
+  }
+
+  public TDataNodeLocation getDataNodeLocation() {
+    return dataNodeLocation;
+  }
+
+  public void setDataNodeLocation(TDataNodeLocation dataNodeLocation) {
+    this.dataNodeLocation = dataNodeLocation;
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/trigger/TriggerTable.java b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/TriggerTable.java
new file mode 100644
index 0000000..574222e
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/TriggerTable.java
@@ -0,0 +1,64 @@
+/*
+ * 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.commons.trigger;
+
+import org.apache.iotdb.confignode.rpc.thrift.TTriggerState;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/** This Class used to save the information of Triggers and implements methods of manipulate it. */
+@NotThreadSafe
+public class TriggerTable {
+  private final Map<String, TriggerInformation> triggerTable;
+
+  public TriggerTable() {
+    triggerTable = new HashMap<>();
+  }
+
+  public TriggerTable(Map<String, TriggerInformation> triggerTable) {
+    this.triggerTable = triggerTable;
+  }
+
+  // for createTrigger
+  public void addTriggerInformation(String triggerName, TriggerInformation triggerInformation) {
+    triggerTable.put(triggerName, triggerInformation);
+  }
+
+  // for dropTrigger
+  public void deleteTriggerInformation(String triggerName) {
+    triggerTable.remove(triggerName);
+  }
+
+  // for showTrigger
+  public Map<String, TTriggerState> getAllTriggerStates() {
+    Map<String, TTriggerState> allTriggerStates = new HashMap<>(triggerTable.size());
+
+    triggerTable.forEach((k, v) -> allTriggerStates.put(k, v.getTriggerState()));
+    return allTriggerStates;
+  }
+
+  // for getTriggerTable
+  public Map<String, TriggerInformation> getTable() {
+    return triggerTable;
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/exception/TriggerExecutionException.java
similarity index 66%
copy from node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java
copy to node-commons/src/main/java/org/apache/iotdb/commons/trigger/exception/TriggerExecutionException.java
index ec0c375..0c4ba54 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/exception/TriggerExecutionException.java
@@ -17,23 +17,14 @@
  * under the License.
  */
 
-package org.apache.iotdb.commons.udf.service;
+package org.apache.iotdb.commons.trigger.exception;
 
-public class UDFExecutableResource {
-
-  private final long requestId;
-  private final String resourceDir;
-
-  public UDFExecutableResource(long requestId, String resourceDir) {
-    this.requestId = requestId;
-    this.resourceDir = resourceDir;
+public class TriggerExecutionException extends RuntimeException {
+  public TriggerExecutionException(String message) {
+    super(message);
   }
 
-  public long getRequestId() {
-    return requestId;
-  }
-
-  public String getResourceDir() {
-    return resourceDir;
+  public TriggerExecutionException(String message, Throwable cause) {
+    super(message, cause);
   }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/exception/TriggerRegistrationException.java
similarity index 66%
copy from node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java
copy to node-commons/src/main/java/org/apache/iotdb/commons/trigger/exception/TriggerRegistrationException.java
index ec0c375..34a18dc 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/exception/TriggerRegistrationException.java
@@ -17,23 +17,14 @@
  * under the License.
  */
 
-package org.apache.iotdb.commons.udf.service;
+package org.apache.iotdb.commons.trigger.exception;
 
-public class UDFExecutableResource {
-
-  private final long requestId;
-  private final String resourceDir;
-
-  public UDFExecutableResource(long requestId, String resourceDir) {
-    this.requestId = requestId;
-    this.resourceDir = resourceDir;
+public class TriggerRegistrationException extends RuntimeException {
+  public TriggerRegistrationException(String message) {
+    super(message);
   }
 
-  public long getRequestId() {
-    return requestId;
-  }
-
-  public String getResourceDir() {
-    return resourceDir;
+  public TriggerRegistrationException(String message, Throwable cause) {
+    super(message, cause);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerClassLoader.java b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/service/TriggerClassLoader.java
similarity index 93%
rename from server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerClassLoader.java
rename to node-commons/src/main/java/org/apache/iotdb/commons/trigger/service/TriggerClassLoader.java
index 0207b10..9356c23 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerClassLoader.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/service/TriggerClassLoader.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.trigger.service;
+package org.apache.iotdb.commons.trigger.service;
 
 import org.apache.iotdb.commons.file.SystemFileFactory;
 
@@ -37,7 +37,7 @@
 
   private final String libRoot;
 
-  TriggerClassLoader(String libRoot) throws IOException {
+  public TriggerClassLoader(String libRoot) throws IOException {
     super(new URL[0]);
     this.libRoot = libRoot;
     LOGGER.info("Trigger lib root: {}", libRoot);
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/trigger/service/TriggerClassLoaderManager.java b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/service/TriggerClassLoaderManager.java
new file mode 100644
index 0000000..f94d343
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/service/TriggerClassLoaderManager.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.commons.trigger.service;
+
+import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.commons.file.SystemFileFactory;
+import org.apache.iotdb.commons.service.IService;
+import org.apache.iotdb.commons.service.ServiceType;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class TriggerClassLoaderManager implements IService {
+  private static final Logger LOGGER = LoggerFactory.getLogger(TriggerClassLoaderManager.class);
+
+  /** The dir that stores jar files. */
+  private final String libRoot;
+
+  /**
+   * activeClassLoader is used to load all classes under libRoot. libRoot may be updated before the
+   * user executes CREATE TRIGGER or after the user executes DROP TRIGGER. Therefore, we need to
+   * continuously maintain the activeClassLoader so that the classes it loads are always up-to-date.
+   */
+  private volatile TriggerClassLoader activeClassLoader;
+
+  private TriggerClassLoaderManager(String libRoot) {
+    this.libRoot = libRoot;
+    LOGGER.info("Trigger lib root: {}", libRoot);
+    activeClassLoader = null;
+  }
+
+  /** Call this method to get up-to-date ClassLoader before registering triggers */
+  public TriggerClassLoader updateAndGetActiveClassLoader() throws IOException {
+    TriggerClassLoader deprecatedClassLoader = activeClassLoader;
+    activeClassLoader = new TriggerClassLoader(libRoot);
+    deprecatedClassLoader.close();
+    return activeClassLoader;
+  }
+
+  public TriggerClassLoader getActiveClassLoader() {
+    return activeClassLoader;
+  }
+
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+  // IService
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+
+  @Override
+  public void start() throws StartupException {
+    try {
+      SystemFileFactory.INSTANCE.makeDirIfNecessary(libRoot);
+      activeClassLoader = new TriggerClassLoader(libRoot);
+    } catch (IOException e) {
+      throw new StartupException(this.getID().getName(), e.getMessage());
+    }
+  }
+
+  @Override
+  public void stop() {
+    // nothing to do
+  }
+
+  @Override
+  public ServiceType getID() {
+    return ServiceType.TRIGGER_CLASSLOADER_MANAGER_SERVICE;
+  }
+
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+  // singleton instance holder
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+
+  private static TriggerClassLoaderManager INSTANCE = null;
+
+  public static synchronized TriggerClassLoaderManager setupAndGetInstance(String libRoot) {
+    if (INSTANCE == null) {
+      INSTANCE = new TriggerClassLoaderManager(libRoot);
+    }
+    return INSTANCE;
+  }
+
+  public static TriggerClassLoaderManager getInstance() {
+    return INSTANCE;
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/trigger/service/TriggerRegistrationService.java b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/service/TriggerRegistrationService.java
new file mode 100644
index 0000000..68e7e20
--- /dev/null
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/trigger/service/TriggerRegistrationService.java
@@ -0,0 +1,89 @@
+/*
+ * 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.commons.trigger.service;
+
+import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.commons.service.IService;
+import org.apache.iotdb.commons.service.ServiceType;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.locks.ReentrantLock;
+
+public class TriggerRegistrationService implements IService {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(TriggerRegistrationService.class);
+
+  private final ReentrantLock registrationLock;
+
+  private TriggerRegistrationService() {
+    this.registrationLock = new ReentrantLock();
+  }
+
+  public void acquireRegistrationLock() {
+    registrationLock.lock();
+  }
+
+  public void releaseRegistrationLock() {
+    registrationLock.unlock();
+  }
+
+  // todo: implementation
+  public void register() {
+    // validate before registering
+    // add to triggerTable and set inactive
+    // throw exception if registered
+  };
+
+  public void activeTrigger(String triggerName) {
+    // active trigger in table
+  };
+
+  @Override
+  public void start() throws StartupException {}
+
+  @Override
+  public void stop() {
+    // nothing to do
+  }
+
+  @Override
+  public ServiceType getID() {
+    return ServiceType.TRIGGER_REGISTRATION_SERVICE;
+  }
+
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+  // singleton instance holder
+  /////////////////////////////////////////////////////////////////////////////////////////////////
+
+  private static TriggerRegistrationService INSTANCE = null;
+
+  public static synchronized TriggerRegistrationService setupAndGetInstance() {
+    if (INSTANCE == null) {
+      INSTANCE = new TriggerRegistrationService();
+    }
+    return INSTANCE;
+  }
+
+  public static TriggerRegistrationService getInstance() {
+    return INSTANCE;
+  }
+}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFClassLoaderManager.java b/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFClassLoaderManager.java
index b74f47c..9d98576 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFClassLoaderManager.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFClassLoaderManager.java
@@ -24,11 +24,9 @@
 import org.apache.iotdb.commons.service.IService;
 import org.apache.iotdb.commons.service.ServiceType;
 
-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.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -91,21 +89,13 @@
   @Override
   public void start() throws StartupException {
     try {
-      makeDirIfNecessary();
+      SystemFileFactory.INSTANCE.makeDirIfNecessary(libRoot);
       activeClassLoader = new UDFClassLoader(libRoot);
     } catch (IOException e) {
       throw new StartupException(this.getID().getName(), e.getMessage());
     }
   }
 
-  private void makeDirIfNecessary() throws IOException {
-    File file = SystemFileFactory.INSTANCE.getFile(libRoot);
-    if (file.exists() && file.isDirectory()) {
-      return;
-    }
-    FileUtils.forceMkdir(file);
-  }
-
   @Override
   public void stop() {
     // nothing to do
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableManager.java b/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableManager.java
index f7f18d7..a1cf6f3 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableManager.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableManager.java
@@ -20,104 +20,19 @@
 package org.apache.iotdb.commons.udf.service;
 
 import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.commons.executable.ExecutableManager;
 import org.apache.iotdb.commons.file.SystemFileFactory;
 import org.apache.iotdb.commons.service.IService;
 import org.apache.iotdb.commons.service.ServiceType;
 import org.apache.iotdb.commons.snapshot.SnapshotProcessor;
-import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
-
-import org.apache.commons.io.FileUtils;
 
 import java.io.File;
 import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicLong;
 
-public class UDFExecutableManager implements IService, SnapshotProcessor {
-
-  private final String temporaryLibRoot;
-  private final String udfLibRoot;
-
-  private final AtomicLong requestCounter;
+public class UDFExecutableManager extends ExecutableManager implements IService, SnapshotProcessor {
 
   private UDFExecutableManager(String temporaryLibRoot, String udfLibRoot) {
-    this.temporaryLibRoot = temporaryLibRoot;
-    this.udfLibRoot = udfLibRoot;
-
-    requestCounter = new AtomicLong(0);
-  }
-
-  public UDFExecutableResource request(List<String> uris) throws URISyntaxException, IOException {
-    final long requestId = generateNextRequestId();
-    downloadExecutables(uris, requestId);
-    return new UDFExecutableResource(requestId, getDirStringByRequestId(requestId));
-  }
-
-  public void moveToExtLibDir(UDFExecutableResource resource, String functionName)
-      throws IOException {
-    FileUtils.moveDirectory(
-        getDirByRequestId(resource.getRequestId()), getDirByFunctionName(functionName));
-  }
-
-  public void removeFromTemporaryLibRoot(UDFExecutableResource resource) {
-    removeFromTemporaryLibRoot(resource.getRequestId());
-  }
-
-  public void removeFromExtLibDir(String functionName) {
-    FileUtils.deleteQuietly(getDirByFunctionName(functionName));
-  }
-
-  private synchronized long generateNextRequestId() throws IOException {
-    long requestId = requestCounter.getAndIncrement();
-    while (FileUtils.isDirectory(getDirByRequestId(requestId))) {
-      requestId = requestCounter.getAndIncrement();
-    }
-    FileUtils.forceMkdir(getDirByRequestId(requestId));
-    return requestId;
-  }
-
-  private void downloadExecutables(List<String> uris, long requestId)
-      throws IOException, URISyntaxException {
-    // TODO: para download
-    try {
-      for (String uriString : uris) {
-        final URL url = new URI(uriString).toURL();
-        final String fileName = uriString.substring(uriString.lastIndexOf("/") + 1);
-        final String destination =
-            temporaryLibRoot + File.separator + requestId + File.separator + fileName;
-        FileUtils.copyURLToFile(url, FSFactoryProducer.getFSFactory().getFile(destination));
-      }
-    } catch (Exception e) {
-      removeFromTemporaryLibRoot(requestId);
-      throw e;
-    }
-  }
-
-  private void removeFromTemporaryLibRoot(long requestId) {
-    FileUtils.deleteQuietly(getDirByRequestId(requestId));
-  }
-
-  /////////////////////////////////////////////////////////////////////////////////////////////////
-  // dir string and dir file generation
-  /////////////////////////////////////////////////////////////////////////////////////////////////
-
-  public File getDirByRequestId(long requestId) {
-    return FSFactoryProducer.getFSFactory().getFile(getDirStringByRequestId(requestId));
-  }
-
-  public String getDirStringByRequestId(long requestId) {
-    return temporaryLibRoot + File.separator + requestId + File.separator;
-  }
-
-  public File getDirByFunctionName(String functionName) {
-    return FSFactoryProducer.getFSFactory().getFile(getDirStringByFunctionName(functionName));
-  }
-
-  public String getDirStringByFunctionName(String functionName) {
-    return udfLibRoot + File.separator + functionName + File.separator;
+    super(temporaryLibRoot, udfLibRoot);
   }
 
   /////////////////////////////////////////////////////////////////////////////////////////////////
@@ -127,8 +42,8 @@
   @Override
   public void start() throws StartupException {
     try {
-      makeDirIfNecessary(temporaryLibRoot);
-      makeDirIfNecessary(udfLibRoot);
+      SystemFileFactory.INSTANCE.makeDirIfNecessary(temporaryLibRoot);
+      SystemFileFactory.INSTANCE.makeDirIfNecessary(libRoot);
     } catch (Exception e) {
       throw new StartupException(e);
     }
@@ -158,14 +73,6 @@
     return INSTANCE;
   }
 
-  private static void makeDirIfNecessary(String dir) throws IOException {
-    File file = SystemFileFactory.INSTANCE.getFile(dir);
-    if (file.exists() && file.isDirectory()) {
-      return;
-    }
-    FileUtils.forceMkdir(file);
-  }
-
   public static UDFExecutableManager getInstance() {
     return INSTANCE;
   }
@@ -180,7 +87,7 @@
             temporaryLibRoot,
             snapshotDir.getAbsolutePath() + File.separator + "ext" + File.separator + "temporary")
         && SnapshotUtils.takeSnapshotForDir(
-            udfLibRoot,
+            libRoot,
             snapshotDir.getAbsolutePath() + File.separator + "ext" + File.separator + "udf");
   }
 
@@ -190,7 +97,6 @@
         snapshotDir.getAbsolutePath() + File.separator + "ext" + File.separator + "temporary",
         temporaryLibRoot);
     SnapshotUtils.loadSnapshotForDir(
-        snapshotDir.getAbsolutePath() + File.separator + "ext" + File.separator + "udf",
-        udfLibRoot);
+        snapshotDir.getAbsolutePath() + File.separator + "ext" + File.separator + "udf", libRoot);
   }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFRegistrationService.java b/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFRegistrationService.java
index 332906e..c0bf6a5 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFRegistrationService.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFRegistrationService.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.commons.udf.service;
 
 import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.commons.executable.ExecutableResource;
 import org.apache.iotdb.commons.file.SystemFileFactory;
 import org.apache.iotdb.commons.service.IService;
 import org.apache.iotdb.commons.service.ServiceType;
@@ -170,7 +171,7 @@
     }
 
     try {
-      final UDFExecutableResource resource = udfExecutableManager.request(uris);
+      final ExecutableResource resource = udfExecutableManager.request(uris);
       try {
         udfExecutableManager.removeFromExtLibDir(functionName);
         udfExecutableManager.moveToExtLibDir(resource, functionName);
diff --git a/node-commons/src/test/java/org/apache/iotdb/commons/client/mock/MockInternalRPCService.java b/node-commons/src/test/java/org/apache/iotdb/commons/client/mock/MockInternalRPCService.java
index 85f1d63..c4a6b4d 100644
--- a/node-commons/src/test/java/org/apache/iotdb/commons/client/mock/MockInternalRPCService.java
+++ b/node-commons/src/test/java/org/apache/iotdb/commons/client/mock/MockInternalRPCService.java
@@ -67,7 +67,7 @@
           new ThriftServiceThread(
               processor,
               getID().getName(),
-              ThreadName.INTERNAL_SERVICE_RPC_SERVER.getName(),
+              ThreadName.DATANODE_INTERNAL_RPC_SERVICE.getName(),
               getBindIP(),
               getBindPort(),
               65535,
@@ -77,7 +77,7 @@
     } catch (RPCServiceException e) {
       throw new IllegalAccessException(e.getMessage());
     }
-    thriftServiceThread.setName(ThreadName.INTERNAL_SERVICE_RPC_SERVER.getName());
+    thriftServiceThread.setName(ThreadName.DATANODE_INTERNAL_RPC_SERVICE.getName());
   }
 
   @Override
diff --git a/pom.xml b/pom.xml
index f43d7aa..e89819a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -118,6 +118,7 @@
         <module>library-udf</module>
         <module>schema-engine-rocksdb</module>
         <module>udf-api</module>
+        <module>trigger-api</module>
         <module>rewrite-tsfile-tool</module>
         <module>external-api</module>
     </modules>
diff --git a/server/src/assembly/resources/conf/iotdb-datanode.properties b/server/src/assembly/resources/conf/iotdb-datanode.properties
index 99afaf6..9322914 100644
--- a/server/src/assembly/resources/conf/iotdb-datanode.properties
+++ b/server/src/assembly/resources/conf/iotdb-datanode.properties
@@ -97,6 +97,15 @@
 # Datatype: int
 # connection_timeout_ms=20000
 
+# The maximum number of clients that can be idle for a node's InternalService.
+# When the number of idle clients on a node exceeds this number, newly returned clients will be released
+# Datatype: int
+# core_connection_for_internal_service=100
+
+# The maximum number of clients that can be applied for a node's InternalService
+# Datatype: int
+# max_connection_for_internal_service=100
+
 # selector thread (TAsyncClientManager) nums for async thread in a clientManager
 # Datatype: int
 # selector_thread_nums_of_client_manager=1
diff --git a/server/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java b/server/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java
index f524f6f..3316006 100644
--- a/server/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java
+++ b/server/src/main/java/org/apache/iotdb/db/auth/AuthorityChecker.java
@@ -39,7 +39,7 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import static org.apache.iotdb.db.utils.ErrorHandlingUtils.onNPEOrUnexpectedException;
+import static org.apache.iotdb.db.utils.ErrorHandlingUtils.onQueryException;
 
 public class AuthorityChecker {
 
@@ -162,8 +162,8 @@
       logger.warn("meet error while checking authorization.", e);
       return RpcUtils.getStatus(TSStatusCode.UNINITIALIZED_AUTH_ERROR, e.getMessage());
     } catch (Exception e) {
-      return onNPEOrUnexpectedException(
-          e, OperationType.CHECK_AUTHORITY, TSStatusCode.EXECUTE_STATEMENT_ERROR);
+      return onQueryException(
+          e, OperationType.CHECK_AUTHORITY.getName(), TSStatusCode.EXECUTE_STATEMENT_ERROR);
     }
     return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
index 937e41c..de2bc80 100644
--- a/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
+++ b/server/src/main/java/org/apache/iotdb/db/client/ConfigNodeClient.java
@@ -52,6 +52,7 @@
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteStorageGroupReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteStorageGroupsReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDropFunctionReq;
+import org.apache.iotdb.confignode.rpc.thrift.TDropTriggerReq;
 import org.apache.iotdb.confignode.rpc.thrift.TGetAllTemplatesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TGetPathsSetTemplatesResp;
 import org.apache.iotdb.confignode.rpc.thrift.TGetTemplateResp;
@@ -886,6 +887,22 @@
   }
 
   @Override
+  public TSStatus dropTrigger(TDropTriggerReq req) throws TException {
+    for (int i = 0; i < RETRY_NUM; i++) {
+      try {
+        TSStatus status = client.dropTrigger(req);
+        if (!updateConfigNodeLeader(status)) {
+          return status;
+        }
+      } catch (TException e) {
+        configLeader = null;
+      }
+      reconnect();
+    }
+    throw new TException(MSG_RECONNECTION_FAIL);
+  }
+
+  @Override
   public TSStatus createSchemaTemplate(TCreateSchemaTemplateReq req) throws TException {
     for (int i = 0; i < RETRY_NUM; i++) {
       try {
diff --git a/server/src/main/java/org/apache/iotdb/db/client/DataNodeClientPoolFactory.java b/server/src/main/java/org/apache/iotdb/db/client/DataNodeClientPoolFactory.java
index 8f3e0d1..996e237 100644
--- a/server/src/main/java/org/apache/iotdb/db/client/DataNodeClientPoolFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/client/DataNodeClientPoolFactory.java
@@ -73,8 +73,12 @@
                   .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnable())
                   .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager())
                   .build(),
-              ThreadName.CONFIG_NODE_CLIENT_POOL_THREAD_NAME.getName()),
-          new ClientPoolProperty.Builder<AsyncConfigNodeIServiceClient>().build().getConfig());
+              ThreadName.ASYNC_CONFIGNODE_CLIENT_POOL.getName()),
+          new ClientPoolProperty.Builder<AsyncConfigNodeIServiceClient>()
+              .setMaxIdleClientForEachNode(conf.getCoreConnectionForInternalService())
+              .setMaxTotalClientForEachNode(conf.getMaxConnectionForInternalService())
+              .build()
+              .getConfig());
     }
   }
 
@@ -91,7 +95,11 @@
                   .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnable())
                   .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager())
                   .build()),
-          new ClientPoolProperty.Builder<SyncDataNodeInternalServiceClient>().build().getConfig());
+          new ClientPoolProperty.Builder<SyncDataNodeInternalServiceClient>()
+              .setMaxIdleClientForEachNode(conf.getCoreConnectionForInternalService())
+              .setMaxTotalClientForEachNode(conf.getMaxConnectionForInternalService())
+              .build()
+              .getConfig());
     }
   }
 
@@ -127,7 +135,7 @@
                   .setRpcThriftCompressionEnabled(conf.isRpcThriftCompressionEnable())
                   .setSelectorNumOfAsyncClientManager(conf.getSelectorNumOfClientManager())
                   .build(),
-              ThreadName.DATA_NODE_MPP_DATA_EXCHANGE_CLIENT_POOL_THREAD_NAME.getName()),
+              ThreadName.ASYNC_DATANODE_MPP_DATA_EXCHANGE_CLIENT_POOL.getName()),
           new ClientPoolProperty.Builder<AsyncDataNodeMPPDataExchangeServiceClient>()
               .build()
               .getConfig());
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 750d10e..0b03d48 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
@@ -910,6 +910,15 @@
   /** Thrift socket and connection timeout between data node and config node. */
   private int connectionTimeoutInMS = (int) TimeUnit.SECONDS.toMillis(20);
 
+  /** the maximum number of clients that can be applied for a node's InternalService */
+  private int maxConnectionForInternalService = 100;
+
+  /**
+   * the maximum number of clients that can be idle for a node's InternalService. When the number of
+   * idle clients on a node exceeds this number, newly returned clients will be released
+   */
+  private int coreConnectionForInternalService = 100;
+
   /**
    * ClientManager will have so many selector threads (TAsyncClientManager) to distribute to its
    * clients.
@@ -2903,6 +2912,22 @@
     this.connectionTimeoutInMS = connectionTimeoutInMS;
   }
 
+  public int getMaxConnectionForInternalService() {
+    return maxConnectionForInternalService;
+  }
+
+  public void setMaxConnectionForInternalService(int maxConnectionForInternalService) {
+    this.maxConnectionForInternalService = maxConnectionForInternalService;
+  }
+
+  public int getCoreConnectionForInternalService() {
+    return coreConnectionForInternalService;
+  }
+
+  public void setCoreConnectionForInternalService(int coreConnectionForInternalService) {
+    this.coreConnectionForInternalService = coreConnectionForInternalService;
+  }
+
   public int getSelectorNumOfClientManager() {
     return selectorNumOfClientManager;
   }
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 be8c9d2..ac3e7d5 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
@@ -239,6 +239,18 @@
             properties.getProperty(
                 "connection_timeout_ms", String.valueOf(conf.getConnectionTimeoutInMS()))));
 
+    conf.setMaxConnectionForInternalService(
+        Integer.parseInt(
+            properties.getProperty(
+                "max_connection_for_internal_service",
+                String.valueOf(conf.getMaxConnectionForInternalService()))));
+
+    conf.setCoreConnectionForInternalService(
+        Integer.parseInt(
+            properties.getProperty(
+                "core_connection_for_internal_service",
+                String.valueOf(conf.getCoreConnectionForInternalService()))));
+
     conf.setSelectorNumOfClientManager(
         Integer.parseInt(
             properties.getProperty(
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java
index c14ebd6..f50abbf 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java
@@ -50,6 +50,7 @@
 import org.apache.iotdb.db.exception.runtime.StorageEngineFailureException;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode;
 import org.apache.iotdb.db.rescon.SystemInfo;
+import org.apache.iotdb.db.sync.SyncService;
 import org.apache.iotdb.db.utils.ThreadUtils;
 import org.apache.iotdb.db.utils.UpgradeUtils;
 import org.apache.iotdb.db.wal.WALManager;
@@ -665,6 +666,7 @@
               .deleteWALNode(
                   region.getStorageGroupName() + FILE_NAME_SEPARATOR + region.getDataRegionId());
         }
+        SyncService.getInstance().deleteSyncManager(region.getDataRegionId());
       } catch (Exception e) {
         logger.error(
             "Error occurs when deleting data region {}-{}",
@@ -682,6 +684,10 @@
     return dataRegionMap.get(regionId);
   }
 
+  public List<DataRegion> getAllDataRegions() {
+    return new ArrayList<>(dataRegionMap.values());
+  }
+
   public List<DataRegionId> getAllDataRegionIds() {
     return new ArrayList<>(dataRegionMap.keySet());
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/snapshot/SnapshotLoader.java b/server/src/main/java/org/apache/iotdb/db/engine/snapshot/SnapshotLoader.java
index cff5886..e3fdf37 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/snapshot/SnapshotLoader.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/snapshot/SnapshotLoader.java
@@ -114,6 +114,13 @@
 
   private DataRegion loadSnapshotWithoutLog() {
     try {
+      try {
+        deleteAllFilesInDataDirs();
+        LOGGER.info("Remove all data files in original data dir");
+      } catch (IOException e) {
+        LOGGER.error("Failed to remove origin data files", e);
+        return null;
+      }
       LOGGER.info("Moving snapshot file to data dirs");
       createLinksFromSnapshotDirToDataDirWithoutLog(new File(snapshotPath));
       return loadSnapshot();
@@ -137,6 +144,7 @@
         deleteAllFilesInDataDirs();
         LOGGER.info("Remove all data files in original data dir");
       } catch (IOException e) {
+        LOGGER.error("Failed to remove origin data files", e);
         return null;
       }
 
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 e58f552..7f959fb 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
@@ -88,7 +88,8 @@
 import org.apache.iotdb.db.service.metrics.MetricService;
 import org.apache.iotdb.db.service.metrics.enums.Metric;
 import org.apache.iotdb.db.service.metrics.enums.Tag;
-import org.apache.iotdb.db.sync.sender.manager.TsFileSyncManager;
+import org.apache.iotdb.db.sync.SyncService;
+import org.apache.iotdb.db.sync.sender.manager.ISyncManager;
 import org.apache.iotdb.db.tools.settle.TsFileAndModSettleTool;
 import org.apache.iotdb.db.utils.CopyOnReadLinkedList;
 import org.apache.iotdb.db.utils.UpgradeUtils;
@@ -275,9 +276,6 @@
 
   private IDTable idTable;
 
-  /** used to collect TsFiles in this virtual storage group */
-  private TsFileSyncManager tsFileSyncManager = TsFileSyncManager.getInstance();
-
   /**
    * constrcut a storage group processor
    *
@@ -751,8 +749,9 @@
     TsFileResource tsFileResource = recoverPerformer.getTsFileResource();
     if (!recoverPerformer.canWrite()) {
       // cannot write, just close it
-      if (tsFileSyncManager.isEnableSync()) {
-        tsFileSyncManager.collectRealTimeTsFile(tsFileResource.getTsFile());
+      for (ISyncManager syncManager :
+          SyncService.getInstance().getOrCreateSyncManager(dataRegionId)) {
+        syncManager.syncRealTimeTsFile(tsFileResource.getTsFile());
       }
       try {
         tsFileResource.close();
@@ -2414,8 +2413,9 @@
         tsFileResource.getProcessor().deleteDataInMemory(deletion, devicePaths);
       }
 
-      if (tsFileSyncManager.isEnableSync()) {
-        tsFileSyncManager.collectRealTimeDeletion(deletion, storageGroupName);
+      for (ISyncManager syncManager :
+          SyncService.getInstance().getOrCreateSyncManager(dataRegionId)) {
+        syncManager.syncRealTimeDeletion(deletion);
       }
 
       // add a record in case of rollback
@@ -3677,14 +3677,15 @@
   /**
    * Used to collect history TsFiles(i.e. the tsfile whose memtable == null).
    *
+   * @param syncManager ISyncManager which invokes to collect history TsFile
    * @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) {
+  public List<File> collectHistoryTsFileForSync(ISyncManager syncManager, long dataStartTime) {
     writeLock("Collect data for sync");
     try {
-      return tsFileManager.collectHistoryTsFileForSync(dataStartTime);
+      return tsFileManager.collectHistoryTsFileForSync(syncManager, dataStartTime);
     } finally {
       writeUnlock();
     }
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 f09347c..32c70b1 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,7 +21,7 @@
 
 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.apache.iotdb.db.sync.sender.manager.ISyncManager;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -373,12 +373,12 @@
     return unsequenceRecoverTsFileResources;
   }
 
-  public List<File> collectHistoryTsFileForSync(long dataStartTime) {
+  public List<File> collectHistoryTsFileForSync(ISyncManager syncManager, long dataStartTime) {
     readLock();
     try {
       List<File> historyTsFiles = new ArrayList<>();
-      collectTsFile(historyTsFiles, getTsFileList(true), dataStartTime);
-      collectTsFile(historyTsFiles, getTsFileList(false), dataStartTime);
+      collectTsFile(historyTsFiles, getTsFileList(true), syncManager, dataStartTime);
+      collectTsFile(historyTsFiles, getTsFileList(false), syncManager, dataStartTime);
       return historyTsFiles;
     } finally {
       readUnlock();
@@ -386,8 +386,10 @@
   }
 
   private void collectTsFile(
-      List<File> historyTsFiles, List<TsFileResource> tsFileResources, long dataStartTime) {
-    TsFileSyncManager syncManager = TsFileSyncManager.getInstance();
+      List<File> historyTsFiles,
+      List<TsFileResource> tsFileResources,
+      ISyncManager syncManager,
+      long dataStartTime) {
 
     for (TsFileResource tsFileResource : tsFileResources) {
       if (tsFileResource.getFileEndTime() < dataStartTime) {
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 c1f7e98..605cc979 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
@@ -59,7 +59,8 @@
 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.sync.SyncService;
+import org.apache.iotdb.db.sync.sender.manager.ISyncManager;
 import org.apache.iotdb.db.utils.MemUtils;
 import org.apache.iotdb.db.utils.datastructure.AlignedTVList;
 import org.apache.iotdb.db.utils.datastructure.TVList;
@@ -166,9 +167,6 @@
   /** 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,
@@ -839,8 +837,10 @@
       if (!flushingMemTables.isEmpty()) {
         modsToMemtable.add(new Pair<>(deletion, flushingMemTables.getLast()));
       }
-      if (tsFileSyncManager.isEnableSync()) {
-        tsFileSyncManager.collectRealTimeDeletion(deletion, storageGroupName);
+      for (ISyncManager syncManager :
+          SyncService.getInstance()
+              .getOrCreateSyncManager(storageGroupInfo.getDataRegion().getDataRegionId())) {
+        syncManager.syncRealTimeDeletion(deletion);
       }
     } finally {
       flushQueryLock.writeLock().unlock();
@@ -996,8 +996,10 @@
         // 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());
+        for (ISyncManager syncManager :
+            SyncService.getInstance()
+                .getOrCreateSyncManager(storageGroupInfo.getDataRegion().getDataRegionId())) {
+          syncManager.syncRealTimeTsFile(tsFileResource.getTsFile());
         }
         logger.info("Memtable {} has been added to flushing list", tmpMemTable);
         shouldClose = true;
@@ -1426,8 +1428,10 @@
     long closeStartTime = System.currentTimeMillis();
     writer.endFile();
     tsFileResource.serialize();
-    if (tsFileSyncManager.isEnableSync()) {
-      tsFileSyncManager.collectRealTimeResource(tsFileResource.getTsFile());
+    for (ISyncManager syncManager :
+        SyncService.getInstance()
+            .getOrCreateSyncManager(storageGroupInfo.getDataRegion().getDataRegionId())) {
+      syncManager.syncRealTimeResource(tsFileResource.getTsFile());
     }
     logger.info("Ended file {}", tsFileResource);
 
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 e079e77..35e55de 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,7 +37,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -466,15 +465,6 @@
     }
   }
 
-  /** 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/engine/trigger/api/Trigger.java b/server/src/main/java/org/apache/iotdb/db/engine/trigger/api/Trigger.java
index 48f9c22..6e9b29f 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/trigger/api/Trigger.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/trigger/api/Trigger.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.db.engine.trigger.api;
 
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.trigger.api.TriggerAttributes;
 import org.apache.iotdb.tsfile.utils.Binary;
 
 /** User Guide: docs/UserGuide/Operation Manual/Triggers.md */
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/trigger/builtin/ForwardTrigger.java b/server/src/main/java/org/apache/iotdb/db/engine/trigger/builtin/ForwardTrigger.java
index 7aa7523..a6df913 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/trigger/builtin/ForwardTrigger.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/trigger/builtin/ForwardTrigger.java
@@ -21,7 +21,6 @@
 
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.engine.trigger.api.Trigger;
-import org.apache.iotdb.db.engine.trigger.api.TriggerAttributes;
 import org.apache.iotdb.db.engine.trigger.sink.api.Configuration;
 import org.apache.iotdb.db.engine.trigger.sink.api.Event;
 import org.apache.iotdb.db.engine.trigger.sink.api.Handler;
@@ -34,6 +33,7 @@
 import org.apache.iotdb.db.engine.trigger.sink.forward.mqtt.MQTTForwardHandler;
 import org.apache.iotdb.db.engine.trigger.utils.BatchHandlerQueue;
 import org.apache.iotdb.db.exception.TriggerExecutionException;
+import org.apache.iotdb.trigger.api.TriggerAttributes;
 import org.apache.iotdb.tsfile.utils.Binary;
 
 import java.util.HashMap;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/trigger/executor/TriggerExecutor.java b/server/src/main/java/org/apache/iotdb/db/engine/trigger/executor/TriggerExecutor.java
index 732b4c0..b782ad5 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/trigger/executor/TriggerExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/trigger/executor/TriggerExecutor.java
@@ -20,14 +20,14 @@
 package org.apache.iotdb.db.engine.trigger.executor;
 
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.trigger.service.TriggerClassLoader;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.engine.trigger.api.Trigger;
-import org.apache.iotdb.db.engine.trigger.api.TriggerAttributes;
-import org.apache.iotdb.db.engine.trigger.service.TriggerClassLoader;
 import org.apache.iotdb.db.engine.trigger.service.TriggerRegistrationInformation;
 import org.apache.iotdb.db.exception.TriggerExecutionException;
 import org.apache.iotdb.db.exception.TriggerManagementException;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.trigger.api.TriggerAttributes;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Binary;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerClassLoaderManager.java b/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerClassLoaderManager.java
index ca46142..400d2ad 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerClassLoaderManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerClassLoaderManager.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.engine.trigger.service;
 
+import org.apache.iotdb.commons.trigger.service.TriggerClassLoader;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.TriggerManagementException;
 import org.apache.iotdb.tsfile.utils.Pair;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerRegistrationService.java b/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerRegistrationService.java
index f5a33a5..f555c11 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerRegistrationService.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/trigger/service/TriggerRegistrationService.java
@@ -25,6 +25,7 @@
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.service.IService;
 import org.apache.iotdb.commons.service.ServiceType;
+import org.apache.iotdb.commons.trigger.service.TriggerClassLoader;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -484,7 +485,7 @@
 
   @Override
   public ServiceType getID() {
-    return ServiceType.TRIGGER_REGISTRATION_SERVICE;
+    return ServiceType.TRIGGER_REGISTRATION_SERVICE_OLD;
   }
 
   public int executorSize() {
diff --git a/server/src/main/java/org/apache/iotdb/db/localconfignode/LocalConfigNode.java b/server/src/main/java/org/apache/iotdb/db/localconfignode/LocalConfigNode.java
index c6be69d..25c1c93 100644
--- a/server/src/main/java/org/apache/iotdb/db/localconfignode/LocalConfigNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/localconfignode/LocalConfigNode.java
@@ -50,6 +50,9 @@
 import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.utils.AuthUtils;
+import org.apache.iotdb.commons.utils.StatusUtils;
+import org.apache.iotdb.confignode.rpc.thrift.TPipeInfo;
+import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.StorageEngineV2;
@@ -65,6 +68,7 @@
 import org.apache.iotdb.db.exception.metadata.template.UndefinedTemplateException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
+import org.apache.iotdb.db.exception.sync.PipeException;
 import org.apache.iotdb.db.exception.sync.PipeSinkException;
 import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
 import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
@@ -79,19 +83,17 @@
 import org.apache.iotdb.db.mpp.plan.constant.DataNodeEndPoints;
 import org.apache.iotdb.db.mpp.plan.statement.sys.AuthorStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeStatement;
 import org.apache.iotdb.db.qp.logical.sys.AuthorOperator;
 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.DeleteTimeSeriesPlan;
 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.SyncService;
-import org.apache.iotdb.db.sync.sender.manager.SchemaSyncManager;
 import org.apache.iotdb.db.sync.sender.pipe.PipeSink;
 import org.apache.iotdb.rpc.RpcUtils;
 import org.apache.iotdb.rpc.TSStatusCode;
@@ -116,8 +118,6 @@
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
-import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
-
 /**
  * This class simulates the behaviour of configNode to manage the configs locally. The schema
  * configs include storage group, schema region and template. The data config is dataRegion.
@@ -274,10 +274,6 @@
       schemaEngine.createSchemaRegion(storageGroup, schemaRegionId);
     }
 
-    if (SchemaSyncManager.getInstance().isEnableSync()) {
-      SchemaSyncManager.getInstance().syncMetadataPlan(new SetStorageGroupPlan(storageGroup));
-    }
-
     if (!config.isEnableMemControl()) {
       MemTableManager.getInstance().addOrDeleteStorageGroup(1);
     }
@@ -291,22 +287,12 @@
       dataPartitionTable.deleteStorageGroup(storageGroup);
     }
 
-    DeleteTimeSeriesPlan deleteTimeSeriesPlan =
-        SchemaSyncManager.getInstance().isEnableSync()
-            ? SchemaSyncManager.getInstance()
-                .splitDeleteTimeseriesPlanByDevice(
-                    storageGroup.concatNode(MULTI_LEVEL_PATH_WILDCARD))
-            : null;
-
     deleteSchemaRegionsInStorageGroup(
         storageGroup, schemaPartitionTable.getSchemaRegionIdsByStorageGroup(storageGroup));
 
     for (Template template : templateManager.getTemplateMap().values()) {
       templateManager.unmarkStorageGroup(template, storageGroup.getFullPath());
     }
-    if (SchemaSyncManager.getInstance().isEnableSync()) {
-      SchemaSyncManager.getInstance().syncMetadataPlan(deleteTimeSeriesPlan);
-    }
 
     if (!config.isEnableMemControl()) {
       MemTableManager.getInstance().addOrDeleteStorageGroup(-1);
@@ -1370,4 +1356,45 @@
       return Collections.singletonList(syncService.getPipeSink(pipeSinkName));
     }
   }
+
+  public TSStatus createPipe(CreatePipeStatement createPipeStatement) {
+    try {
+      syncService.addPipe(createPipeStatement);
+    } catch (PipeException e) {
+      return RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+  }
+
+  public TSStatus startPipe(String pipeName) {
+    try {
+      syncService.startPipe(pipeName);
+    } catch (PipeException e) {
+      return RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+  }
+
+  public TSStatus stopPipe(String pipeName) {
+    try {
+      syncService.stopPipe(pipeName);
+    } catch (PipeException e) {
+      return RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+  }
+
+  public TSStatus dropPipe(String pipeName) {
+    try {
+      syncService.dropPipe(pipeName);
+    } catch (PipeException e) {
+      return RpcUtils.getStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR, e.getMessage());
+    }
+    return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+  }
+
+  public TShowPipeResp showPipe(String pipeName) {
+    List<TPipeInfo> pipeInfos = SyncService.getInstance().showPipe(pipeName);
+    return new TShowPipeResp().setPipeInfoList(pipeInfos).setStatus(StatusUtils.OK);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
index 6c3e308..90394b6 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
@@ -79,7 +79,6 @@
 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.sync.sender.manager.SchemaSyncManager;
 import org.apache.iotdb.db.utils.SchemaUtils;
 import org.apache.iotdb.external.api.ISeriesNumerLimiter;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
@@ -178,7 +177,6 @@
   // device -> DeviceMNode
   private LoadingCache<PartialPath, IMNode> mNodeCache;
   private TagManager tagManager;
-  private SchemaSyncManager syncManager = SchemaSyncManager.getInstance();
 
   private final ISeriesNumerLimiter seriesNumerLimiter;
 
@@ -646,9 +644,6 @@
         }
         plan.setTagOffset(offset);
         writeToMLog(plan);
-        if (syncManager.isEnableSync()) {
-          syncManager.syncMetadataPlan(plan);
-        }
       }
       if (offset != -1) {
         leafMNode.setOffset(offset);
@@ -792,9 +787,6 @@
         }
         plan.setTagOffsets(tagOffsets);
         writeToMLog(plan);
-        if (syncManager.isEnableSync()) {
-          syncManager.syncMetadataPlan(plan);
-        }
       }
       tagOffsets = plan.getTagOffsets();
       for (int i = 0; i < measurements.size(); i++) {
@@ -861,9 +853,6 @@
         }
         deleteTimeSeriesPlan.setDeletePathList(Collections.singletonList(p));
         writeToMLog(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/metadata/schemaregion/SchemaRegionSchemaFileImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
index 2194b41..051f788 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
@@ -76,7 +76,6 @@
 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.sync.sender.manager.SchemaSyncManager;
 import org.apache.iotdb.db.utils.SchemaUtils;
 import org.apache.iotdb.external.api.ISeriesNumerLimiter;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
@@ -172,7 +171,6 @@
   // device -> DeviceMNode
   private LoadingCache<PartialPath, IMNode> mNodeCache;
   private TagManager tagManager;
-  private SchemaSyncManager syncManager = SchemaSyncManager.getInstance();
 
   private final ISeriesNumerLimiter seriesNumerLimiter;
 
@@ -538,9 +536,6 @@
           }
           plan.setTagOffset(offset);
           logWriter.createTimeseries(plan);
-          if (syncManager.isEnableSync()) {
-            syncManager.syncMetadataPlan(plan);
-          }
         }
         if (offset != -1) {
           leafMNode.setOffset(offset);
@@ -708,9 +703,6 @@
           }
           plan.setTagOffsets(tagOffsets);
           logWriter.createAlignedTimeseries(plan);
-          if (syncManager.isEnableSync()) {
-            syncManager.syncMetadataPlan(plan);
-          }
         }
         tagOffsets = plan.getTagOffsets();
         for (int i = 0; i < measurements.size(); i++) {
@@ -783,9 +775,6 @@
         }
         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/mpp/common/header/ColumnHeaderConstant.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeaderConstant.java
index 55faa87..e642a45 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeaderConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/ColumnHeaderConstant.java
@@ -96,6 +96,14 @@
   public static final String COLUMN_PIPESINK_NAME = "name";
   public static final String COLUMN_PIPESINK_ATTRIBUTES = "attributes";
 
+  // column names for show pipe
+  public static final String COLUMN_PIPE_CREATE_TIME = "create time";
+  public static final String COLUMN_PIPE_NAME = "name";
+  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_MESSAGE = "message";
+
   public static final List<ColumnHeader> lastQueryColumnHeaders =
       ImmutableList.of(
           new ColumnHeader(COLUMN_TIMESERIES, TSDataType.TEXT),
@@ -235,4 +243,13 @@
           new ColumnHeader(COLUMN_PIPESINK_NAME, TSDataType.TEXT),
           new ColumnHeader(COLUMN_PIPESINK_TYPE, TSDataType.TEXT),
           new ColumnHeader(COLUMN_PIPESINK_ATTRIBUTES, TSDataType.TEXT));
+
+  public static final List<ColumnHeader> showPipeColumnHeaders =
+      ImmutableList.of(
+          new ColumnHeader(COLUMN_PIPE_CREATE_TIME, TSDataType.TEXT),
+          new ColumnHeader(COLUMN_PIPE_NAME, TSDataType.TEXT),
+          new ColumnHeader(COLUMN_PIPE_ROLE, TSDataType.TEXT),
+          new ColumnHeader(COLUMN_PIPE_REMOTE, TSDataType.TEXT),
+          new ColumnHeader(COLUMN_PIPE_STATUS, TSDataType.TEXT),
+          new ColumnHeader(COLUMN_PIPE_MESSAGE, TSDataType.TEXT));
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeaderFactory.java b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeaderFactory.java
index b079f3a..b6f9f5a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeaderFactory.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/common/header/DatasetHeaderFactory.java
@@ -120,4 +120,8 @@
   public static DatasetHeader getShowPipeSinkHeader() {
     return new DatasetHeader(ColumnHeaderConstant.showPipeSinkColumnHeaders, true);
   }
+
+  public static DatasetHeader getShowPipeHeader() {
+    return new DatasetHeader(ColumnHeaderConstant.showPipeColumnHeaders, true);
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeManager.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeManager.java
index 2742816..e14672e 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeManager.java
@@ -262,6 +262,7 @@
 
     @Override
     public void onFinish(ISinkHandle sinkHandle) {
+      logger.info("onFinish is invoked");
       removeFromMPPDataExchangeManager(sinkHandle);
       context.finished();
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeService.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeService.java
index 5dcd286..56df15a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeService.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/MPPDataExchangeService.java
@@ -98,7 +98,7 @@
           new ThriftServiceThread(
               processor,
               getID().getName(),
-              ThreadName.MPP_DATA_EXCHANGE_RPC_CLIENT.getName(),
+              ThreadName.MPP_DATA_EXCHANGE_RPC_PROCESSOR.getName(),
               getBindIP(),
               getBindPort(),
               config.getRpcMaxConcurrentClientNum(),
@@ -109,7 +109,7 @@
     } catch (RPCServiceException e) {
       throw new IllegalAccessException(e.getMessage());
     }
-    thriftServiceThread.setName(ThreadName.MPP_DATA_EXCHANGE_RPC_SERVER.getName());
+    thriftServiceThread.setName(ThreadName.MPP_DATA_EXCHANGE_RPC_SERVICE.getName());
     MetricService.getInstance()
         .getOrCreateAutoGauge(
             Metric.THRIFT_ACTIVE_THREADS.toString(),
@@ -117,7 +117,7 @@
             thriftServiceThread,
             AbstractThriftServiceThread::getActiveThreadCount,
             Tag.NAME.toString(),
-            ThreadName.MPP_DATA_EXCHANGE_RPC_SERVER.getName());
+            ThreadName.MPP_DATA_EXCHANGE_RPC_SERVICE.getName());
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SinkHandle.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SinkHandle.java
index 4cc0539..8d58327 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SinkHandle.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SinkHandle.java
@@ -279,7 +279,7 @@
     Pair<TsBlock, Long> pair = sequenceIdToTsBlock.get(sequenceId);
     if (pair == null || pair.left == null) {
       logger.error(
-          "The data block doesn't exist. Sequence ID is {}, remaining map is {}",
+          "The TsBlock doesn't exist. Sequence ID is {}, remaining map is {}",
           sequenceId,
           sequenceIdToTsBlock.entrySet());
       throw new IllegalStateException("The data block doesn't exist. Sequence ID: " + sequenceId);
@@ -307,6 +307,7 @@
         freedBytes += entry.getValue().right;
         bufferRetainedSizeInBytes -= entry.getValue().right;
         iterator.remove();
+        logger.info("ack TsBlock {}.", entry.getKey());
       }
     }
     if (isFinished()) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SourceHandle.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SourceHandle.java
index c0a353f..bd95291 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SourceHandle.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/exchange/SourceHandle.java
@@ -410,7 +410,12 @@
             }
             break;
           } catch (Throwable e) {
-            logger.error("failed to get data block, attempt times: {}", attempt, e);
+            logger.error(
+                "failed to get data block [{}, {}), attempt times: {}",
+                startSequenceId,
+                endSequenceId,
+                attempt,
+                e);
             if (attempt == MAX_ATTEMPT_TIMES) {
               synchronized (SourceHandle.this) {
                 bufferRetainedSizeInBytes -= reservedBytes;
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/constant/StatementType.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/constant/StatementType.java
index f344faa..b743458 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/constant/StatementType.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/constant/StatementType.java
@@ -147,5 +147,5 @@
   CREATE_PIPE,
   START_PIPE,
   STOP_PIPE,
-  DROP_PIPE
+  DROP_PIPE,
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java
index 75df8c7..931ea8a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/ConfigTaskVisitor.java
@@ -24,6 +24,7 @@
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.CreateTriggerTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.DeleteStorageGroupTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.DropFunctionTask;
+import org.apache.iotdb.db.mpp.plan.execution.config.metadata.DropTriggerTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.SetStorageGroupTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.SetTTLTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowClusterTask;
@@ -61,6 +62,7 @@
 import org.apache.iotdb.db.mpp.plan.statement.metadata.CreateTriggerStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.DeleteStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.DropFunctionStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.DropTriggerStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.SetStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.SetTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowClusterStatement;
@@ -194,15 +196,20 @@
     return new DropFunctionTask(dropFunctionStatement);
   }
 
+  @Override
+  public IConfigTask visitShowFunctions(
+      ShowFunctionsStatement showFunctionsStatement, TaskContext context) {
+    return new ShowFunctionsTask();
+  }
+
   public IConfigTask visitCreateTrigger(
       CreateTriggerStatement createTriggerStatement, TaskContext context) {
     return new CreateTriggerTask(createTriggerStatement);
   }
 
-  @Override
-  public IConfigTask visitShowFunctions(
-      ShowFunctionsStatement showFunctionsStatement, TaskContext context) {
-    return new ShowFunctionsTask();
+  public IConfigTask visitDropTrigger(
+      DropTriggerStatement dropTriggerStatement, TaskContext context) {
+    return new DropTriggerTask(dropTriggerStatement);
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java
index 76f2c6c..0f770e6 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/ClusterConfigTaskExecutor.java
@@ -34,6 +34,7 @@
 import org.apache.iotdb.confignode.rpc.thrift.TCreateTriggerReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDeleteStorageGroupsReq;
 import org.apache.iotdb.confignode.rpc.thrift.TDropFunctionReq;
+import org.apache.iotdb.confignode.rpc.thrift.TDropTriggerReq;
 import org.apache.iotdb.confignode.rpc.thrift.TGetTemplateResp;
 import org.apache.iotdb.confignode.rpc.thrift.TSetStorageGroupReq;
 import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp;
@@ -76,8 +77,13 @@
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.DropPipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.DropPipeStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.ShowPipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.ShowPipeStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.StartPipeStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.StopPipeStatement;
 import org.apache.iotdb.rpc.StatementExecutionException;
 import org.apache.iotdb.rpc.TSStatusCode;
 
@@ -251,6 +257,8 @@
   public SettableFuture<ConfigTaskResult> createTrigger(
       String triggerName,
       String className,
+      String jarPath,
+      boolean usingURI,
       TriggerEvent triggerEvent,
       TriggerType triggerType,
       PartialPath pathPattern) {
@@ -262,8 +270,27 @@
           client.createTrigger(new TCreateTriggerReq(triggerName, null, null));
 
       if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != executionStatus.getCode()) {
-        LOGGER.error(
-            "[{}] Failed to create trigger {} in config node.", executionStatus, triggerName);
+        LOGGER.error("[{}] Failed to create trigger {}.", executionStatus, triggerName);
+        future.setException(new IoTDBException(executionStatus.message, executionStatus.code));
+      } else {
+        future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS));
+      }
+    } catch (TException | IOException e) {
+      future.setException(e);
+    }
+    return future;
+  }
+
+  @Override
+  public SettableFuture<ConfigTaskResult> dropTrigger(String triggerName) {
+    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    try (ConfigNodeClient client =
+        CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.partitionRegionId)) {
+      // todo: implementation
+      final TSStatus executionStatus = client.dropTrigger(new TDropTriggerReq(triggerName));
+
+      if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != executionStatus.getCode()) {
+        LOGGER.error("[{}] Failed to drop trigger {}.", executionStatus, triggerName);
         future.setException(new IoTDBException(executionStatus.message, executionStatus.code));
       } else {
         future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS));
@@ -637,7 +664,7 @@
   }
 
   @Override
-  public SettableFuture<ConfigTaskResult> createPipe() {
+  public SettableFuture<ConfigTaskResult> createPipe(CreatePipeStatement createPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     future.setException(
         new IoTDBException(
@@ -658,7 +685,7 @@
   }
 
   @Override
-  public SettableFuture<ConfigTaskResult> dropPipe() {
+  public SettableFuture<ConfigTaskResult> dropPipe(DropPipeStatement dropPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     future.setException(
         new IoTDBException(
@@ -679,7 +706,7 @@
   }
 
   @Override
-  public SettableFuture<ConfigTaskResult> showPipe() {
+  public SettableFuture<ConfigTaskResult> showPipe(ShowPipeStatement showPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     future.setException(
         new IoTDBException(
@@ -700,7 +727,7 @@
   }
 
   @Override
-  public SettableFuture<ConfigTaskResult> startPipe() {
+  public SettableFuture<ConfigTaskResult> startPipe(StartPipeStatement startPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     future.setException(
         new IoTDBException(
@@ -710,7 +737,7 @@
   }
 
   @Override
-  public SettableFuture<ConfigTaskResult> stopPipe() {
+  public SettableFuture<ConfigTaskResult> stopPipe(StopPipeStatement stopPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     future.setException(
         new IoTDBException(
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/IConfigTaskExecutor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/IConfigTaskExecutor.java
index e35d224..861f422 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/IConfigTaskExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/IConfigTaskExecutor.java
@@ -39,8 +39,13 @@
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.DropPipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.DropPipeStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.ShowPipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.ShowPipeStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.StartPipeStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.StopPipeStatement;
 
 import com.google.common.util.concurrent.SettableFuture;
 
@@ -68,10 +73,14 @@
   SettableFuture<ConfigTaskResult> createTrigger(
       String triggerName,
       String className,
+      String jarPath,
+      boolean usingURI,
       TriggerEvent triggerEvent,
       TriggerType triggerType,
       PartialPath pathPattern);
 
+  SettableFuture<ConfigTaskResult> dropTrigger(String triggerName);
+
   SettableFuture<ConfigTaskResult> setTTL(SetTTLStatement setTTLStatement, String taskName);
 
   SettableFuture<ConfigTaskResult> merge(boolean onCluster);
@@ -115,13 +124,13 @@
 
   SettableFuture<ConfigTaskResult> showPipeSink(ShowPipeSinkStatement showPipeSinkStatement);
 
-  SettableFuture<ConfigTaskResult> dropPipe();
+  SettableFuture<ConfigTaskResult> dropPipe(DropPipeStatement dropPipeStatement);
 
-  SettableFuture<ConfigTaskResult> createPipe();
+  SettableFuture<ConfigTaskResult> createPipe(CreatePipeStatement createPipeStatement);
 
-  SettableFuture<ConfigTaskResult> showPipe();
+  SettableFuture<ConfigTaskResult> startPipe(StartPipeStatement startPipeStatement);
 
-  SettableFuture<ConfigTaskResult> startPipe();
+  SettableFuture<ConfigTaskResult> stopPipe(StopPipeStatement stopPipeStatement);
 
-  SettableFuture<ConfigTaskResult> stopPipe();
+  SettableFuture<ConfigTaskResult> showPipe(ShowPipeStatement showPipeStatement);
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/StandaloneConfigTaskExecutor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/StandaloneConfigTaskExecutor.java
index 2e85751..27aa838 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/StandaloneConfigTaskExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/executor/StandaloneConfigTaskExecutor.java
@@ -29,6 +29,7 @@
 import org.apache.iotdb.commons.trigger.enums.TriggerType;
 import org.apache.iotdb.commons.udf.service.UDFExecutableManager;
 import org.apache.iotdb.commons.udf.service.UDFRegistrationService;
+import org.apache.iotdb.confignode.rpc.thrift.TShowPipeResp;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupInfo;
 import org.apache.iotdb.confignode.rpc.thrift.TStorageGroupSchema;
 import org.apache.iotdb.db.localconfignode.LocalConfigNode;
@@ -38,6 +39,7 @@
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowStorageGroupTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.metadata.ShowTTLTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.sys.sync.ShowPipeSinkTask;
+import org.apache.iotdb.db.mpp.plan.execution.config.sys.sync.ShowPipeTask;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.CountStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.DeleteStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.SetStorageGroupStatement;
@@ -52,8 +54,13 @@
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowPathSetTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.template.ShowSchemaTemplateStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.DropPipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.DropPipeStatement;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.ShowPipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.ShowPipeStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.StartPipeStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.StopPipeStatement;
 import org.apache.iotdb.db.sync.sender.pipe.PipeSink;
 import org.apache.iotdb.rpc.StatementExecutionException;
 import org.apache.iotdb.rpc.TSStatusCode;
@@ -227,6 +234,8 @@
   public SettableFuture<ConfigTaskResult> createTrigger(
       String triggerName,
       String className,
+      String jarPath,
+      boolean usingURI,
       TriggerEvent triggerEvent,
       TriggerType triggerType,
       PartialPath pathPattern) {
@@ -245,6 +254,22 @@
   }
 
   @Override
+  public SettableFuture<ConfigTaskResult> dropTrigger(String triggerName) {
+    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    try {
+      // todo: implementation
+      future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS));
+    } catch (Exception e) {
+      final String message =
+          String.format("Failed to drop trigger %s, because %s.", triggerName, e.getMessage());
+      LOGGER.error(message, e);
+      future.setException(
+          new IoTDBException(message, TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()));
+    }
+    return future;
+  }
+
+  @Override
   public SettableFuture<ConfigTaskResult> setTTL(SetTTLStatement setTTLStatement, String taskName) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try {
@@ -449,16 +474,6 @@
   }
 
   @Override
-  public SettableFuture<ConfigTaskResult> createPipe() {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
-    future.setException(
-        new IoTDBException(
-            "Executing create pipe is not supported",
-            TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()));
-    return future;
-  }
-
-  @Override
   public SettableFuture<ConfigTaskResult> createPipeSink(
       CreatePipeSinkStatement createPipeSinkStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
@@ -496,42 +511,59 @@
   }
 
   @Override
-  public SettableFuture<ConfigTaskResult> dropPipe() {
+  public SettableFuture<ConfigTaskResult> createPipe(CreatePipeStatement createPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
-    future.setException(
-        new IoTDBException(
-            "Executing drop pipe is not supported",
-            TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()));
+    TSStatus tsStatus = LocalConfigNode.getInstance().createPipe(createPipeStatement);
+    if (tsStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS));
+    } else {
+      future.setException(new StatementExecutionException(tsStatus));
+    }
     return future;
   }
 
   @Override
-  public SettableFuture<ConfigTaskResult> showPipe() {
+  public SettableFuture<ConfigTaskResult> startPipe(StartPipeStatement startPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
-    future.setException(
-        new IoTDBException(
-            "Executing show pipe is not supported",
-            TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()));
+    TSStatus tsStatus = LocalConfigNode.getInstance().startPipe(startPipeStatement.getPipeName());
+    if (tsStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS));
+    } else {
+      future.setException(new StatementExecutionException(tsStatus));
+    }
     return future;
   }
 
   @Override
-  public SettableFuture<ConfigTaskResult> startPipe() {
+  public SettableFuture<ConfigTaskResult> stopPipe(StopPipeStatement stopPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
-    future.setException(
-        new IoTDBException(
-            "Executing Start pipe is not supported",
-            TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()));
+    TSStatus tsStatus = LocalConfigNode.getInstance().stopPipe(stopPipeStatement.getPipeName());
+    if (tsStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS));
+    } else {
+      future.setException(new StatementExecutionException(tsStatus));
+    }
     return future;
   }
 
   @Override
-  public SettableFuture<ConfigTaskResult> stopPipe() {
+  public SettableFuture<ConfigTaskResult> dropPipe(DropPipeStatement dropPipeStatement) {
     SettableFuture<ConfigTaskResult> future = SettableFuture.create();
-    future.setException(
-        new IoTDBException(
-            "Executing stop pipe is not supported",
-            TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()));
+    TSStatus tsStatus = LocalConfigNode.getInstance().dropPipe(dropPipeStatement.getPipeName());
+    if (tsStatus.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS));
+    } else {
+      future.setException(new StatementExecutionException(tsStatus));
+    }
+    return future;
+  }
+
+  @Override
+  public SettableFuture<ConfigTaskResult> showPipe(ShowPipeStatement showPipeStatement) {
+    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    TShowPipeResp showPipeResp =
+        LocalConfigNode.getInstance().showPipe(showPipeStatement.getPipeName());
+    ShowPipeTask.buildTSBlock(showPipeResp.getPipeInfoList(), future);
     return future;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/CreateTriggerTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/CreateTriggerTask.java
index b335f60..98d3085 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/CreateTriggerTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/CreateTriggerTask.java
@@ -34,6 +34,10 @@
 
   private final String className;
 
+  private final String jarPath;
+
+  private final boolean usingURI;
+
   private final TriggerEvent triggerEvent;
 
   private final TriggerType triggerType;
@@ -43,6 +47,8 @@
   public CreateTriggerTask(CreateTriggerStatement createTriggerStatement) {
     this.triggerName = createTriggerStatement.getTriggerName();
     this.className = createTriggerStatement.getClassName();
+    this.jarPath = createTriggerStatement.getJarPath();
+    this.usingURI = createTriggerStatement.isUsingURI();
     this.triggerEvent = createTriggerStatement.getTriggerEvent();
     this.triggerType = createTriggerStatement.getTriggerType();
     this.pathPattern = createTriggerStatement.getPathPattern();
@@ -52,6 +58,6 @@
   public ListenableFuture<ConfigTaskResult> execute(IConfigTaskExecutor configTaskExecutor)
       throws InterruptedException {
     return configTaskExecutor.createTrigger(
-        triggerName, className, triggerEvent, triggerType, pathPattern);
+        triggerName, className, jarPath, usingURI, triggerEvent, triggerType, pathPattern);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/DropTriggerTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/DropTriggerTask.java
new file mode 100644
index 0000000..e9a3dd8
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/DropTriggerTask.java
@@ -0,0 +1,41 @@
+/*
+ * 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.mpp.plan.execution.config.metadata;
+
+import org.apache.iotdb.db.mpp.plan.execution.config.ConfigTaskResult;
+import org.apache.iotdb.db.mpp.plan.execution.config.IConfigTask;
+import org.apache.iotdb.db.mpp.plan.execution.config.executor.IConfigTaskExecutor;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.DropTriggerStatement;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+public class DropTriggerTask implements IConfigTask {
+  private final String triggerName;
+
+  public DropTriggerTask(DropTriggerStatement dropTriggerStatement) {
+    this.triggerName = dropTriggerStatement.getTriggerName();
+  }
+
+  @Override
+  public ListenableFuture<ConfigTaskResult> execute(IConfigTaskExecutor configTaskExecutor)
+      throws InterruptedException {
+    return configTaskExecutor.dropTrigger(triggerName);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/CreatePipeTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/CreatePipeTask.java
index e01b159..9b22eac 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/CreatePipeTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/CreatePipeTask.java
@@ -28,7 +28,7 @@
 
 public class CreatePipeTask implements IConfigTask {
 
-  private CreatePipeStatement createPipeStatement;
+  private final CreatePipeStatement createPipeStatement;
 
   public CreatePipeTask(CreatePipeStatement createPipeStatement) {
     this.createPipeStatement = createPipeStatement;
@@ -37,6 +37,6 @@
   @Override
   public ListenableFuture<ConfigTaskResult> execute(IConfigTaskExecutor configTaskExecutor)
       throws InterruptedException {
-    return configTaskExecutor.createPipe();
+    return configTaskExecutor.createPipe(createPipeStatement);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/DropPipeTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/DropPipeTask.java
index b123692..8a878de 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/DropPipeTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/DropPipeTask.java
@@ -28,7 +28,7 @@
 
 public class DropPipeTask implements IConfigTask {
 
-  private DropPipeStatement dropPipeStatement;
+  private final DropPipeStatement dropPipeStatement;
 
   public DropPipeTask(DropPipeStatement dropPipeStatement) {
     this.dropPipeStatement = dropPipeStatement;
@@ -37,6 +37,6 @@
   @Override
   public ListenableFuture<ConfigTaskResult> execute(IConfigTaskExecutor configTaskExecutor)
       throws InterruptedException {
-    return configTaskExecutor.dropPipe();
+    return configTaskExecutor.dropPipe(dropPipeStatement);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/ShowPipeTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/ShowPipeTask.java
index 6ea5696..180cdd2 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/ShowPipeTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/ShowPipeTask.java
@@ -19,16 +19,30 @@
 
 package org.apache.iotdb.db.mpp.plan.execution.config.sys.sync;
 
+import org.apache.iotdb.confignode.rpc.thrift.TPipeInfo;
+import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
+import org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant;
+import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
+import org.apache.iotdb.db.mpp.common.header.DatasetHeaderFactory;
 import org.apache.iotdb.db.mpp.plan.execution.config.ConfigTaskResult;
 import org.apache.iotdb.db.mpp.plan.execution.config.IConfigTask;
 import org.apache.iotdb.db.mpp.plan.execution.config.executor.IConfigTaskExecutor;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.ShowPipeStatement;
+import org.apache.iotdb.db.qp.utils.DatetimeUtils;
+import org.apache.iotdb.rpc.TSStatusCode;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.block.TsBlockBuilder;
+import org.apache.iotdb.tsfile.utils.Binary;
 
 import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+
+import java.util.List;
+import java.util.stream.Collectors;
 
 public class ShowPipeTask implements IConfigTask {
 
-  private ShowPipeStatement showPipeStatement;
+  private final ShowPipeStatement showPipeStatement;
 
   public ShowPipeTask(ShowPipeStatement showPipeStatement) {
     this.showPipeStatement = showPipeStatement;
@@ -37,6 +51,29 @@
   @Override
   public ListenableFuture<ConfigTaskResult> execute(IConfigTaskExecutor configTaskExecutor)
       throws InterruptedException {
-    return configTaskExecutor.showPipe();
+    return configTaskExecutor.showPipe(showPipeStatement);
+  }
+
+  public static void buildTSBlock(
+      List<TPipeInfo> pipeInfoList, SettableFuture<ConfigTaskResult> future) {
+    List<TSDataType> outputDataTypes =
+        ColumnHeaderConstant.showPipeColumnHeaders.stream()
+            .map(ColumnHeader::getColumnType)
+            .collect(Collectors.toList());
+    TsBlockBuilder builder = new TsBlockBuilder(outputDataTypes);
+    for (TPipeInfo tPipeInfo : pipeInfoList) {
+      builder.getTimeColumnBuilder().writeLong(0L);
+      builder
+          .getColumnBuilder(0)
+          .writeBinary(new Binary(DatetimeUtils.convertLongToDate(tPipeInfo.getCreateTime())));
+      builder.getColumnBuilder(1).writeBinary(new Binary(tPipeInfo.getPipeName()));
+      builder.getColumnBuilder(2).writeBinary(new Binary(tPipeInfo.getRole()));
+      builder.getColumnBuilder(3).writeBinary(new Binary(tPipeInfo.getRemote()));
+      builder.getColumnBuilder(4).writeBinary(new Binary(tPipeInfo.getStatus()));
+      builder.getColumnBuilder(5).writeBinary(new Binary(tPipeInfo.getMessage()));
+      builder.declarePosition();
+    }
+    DatasetHeader datasetHeader = DatasetHeaderFactory.getShowPipeHeader();
+    future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS, builder.build(), datasetHeader));
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/StartPipeTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/StartPipeTask.java
index aa10217..03dc214 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/StartPipeTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/StartPipeTask.java
@@ -28,7 +28,7 @@
 
 public class StartPipeTask implements IConfigTask {
 
-  private StartPipeStatement startPipeStatement;
+  private final StartPipeStatement startPipeStatement;
 
   public StartPipeTask(StartPipeStatement startPipeStatement) {
     this.startPipeStatement = startPipeStatement;
@@ -37,6 +37,6 @@
   @Override
   public ListenableFuture<ConfigTaskResult> execute(IConfigTaskExecutor configTaskExecutor)
       throws InterruptedException {
-    return configTaskExecutor.startPipe();
+    return configTaskExecutor.startPipe(startPipeStatement);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/StopPipeTask.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/StopPipeTask.java
index 14ae9a1..e53b4f9 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/StopPipeTask.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/sys/sync/StopPipeTask.java
@@ -28,7 +28,7 @@
 
 public class StopPipeTask implements IConfigTask {
 
-  private StopPipeStatement stopPipeStatement;
+  private final StopPipeStatement stopPipeStatement;
 
   public StopPipeTask(StopPipeStatement stopPipeStatement) {
     this.stopPipeStatement = stopPipeStatement;
@@ -37,6 +37,6 @@
   @Override
   public ListenableFuture<ConfigTaskResult> execute(IConfigTaskExecutor configTaskExecutor)
       throws InterruptedException {
-    return configTaskExecutor.stopPipe();
+    return configTaskExecutor.stopPipe(stopPipeStatement);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
index f070b0d..143c6d3 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java
@@ -100,6 +100,7 @@
 import org.apache.iotdb.db.mpp.plan.statement.metadata.DeleteStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.DeleteTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.DropFunctionStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.DropTriggerStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.SetStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.SetTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowChildNodesStatement;
@@ -727,6 +728,19 @@
     if (ctx.triggerType() == null) {
       throw new SemanticException("Please specify trigger type: STATELESS or STATEFUL.");
     }
+    if (ctx.jarLocation() == null) {
+      throw new SemanticException("Please specify the location of jar.");
+    }
+    // parse jarPath
+    String jarPath;
+    boolean usingURI;
+    if (ctx.jarLocation().FILE() != null) {
+      usingURI = false;
+      jarPath = parseFilePath(ctx.jarLocation().fileName.getText());
+    } else {
+      usingURI = true;
+      jarPath = parseFilePath(ctx.jarLocation().uri().getText());
+    }
     Map<String, String> attributes = new HashMap<>();
     if (ctx.triggerAttributeClause() != null) {
       for (IoTDBSqlParser.TriggerAttributeContext triggerAttributeContext :
@@ -739,6 +753,8 @@
     return new CreateTriggerStatement(
         parseIdentifier(ctx.triggerName.getText()),
         parseStringLiteral(ctx.className.getText()),
+        jarPath,
+        usingURI,
         ctx.triggerEventClause().BEFORE() != null
             ? TriggerEvent.BEFORE_INSERT
             : TriggerEvent.AFTER_INSERT,
@@ -747,6 +763,11 @@
         attributes);
   }
 
+  @Override
+  public Statement visitDropTrigger(IoTDBSqlParser.DropTriggerContext ctx) {
+    return new DropTriggerStatement(parseIdentifier(ctx.triggerName.getText()));
+  }
+
   // Show Child Paths =====================================================================
   @Override
   public Statement visitShowChildPaths(IoTDBSqlParser.ShowChildPathsContext ctx) {
@@ -2802,6 +2823,8 @@
     }
     if (ctx.syncAttributeClauses() != null) {
       createPipeStatement.setPipeAttributes(parseSyncAttributeClauses(ctx.syncAttributeClauses()));
+    } else {
+      createPipeStatement.setPipeAttributes(new HashMap<>());
     }
     return createPipeStatement;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
index ebfbd94..de9ad46 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/StatementVisitor.java
@@ -43,6 +43,7 @@
 import org.apache.iotdb.db.mpp.plan.statement.metadata.DeleteStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.DeleteTimeSeriesStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.DropFunctionStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.DropTriggerStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.SetStorageGroupStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.SetTTLStatement;
 import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowChildNodesStatement;
@@ -180,6 +181,10 @@
     return visitStatement(createTriggerStatement, context);
   }
 
+  public R visitDropTrigger(DropTriggerStatement dropTriggerStatement, C context) {
+    return visitStatement(dropTriggerStatement, context);
+  }
+
   /** Data Manipulation Language (DML) */
 
   // Select Statement
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/CreateTriggerStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/CreateTriggerStatement.java
index 495e7ca..005d552 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/CreateTriggerStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/CreateTriggerStatement.java
@@ -23,8 +23,10 @@
 import org.apache.iotdb.commons.trigger.enums.TriggerEvent;
 import org.apache.iotdb.commons.trigger.enums.TriggerType;
 import org.apache.iotdb.db.mpp.plan.analyze.QueryType;
+import org.apache.iotdb.db.mpp.plan.constant.StatementType;
 import org.apache.iotdb.db.mpp.plan.statement.IConfigStatement;
 import org.apache.iotdb.db.mpp.plan.statement.Statement;
+import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
 
 import java.util.Collections;
 import java.util.List;
@@ -36,6 +38,11 @@
 
   private final String className;
 
+  private final String jarPath;
+
+  /** usingURI == true indicates that jarPath is a URI */
+  private final boolean usingURI;
+
   private final TriggerEvent triggerEvent;
 
   private final TriggerType triggerType;
@@ -47,12 +54,18 @@
   public CreateTriggerStatement(
       String triggerName,
       String className,
+      String jarPath,
+      boolean usingURI,
       TriggerEvent triggerEvent,
       TriggerType triggerType,
       PartialPath pathPattern,
       Map<String, String> attributes) {
+    super();
+    statementType = StatementType.CREATE_TRIGGER;
     this.triggerName = triggerName;
     this.className = className;
+    this.jarPath = jarPath;
+    this.usingURI = usingURI;
     this.triggerEvent = triggerEvent;
     this.triggerType = triggerType;
     this.pathPattern = pathPattern;
@@ -83,6 +96,19 @@
     return attributes;
   }
 
+  public String getJarPath() {
+    return jarPath;
+  }
+
+  public boolean isUsingURI() {
+    return usingURI;
+  }
+
+  @Override
+  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
+    return visitor.visitCreateTrigger(this, context);
+  }
+
   @Override
   public QueryType getQueryType() {
     return QueryType.WRITE;
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/DropTriggerStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/DropTriggerStatement.java
new file mode 100644
index 0000000..a48b9e9
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/DropTriggerStatement.java
@@ -0,0 +1,59 @@
+/*
+ * 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.mpp.plan.statement.metadata;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.mpp.plan.analyze.QueryType;
+import org.apache.iotdb.db.mpp.plan.constant.StatementType;
+import org.apache.iotdb.db.mpp.plan.statement.IConfigStatement;
+import org.apache.iotdb.db.mpp.plan.statement.Statement;
+import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
+
+import java.util.Collections;
+import java.util.List;
+
+public class DropTriggerStatement extends Statement implements IConfigStatement {
+  private final String triggerName;
+
+  public DropTriggerStatement(String triggerName) {
+    super();
+    statementType = StatementType.DROP_TRIGGER;
+    this.triggerName = triggerName;
+  }
+
+  public String getTriggerName() {
+    return triggerName;
+  }
+
+  @Override
+  public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
+    return visitor.visitDropTrigger(this, context);
+  }
+
+  @Override
+  public QueryType getQueryType() {
+    return QueryType.WRITE;
+  }
+
+  @Override
+  public List<? extends PartialPath> getPaths() {
+    return Collections.emptyList();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/sys/sync/CreatePipeStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/sys/sync/CreatePipeStatement.java
index e2763b9..a3d59b5 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/sys/sync/CreatePipeStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/sys/sync/CreatePipeStatement.java
@@ -20,13 +20,16 @@
 package org.apache.iotdb.db.mpp.plan.statement.sys.sync;
 
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.sync.SyncConstant;
 import org.apache.iotdb.db.mpp.plan.analyze.QueryType;
 import org.apache.iotdb.db.mpp.plan.constant.StatementType;
 import org.apache.iotdb.db.mpp.plan.statement.IConfigStatement;
 import org.apache.iotdb.db.mpp.plan.statement.Statement;
 import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor;
 
+import java.io.IOException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -87,4 +90,39 @@
   public <R, C> R accept(StatementVisitor<R, C> visitor, C context) {
     return visitor.visitCreatePipe(this, context);
   }
+
+  public static CreatePipeStatement parseString(String parsedString) throws IOException {
+    String[] split = parsedString.split(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    if (split.length < 4) {
+      throw new IOException("Parsing CreatePipePlan error. Attributes is less than expected.");
+    }
+    CreatePipeStatement statement = new CreatePipeStatement(StatementType.CREATE_PIPE);
+    statement.setPipeName(split[0]);
+    statement.setPipeSinkName(split[1]);
+    statement.setStartTime(Long.parseLong(split[2]));
+    int size = (Integer.parseInt(split[3]) << 1);
+    if (split.length != (size + 4)) {
+      throw new IOException("Parsing CreatePipePlan error. Attributes number is wrong.");
+    }
+    Map<String, String> attributes = new HashMap<>();
+    for (int i = 0; i < size; i += 2) {
+      attributes.put(split[i + 4], split[i + 5]);
+    }
+    statement.setPipeAttributes(attributes);
+    return statement;
+  }
+
+  @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(startTime).append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    builder.append(pipeAttributes.size()).append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    for (Map.Entry<String, String> entry : pipeAttributes.entrySet()) {
+      builder.append(entry.getKey()).append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+      builder.append(entry.getValue()).append(SyncConstant.PLAN_SERIALIZE_SPLIT_CHARACTER);
+    }
+    return builder.toString();
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/transformation/dag/column/binary/CompareBinaryColumnTransformer.java b/server/src/main/java/org/apache/iotdb/db/mpp/transformation/dag/column/binary/CompareBinaryColumnTransformer.java
index f184289..3055275 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/transformation/dag/column/binary/CompareBinaryColumnTransformer.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/transformation/dag/column/binary/CompareBinaryColumnTransformer.java
@@ -53,11 +53,13 @@
                       leftTransformer.getType().getBoolean(leftColumn, i),
                       rightTransformer.getType().getBoolean(rightColumn, i)));
         } else {
-          flag =
-              transform(
-                  compare(
-                      leftTransformer.getType().getDouble(leftColumn, i),
-                      rightTransformer.getType().getDouble(rightColumn, i)));
+          double left = leftTransformer.getType().getDouble(leftColumn, i);
+          double right = rightTransformer.getType().getDouble(rightColumn, i);
+          if (Double.isNaN(left) || Double.isNaN(right)) {
+            flag = false;
+          } else {
+            flag = transform(compare(left, right));
+          }
         }
         returnType.writeBoolean(builder, flag);
       } else {
diff --git a/server/src/main/java/org/apache/iotdb/db/protocol/mqtt/MPPPublishHandler.java b/server/src/main/java/org/apache/iotdb/db/protocol/mqtt/MPPPublishHandler.java
index 67785a6..a28bde4 100644
--- a/server/src/main/java/org/apache/iotdb/db/protocol/mqtt/MPPPublishHandler.java
+++ b/server/src/main/java/org/apache/iotdb/db/protocol/mqtt/MPPPublishHandler.java
@@ -49,6 +49,7 @@
 
 import java.time.ZoneId;
 import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
 
 /** PublishHandler handle the messages from MQTT clients. */
 public class MPPPublishHandler extends AbstractInterceptHandler {
@@ -57,7 +58,7 @@
 
   private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
   private final SessionManager SESSION_MANAGER = SessionManager.getInstance();
-  private long sessionId;
+  private final ConcurrentHashMap<String, Long> clientIdToSessionIdMap = new ConcurrentHashMap<>();
   private final PayloadFormatter payloadFormat;
   private final IPartitionFetcher partitionFetcher;
   private final ISchemaFetcher schemaFetcher;
@@ -75,32 +76,41 @@
 
   @Override
   public String getID() {
-    return "iotdb-mqtt-broker-listener-" + sessionId;
+    return "iotdb-mqtt-broker-listener";
   }
 
   @Override
   public void onConnect(InterceptConnectMessage msg) {
-    try {
-      BasicOpenSessionResp basicOpenSessionResp =
-          SESSION_MANAGER.openSession(
-              msg.getUsername(),
-              new String(msg.getPassword()),
-              ZoneId.systemDefault().toString(),
-              TSProtocolVersion.IOTDB_SERVICE_PROTOCOL_V3);
-      sessionId = basicOpenSessionResp.getSessionId();
-    } catch (TException e) {
-      throw new RuntimeException(e);
+    if (!clientIdToSessionIdMap.containsKey(msg.getClientID())) {
+      try {
+        BasicOpenSessionResp basicOpenSessionResp =
+            SESSION_MANAGER.openSession(
+                msg.getUsername(),
+                new String(msg.getPassword()),
+                ZoneId.systemDefault().toString(),
+                TSProtocolVersion.IOTDB_SERVICE_PROTOCOL_V3);
+        clientIdToSessionIdMap.put(msg.getClientID(), basicOpenSessionResp.getSessionId());
+      } catch (TException e) {
+        throw new RuntimeException(e);
+      }
     }
   }
 
   @Override
   public void onDisconnect(InterceptDisconnectMessage msg) {
-    SESSION_MANAGER.closeSession(sessionId);
+    Long sessionId = clientIdToSessionIdMap.remove(msg.getClientID());
+    if (null != sessionId) {
+      SESSION_MANAGER.closeSession(sessionId);
+    }
   }
 
   @Override
   public void onPublish(InterceptPublishMessage msg) {
     String clientId = msg.getClientID();
+    if (!clientIdToSessionIdMap.containsKey(clientId)) {
+      return;
+    }
+    long sessionId = clientIdToSessionIdMap.get(msg.getClientID());
     ByteBuf payload = msg.getPayload();
     String topic = msg.getTopicName();
     String username = msg.getUsername();
diff --git a/server/src/main/java/org/apache/iotdb/db/protocol/mqtt/PublishHandler.java b/server/src/main/java/org/apache/iotdb/db/protocol/mqtt/PublishHandler.java
index 0c20c32..7fb4ee8 100644
--- a/server/src/main/java/org/apache/iotdb/db/protocol/mqtt/PublishHandler.java
+++ b/server/src/main/java/org/apache/iotdb/db/protocol/mqtt/PublishHandler.java
@@ -38,14 +38,14 @@
 
 import java.time.ZoneId;
 import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
 
 /** PublishHandler handle the messages from MQTT clients. */
 public class PublishHandler extends AbstractInterceptHandler {
-
-  private final SessionManager SESSION_MANAGER = SessionManager.getInstance();
-  private long sessionId;
-
   private static final Logger LOG = LoggerFactory.getLogger(PublishHandler.class);
+  private final SessionManager SESSION_MANAGER = SessionManager.getInstance();
+
+  private final ConcurrentHashMap<String, Long> clientIdToSessionIdMap = new ConcurrentHashMap<>();
 
   private final PayloadFormatter payloadFormat;
 
@@ -59,32 +59,41 @@
 
   @Override
   public String getID() {
-    return "iotdb-mqtt-broker-listener-" + sessionId;
+    return "iotdb-mqtt-broker-listener";
   }
 
   @Override
   public void onConnect(InterceptConnectMessage msg) {
-    try {
-      BasicOpenSessionResp basicOpenSessionResp =
-          SESSION_MANAGER.openSession(
-              msg.getUsername(),
-              new String(msg.getPassword()),
-              ZoneId.systemDefault().toString(),
-              TSProtocolVersion.IOTDB_SERVICE_PROTOCOL_V3);
-      sessionId = basicOpenSessionResp.getSessionId();
-    } catch (TException e) {
-      throw new RuntimeException(e);
+    if (!clientIdToSessionIdMap.containsKey(msg.getClientID())) {
+      try {
+        BasicOpenSessionResp basicOpenSessionResp =
+            SESSION_MANAGER.openSession(
+                msg.getUsername(),
+                new String(msg.getPassword()),
+                ZoneId.systemDefault().toString(),
+                TSProtocolVersion.IOTDB_SERVICE_PROTOCOL_V3);
+        clientIdToSessionIdMap.put(msg.getClientID(), basicOpenSessionResp.getSessionId());
+      } catch (TException e) {
+        throw new RuntimeException(e);
+      }
     }
   }
 
   @Override
   public void onDisconnect(InterceptDisconnectMessage msg) {
-    SESSION_MANAGER.closeSession(sessionId);
+    Long sessionId = clientIdToSessionIdMap.remove(msg.getClientID());
+    if (null != sessionId) {
+      SESSION_MANAGER.closeSession(sessionId);
+    }
   }
 
   @Override
   public void onPublish(InterceptPublishMessage msg) {
     String clientId = msg.getClientID();
+    if (!clientIdToSessionIdMap.containsKey(clientId)) {
+      return;
+    }
+    long sessionId = clientIdToSessionIdMap.get(msg.getClientID());
     ByteBuf payload = msg.getPayload();
     String topic = msg.getTopicName();
     String username = msg.getUsername();
diff --git a/server/src/main/java/org/apache/iotdb/db/query/control/SessionManager.java b/server/src/main/java/org/apache/iotdb/db/query/control/SessionManager.java
index f1a1b25..c99a676 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/control/SessionManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/control/SessionManager.java
@@ -22,6 +22,7 @@
 import org.apache.iotdb.commons.auth.AuthException;
 import org.apache.iotdb.commons.auth.entity.PrivilegeType;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
+import org.apache.iotdb.commons.exception.IoTDBException;
 import org.apache.iotdb.db.auth.AuthorityChecker;
 import org.apache.iotdb.db.auth.AuthorizerManager;
 import org.apache.iotdb.db.conf.OperationType;
@@ -50,6 +51,7 @@
 import java.util.function.Consumer;
 
 import static org.apache.iotdb.db.utils.ErrorHandlingUtils.onNPEOrUnexpectedException;
+import static org.apache.iotdb.db.utils.ErrorHandlingUtils.onQueryException;
 
 public class SessionManager {
   private static final Logger LOGGER = LoggerFactory.getLogger(SessionManager.class);
@@ -340,7 +342,7 @@
   /** Check whether specific Session has the authorization to given plan. */
   public TSStatus checkAuthority(PhysicalPlan plan, long sessionId) {
     try {
-      if (!checkAuthorization(plan, sessionIdToUsername.get(sessionId))) {
+      if (!checkAuthorization(plan, getUsername(sessionId))) {
         return RpcUtils.getStatus(
             TSStatusCode.NO_PERMISSION_ERROR,
             "No permissions for this operation, please add privilege "
@@ -351,8 +353,8 @@
       LOGGER.warn("meet error while checking authorization.", e);
       return RpcUtils.getStatus(TSStatusCode.UNINITIALIZED_AUTH_ERROR, e.getMessage());
     } catch (Exception e) {
-      return onNPEOrUnexpectedException(
-          e, OperationType.CHECK_AUTHORITY, TSStatusCode.EXECUTE_STATEMENT_ERROR);
+      return onQueryException(
+          e, OperationType.CHECK_AUTHORITY.getName(), TSStatusCode.EXECUTE_STATEMENT_ERROR);
     }
     return null;
   }
@@ -371,7 +373,13 @@
   }
 
   public String getUsername(Long sessionId) {
-    return sessionIdToUsername.get(sessionId);
+    String username = sessionIdToUsername.get(sessionId);
+    if (username == null) {
+      throw new RuntimeException(
+          new IoTDBException(
+              "session expired, please re-login.", TSStatusCode.SESSION_EXPIRED.getStatusCode()));
+    }
+    return username;
   }
 
   public ZoneId getZoneId(Long sessionId) {
diff --git a/server/src/main/java/org/apache/iotdb/db/service/DataNodeInternalRPCService.java b/server/src/main/java/org/apache/iotdb/db/service/DataNodeInternalRPCService.java
index 0724796..1aed9d9 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/DataNodeInternalRPCService.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/DataNodeInternalRPCService.java
@@ -64,7 +64,7 @@
           new ThriftServiceThread(
               processor,
               getID().getName(),
-              ThreadName.INTERNAL_SERVICE_RPC_CLIENT.getName(),
+              ThreadName.DATANODE_INTERNAL_RPC_PROCESSOR.getName(),
               getBindIP(),
               getBindPort(),
               config.getRpcMaxConcurrentClientNum(),
@@ -75,7 +75,7 @@
     } catch (RPCServiceException e) {
       throw new IllegalAccessException(e.getMessage());
     }
-    thriftServiceThread.setName(ThreadName.INTERNAL_SERVICE_RPC_SERVER.getName());
+    thriftServiceThread.setName(ThreadName.DATANODE_INTERNAL_RPC_SERVICE.getName());
     MetricService.getInstance()
         .getOrCreateAutoGauge(
             Metric.THRIFT_ACTIVE_THREADS.toString(),
@@ -83,7 +83,7 @@
             thriftServiceThread,
             AbstractThriftServiceThread::getActiveThreadCount,
             Tag.NAME.toString(),
-            ThreadName.INTERNAL_SERVICE_RPC_SERVER.getName());
+            ThreadName.DATANODE_INTERNAL_RPC_SERVICE.getName());
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/service/InfluxDBRPCService.java b/server/src/main/java/org/apache/iotdb/db/service/InfluxDBRPCService.java
index 3930e27..5becc93 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/InfluxDBRPCService.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/InfluxDBRPCService.java
@@ -67,7 +67,7 @@
           new ThriftServiceThread(
               processor,
               getID().getName(),
-              ThreadName.INFLUXDB_CLIENT.getName(),
+              ThreadName.INFLUXDB_RPC_PROCESSOR.getName(),
               config.getRpcAddress(),
               config.getInfluxDBRpcPort(),
               config.getRpcMaxConcurrentClientNum(),
@@ -77,7 +77,7 @@
     } catch (RPCServiceException e) {
       throw new IllegalAccessException(e.getMessage());
     }
-    thriftServiceThread.setName(ThreadName.INFLUXDB_SERVICE.getName());
+    thriftServiceThread.setName(ThreadName.INFLUXDB_RPC_SERVICE.getName());
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/service/RPCService.java b/server/src/main/java/org/apache/iotdb/db/service/RPCService.java
index 9011945..47b77d0 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/RPCService.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/RPCService.java
@@ -38,7 +38,7 @@
 
 import java.lang.reflect.InvocationTargetException;
 
-/** A service to handle jdbc request from client. */
+/** A service to handle RPC request from client. */
 public class RPCService extends ThriftService implements RPCServiceMBean {
 
   private IClientRPCServiceWithHandler impl;
@@ -72,7 +72,7 @@
           new ThriftServiceThread(
               processor,
               getID().getName(),
-              ThreadName.RPC_CLIENT.getName(),
+              ThreadName.CLIENT_RPC_PROCESSOR.getName(),
               config.getRpcAddress(),
               config.getRpcPort(),
               config.getRpcMaxConcurrentClientNum(),
@@ -82,7 +82,7 @@
     } catch (RPCServiceException e) {
       throw new IllegalAccessException(e.getMessage());
     }
-    thriftServiceThread.setName(ThreadName.RPC_SERVICE.getName());
+    thriftServiceThread.setName(ThreadName.CLIENT_RPC_SERVICE.getName());
     MetricService.getInstance()
         .getOrCreateAutoGauge(
             Metric.THRIFT_ACTIVE_THREADS.toString(),
@@ -90,7 +90,7 @@
             thriftServiceThread,
             AbstractThriftServiceThread::getActiveThreadCount,
             Tag.NAME.toString(),
-            ThreadName.RPC_SERVICE.getName());
+            ThreadName.CLIENT_RPC_SERVICE.getName());
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java
index 71b83b7..bb026e8 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeInternalRPCServiceImpl.java
@@ -89,6 +89,7 @@
 import org.apache.iotdb.mpp.rpc.thrift.TCreateSchemaRegionReq;
 import org.apache.iotdb.mpp.rpc.thrift.TDisableDataNodeReq;
 import org.apache.iotdb.mpp.rpc.thrift.TDropFunctionRequest;
+import org.apache.iotdb.mpp.rpc.thrift.TDropTriggerInstanceReq;
 import org.apache.iotdb.mpp.rpc.thrift.TFetchFragmentInstanceStateReq;
 import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceStateResp;
 import org.apache.iotdb.mpp.rpc.thrift.THeartbeatReq;
@@ -750,6 +751,12 @@
   }
 
   @Override
+  public TSStatus dropTriggerInstance(TDropTriggerInstanceReq req) throws TException {
+    // todo: implementation
+    return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
+  }
+
+  @Override
   public TSStatus addRegionPeer(TMigrateRegionReq req) throws TException {
     TConsensusGroupId regionId = req.getRegionId();
     String toNodeIp = req.getToNode().getInternalEndPoint().getIp();
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/SyncService.java b/server/src/main/java/org/apache/iotdb/db/sync/SyncService.java
index 62a8699..285c584 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/SyncService.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/SyncService.java
@@ -28,9 +28,11 @@
 import org.apache.iotdb.commons.sync.SyncConstant;
 import org.apache.iotdb.commons.sync.SyncPathUtil;
 import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.confignode.rpc.thrift.TPipeInfo;
 import org.apache.iotdb.db.exception.sync.PipeException;
 import org.apache.iotdb.db.exception.sync.PipeSinkException;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeStatement;
 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.ShowPipePlan;
@@ -41,6 +43,7 @@
 import org.apache.iotdb.db.sync.externalpipe.ExtPipePluginManager;
 import org.apache.iotdb.db.sync.externalpipe.ExtPipePluginRegister;
 import org.apache.iotdb.db.sync.externalpipe.ExternalPipeStatus;
+import org.apache.iotdb.db.sync.sender.manager.ISyncManager;
 import org.apache.iotdb.db.sync.sender.pipe.ExternalPipeSink;
 import org.apache.iotdb.db.sync.sender.pipe.IoTDBPipeSink;
 import org.apache.iotdb.db.sync.sender.pipe.Pipe;
@@ -59,6 +62,7 @@
 import org.apache.iotdb.tsfile.read.common.RowRecord;
 import org.apache.iotdb.tsfile.utils.Binary;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -66,6 +70,7 @@
 import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
@@ -158,6 +163,7 @@
 
   // region Interfaces and Implementation of Pipe
 
+  // TODO(sync): delete this in new-standalone version
   public synchronized void addPipe(CreatePipePlan plan) throws PipeException {
     // check plan
     long currentTime = DatetimeUtils.currentTime();
@@ -197,6 +203,45 @@
     }
   }
 
+  public synchronized void addPipe(CreatePipeStatement statement) throws PipeException {
+    // check statement
+    long currentTime = DatetimeUtils.currentTime();
+    if (statement.getStartTime() > currentTime) {
+      throw new PipeException(
+          String.format(
+              "Start time %s is later than current time %s, this is not supported yet.",
+              DatetimeUtils.convertLongToDate(statement.getStartTime()),
+              DatetimeUtils.convertLongToDate(currentTime)));
+    }
+    // add pipe
+    TSStatus status = syncInfoFetcher.addPipe(statement, currentTime);
+    if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+      throw new PipeException(status.message);
+    }
+
+    PipeSink runningPipeSink = getPipeSink(statement.getPipeSinkName());
+    runningPipe = SyncPipeUtil.parseCreatePipePlanAsPipe(statement, runningPipeSink, currentTime);
+    if (runningPipe.getPipeSink().getType() == PipeSink.PipeSinkType.IoTDB) {
+      try {
+        senderManager = new SenderManager(runningPipe, (IoTDBPipeSink) runningPipeSink);
+      } catch (ClassCastException e) {
+        logger.error(
+            String.format(
+                "Cast Class to %s error when create pipe %s.",
+                IoTDBPipeSink.class.getName(), statement.getPipeName()),
+            e);
+        runningPipe = null;
+        throw new PipeException(
+            String.format(
+                "Wrong pipeSink type %s for create pipe %s",
+                runningPipeSink.getType(), runningPipeSink.getPipeSinkName()));
+      }
+    } else { // for external pipe
+      // == start ExternalPipeProcessor for send data to external pipe plugin
+      startExternalPipeManager(false);
+    }
+  }
+
   public synchronized void stopPipe(String pipeName) throws PipeException {
     checkRunningPipeExistAndName(pipeName);
     if (runningPipe.getStatus() == Pipe.PipeStatus.RUNNING) {
@@ -312,6 +357,40 @@
     }
   }
 
+  public List<TPipeInfo> showPipe(String pipeName) {
+    boolean showAll = StringUtils.isEmpty(pipeName);
+    List<TPipeInfo> list = new ArrayList<>();
+    // show pipe in sender
+    for (PipeInfo pipe : SyncService.getInstance().getAllPipeInfos()) {
+      if (showAll || pipeName.equals(pipe.getPipeName())) {
+        TPipeInfo tPipeInfo =
+            new TPipeInfo(
+                pipe.getCreateTime(),
+                pipe.getPipeName(),
+                SyncConstant.ROLE_SENDER,
+                pipe.getPipeSinkName(),
+                pipe.getStatus().name(),
+                "");
+        list.add(tPipeInfo);
+      }
+    }
+    // show pipe in receiver
+    for (TSyncIdentityInfo identityInfo : receiverManager.getAllTSyncIdentityInfos()) {
+      if (showAll || pipeName.equals(identityInfo.getPipeName())) {
+        TPipeInfo tPipeInfo =
+            new TPipeInfo(
+                identityInfo.getCreateTime(),
+                identityInfo.getPipeName(),
+                SyncConstant.ROLE_RECEIVER,
+                identityInfo.getAddress(),
+                Pipe.PipeStatus.RUNNING.name(),
+                "");
+        list.add(tPipeInfo);
+      }
+    }
+    return list;
+  }
+
   public void showPipe(ShowPipePlan plan, ListDataSet listDataSet) {
     boolean showAll = "".equals(plan.getPipeName());
     // show pipe in sender
@@ -523,6 +602,22 @@
     }
   }
 
+  public List<ISyncManager> getOrCreateSyncManager(String dataRegionId) {
+    // TODO(sync): maybe add cache to accelerate
+    List<ISyncManager> syncManagerList = new ArrayList<>();
+    if (runningPipe != null) {
+      syncManagerList.add(runningPipe.getOrCreateSyncManager(dataRegionId));
+    }
+    return syncManagerList;
+  }
+
+  /** This method will be called before deleting dataRegion */
+  public void deleteSyncManager(String dataRegionId) {
+    if (runningPipe != null) {
+      runningPipe.deleteSyncManager(dataRegionId);
+    }
+  }
+
   @TestOnly
   public SenderManager getSenderManager() {
     return senderManager;
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/common/ISyncInfoFetcher.java b/server/src/main/java/org/apache/iotdb/db/sync/common/ISyncInfoFetcher.java
index a419d5b..9e92d83 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/common/ISyncInfoFetcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/common/ISyncInfoFetcher.java
@@ -20,6 +20,7 @@
 
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeStatement;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipeSinkPlan;
 import org.apache.iotdb.db.sync.sender.pipe.PipeInfo;
@@ -45,9 +46,11 @@
 
   // region Interfaces of Pipe
 
-  // TODO: use CreatePipeNode as parameter
+  // TODO(sync): delete this in new-standalone version
   TSStatus addPipe(CreatePipePlan plan, long createTime);
 
+  TSStatus addPipe(CreatePipeStatement createPipeStatement, long createTime);
+
   TSStatus stopPipe(String pipeName);
 
   TSStatus startPipe(String pipeName);
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/common/LocalSyncInfoFetcher.java b/server/src/main/java/org/apache/iotdb/db/sync/common/LocalSyncInfoFetcher.java
index 7f10723..9ba3750 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/common/LocalSyncInfoFetcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/common/LocalSyncInfoFetcher.java
@@ -22,8 +22,9 @@
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.exception.sync.PipeException;
 import org.apache.iotdb.db.exception.sync.PipeSinkException;
+import org.apache.iotdb.db.mpp.plan.constant.StatementType;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeSinkStatement;
-import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeStatement;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipeSinkPlan;
 import org.apache.iotdb.db.sync.sender.pipe.PipeInfo;
@@ -103,9 +104,19 @@
   }
 
   @Override
+  public TSStatus addPipe(CreatePipeStatement createPipeStatement, long createTime) {
+    try {
+      syncInfo.addPipe(createPipeStatement, createTime);
+    } catch (PipeException | IOException e) {
+      RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR, e.getMessage());
+    }
+    return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
+  }
+
+  @Override
   public TSStatus stopPipe(String pipeName) {
     try {
-      syncInfo.operatePipe(pipeName, Operator.OperatorType.STOP_PIPE);
+      syncInfo.operatePipe(pipeName, StatementType.STOP_PIPE);
     } catch (PipeException | IOException e) {
       RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR, e.getMessage());
     }
@@ -115,7 +126,7 @@
   @Override
   public TSStatus startPipe(String pipeName) {
     try {
-      syncInfo.operatePipe(pipeName, Operator.OperatorType.START_PIPE);
+      syncInfo.operatePipe(pipeName, StatementType.START_PIPE);
     } catch (PipeException | IOException e) {
       RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR, e.getMessage());
     }
@@ -125,7 +136,7 @@
   @Override
   public TSStatus dropPipe(String pipeName) {
     try {
-      syncInfo.operatePipe(pipeName, Operator.OperatorType.DROP_PIPE);
+      syncInfo.operatePipe(pipeName, StatementType.DROP_PIPE);
     } catch (PipeException | IOException e) {
       RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR, e.getMessage());
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/common/SyncInfo.java b/server/src/main/java/org/apache/iotdb/db/sync/common/SyncInfo.java
index 218e099..560605b 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/common/SyncInfo.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/common/SyncInfo.java
@@ -22,8 +22,9 @@
 import org.apache.iotdb.commons.sync.SyncPathUtil;
 import org.apache.iotdb.db.exception.sync.PipeException;
 import org.apache.iotdb.db.exception.sync.PipeSinkException;
+import org.apache.iotdb.db.mpp.plan.constant.StatementType;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeSinkStatement;
-import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeStatement;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipeSinkPlan;
 import org.apache.iotdb.db.sync.common.persistence.SyncLogReader;
@@ -144,7 +145,7 @@
   // endregion
 
   // region Implement of Pipe
-
+  // TODO: delete this in new-standalone version
   public void addPipe(CreatePipePlan plan, long createTime) throws PipeException, IOException {
     // common check
     if (runningPipe != null && runningPipe.getStatus() != Pipe.PipeStatus.DROP) {
@@ -163,10 +164,32 @@
     syncLogWriter.addPipe(plan, createTime);
   }
 
-  public void operatePipe(String pipeName, Operator.OperatorType operatorType)
+  public void addPipe(CreatePipeStatement createPipeStatement, long createTime)
+      throws PipeException, IOException {
+    // common check
+    if (runningPipe != null && runningPipe.getStatus() != Pipe.PipeStatus.DROP) {
+      throw new PipeException(
+          String.format(
+              "Pipe %s is %s, please retry after drop it.",
+              runningPipe.getPipeName(), runningPipe.getStatus().name()));
+    }
+    if (!isPipeSinkExist(createPipeStatement.getPipeSinkName())) {
+      throw new PipeException(
+          String.format("Can not find pipeSink %s.", createPipeStatement.getPipeSinkName()));
+    }
+
+    PipeSink runningPipeSink = getPipeSink(createPipeStatement.getPipeSinkName());
+    runningPipe =
+        SyncPipeUtil.parseCreatePipePlanAsPipeInfo(
+            createPipeStatement, runningPipeSink, createTime);
+    pipes.add(runningPipe);
+    syncLogWriter.addPipe(createPipeStatement, createTime);
+  }
+
+  public void operatePipe(String pipeName, StatementType statementType)
       throws PipeException, IOException {
     checkIfPipeExistAndRunning(pipeName);
-    switch (operatorType) {
+    switch (statementType) {
       case START_PIPE:
         runningPipe.start();
         break;
@@ -177,9 +200,9 @@
         runningPipe.drop();
         break;
       default:
-        throw new PipeException("Unknown operatorType " + operatorType);
+        throw new PipeException("Unknown operatorType " + statementType);
     }
-    syncLogWriter.operatePipe(pipeName, operatorType);
+    syncLogWriter.operatePipe(pipeName, statementType);
   }
 
   public List<PipeInfo> getAllPipeInfos() {
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/common/persistence/SyncLogReader.java b/server/src/main/java/org/apache/iotdb/db/sync/common/persistence/SyncLogReader.java
index a9e91c0..344b4b8 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/common/persistence/SyncLogReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/common/persistence/SyncLogReader.java
@@ -24,7 +24,7 @@
 import org.apache.iotdb.commons.sync.SyncPathUtil;
 import org.apache.iotdb.db.mpp.plan.constant.StatementType;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeSinkStatement;
-import org.apache.iotdb.db.qp.physical.sys.CreatePipePlan;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeStatement;
 import org.apache.iotdb.db.sync.sender.pipe.PipeInfo;
 import org.apache.iotdb.db.sync.sender.pipe.PipeMessage;
 import org.apache.iotdb.db.sync.sender.pipe.PipeSink;
@@ -128,11 +128,11 @@
           case CREATE_PIPE:
             readLine = br.readLine();
             lineNumber += 1;
-            CreatePipePlan pipePlan = CreatePipePlan.parseString(readLine);
+            CreatePipeStatement createPipeStatement = CreatePipeStatement.parseString(readLine);
             runningPipe =
                 SyncPipeUtil.parseCreatePipePlanAsPipeInfo(
-                    pipePlan,
-                    pipeSinks.get(pipePlan.getPipeSinkName()),
+                    createPipeStatement,
+                    pipeSinks.get(createPipeStatement.getPipeSinkName()),
                     Long.parseLong(parseStrings[1]));
             pipes.add(runningPipe);
             break;
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/common/persistence/SyncLogWriter.java b/server/src/main/java/org/apache/iotdb/db/sync/common/persistence/SyncLogWriter.java
index 01e3068..556e85f 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/common/persistence/SyncLogWriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/common/persistence/SyncLogWriter.java
@@ -20,7 +20,9 @@
 
 import org.apache.iotdb.commons.sync.SyncConstant;
 import org.apache.iotdb.commons.sync.SyncPathUtil;
+import org.apache.iotdb.db.mpp.plan.constant.StatementType;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeStatement;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipeSinkPlan;
@@ -55,6 +57,7 @@
     }
   }
 
+  // TODO(sync): delete this in new-standalone version
   public synchronized void addPipeSink(CreatePipeSinkPlan plan) throws IOException {
     getBufferedWriter();
     pipeInfoWriter.write(Operator.OperatorType.CREATE_PIPESINK.name());
@@ -83,6 +86,7 @@
     pipeInfoWriter.flush();
   }
 
+  // TODO(sync): delete this in new-standalone version
   public synchronized void addPipe(CreatePipePlan plan, long pipeCreateTime) throws IOException {
     getBufferedWriter();
     pipeInfoWriter.write(Operator.OperatorType.CREATE_PIPE.name());
@@ -94,9 +98,20 @@
     pipeInfoWriter.flush();
   }
 
-  public synchronized void operatePipe(String pipeName, Operator.OperatorType type)
+  public synchronized void addPipe(CreatePipeStatement createPipeStatement, long pipeCreateTime)
       throws IOException {
     getBufferedWriter();
+    pipeInfoWriter.write(createPipeStatement.getType().name());
+    pipeInfoWriter.write(SyncConstant.SENDER_LOG_SPLIT_CHARACTER);
+    pipeInfoWriter.write(String.valueOf(pipeCreateTime));
+    pipeInfoWriter.newLine();
+    pipeInfoWriter.write(createPipeStatement.toString());
+    pipeInfoWriter.newLine();
+    pipeInfoWriter.flush();
+  }
+
+  public synchronized void operatePipe(String pipeName, StatementType type) throws IOException {
+    getBufferedWriter();
     pipeInfoWriter.write(type.name());
     pipeInfoWriter.write(SyncConstant.SENDER_LOG_SPLIT_CHARACTER);
     pipeInfoWriter.write(pipeName);
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
index c46e66a..7074687 100644
--- 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
@@ -18,21 +18,33 @@
  */
 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.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.exception.sync.PipeDataLoadException;
-import org.apache.iotdb.db.tools.TsFileSplitByPartitionTool;
-import org.apache.iotdb.db.utils.FileLoaderUtils;
+import org.apache.iotdb.db.qp.executor.PlanExecutor;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.sys.OperateFilePlan;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 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 static final Logger logger = LoggerFactory.getLogger(TsFileLoader.class);
+  private static PlanExecutor planExecutor;
 
-  private File tsFile;
+  static {
+    try {
+      planExecutor = new PlanExecutor();
+    } catch (QueryProcessException e) {
+      logger.error(e.getMessage());
+    }
+  }
+
+  private final File tsFile;
 
   public TsFileLoader(File tsFile) {
     this.tsFile = tsFile;
@@ -41,24 +53,14 @@
   @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()) {
-        TsFileSplitByPartitionTool.rewriteTsFile(tsFileResource, splitResources);
-        tsFileResource.writeLock();
-        tsFileResource.removeModFile();
-        tsFileResource.writeUnlock();
-      }
-
-      if (splitResources.isEmpty()) {
-        splitResources.add(tsFileResource);
-      }
-
-      for (TsFileResource resource : splitResources) {
-        StorageEngine.getInstance().loadNewTsFile(resource, false);
-      }
+      PhysicalPlan plan =
+          new OperateFilePlan(
+              tsFile,
+              Operator.OperatorType.LOAD_FILES,
+              true,
+              IoTDBDescriptor.getInstance().getConfig().getDefaultStorageGroupLevel(),
+              true);
+      planExecutor.processNonQuery(plan);
     } catch (Exception e) {
       throw new PipeDataLoadException(e.getMessage());
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/manager/ISyncManager.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/manager/ISyncManager.java
new file mode 100644
index 0000000..33bdf79
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/manager/ISyncManager.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.sync.sender.manager;
+
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.sync.sender.pipe.TsFilePipe;
+
+import java.io.File;
+import java.util.List;
+
+/**
+ * ISyncManager is designed for collect all history TsFiles(i.e. before the pipe start time, all
+ * tsfiles whose memtable is set to null.) and realtime TsFiles for registered {@link TsFilePipe}.
+ */
+public interface ISyncManager {
+  /** tsfile */
+  void syncRealTimeDeletion(Deletion deletion);
+
+  void syncRealTimeTsFile(File tsFile);
+
+  void syncRealTimeResource(File tsFile);
+
+  List<File> syncHistoryTsFile(long dataStartTime);
+
+  File createHardlink(File tsFile, long modsOffset);
+
+  void delete();
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/manager/LocalSyncManager.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/manager/LocalSyncManager.java
new file mode 100644
index 0000000..b366ab3
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/manager/LocalSyncManager.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.sync.sender.manager;
+
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.engine.storagegroup.DataRegion;
+import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
+import org.apache.iotdb.db.sync.sender.pipe.Pipe;
+import org.apache.iotdb.db.sync.sender.pipe.TsFilePipe;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+public class LocalSyncManager implements ISyncManager {
+
+  private TsFilePipe syncPipe;
+  private final DataRegion dataRegion;
+
+  public LocalSyncManager(DataRegion dataRegion, Pipe pipe) {
+    this.dataRegion = dataRegion;
+    this.syncPipe = (TsFilePipe) pipe;
+  }
+
+  /** tsfile */
+  @Override
+  public void syncRealTimeDeletion(Deletion deletion) {
+    syncPipe.collectRealTimeDeletion(deletion, dataRegion.getStorageGroupName());
+  }
+
+  @Override
+  public void syncRealTimeTsFile(File tsFile) {
+    syncPipe.collectRealTimeTsFile(tsFile);
+  }
+
+  @Override
+  public void syncRealTimeResource(File tsFile) {
+    syncPipe.collectRealTimeResource(tsFile);
+  }
+
+  @Override
+  public List<File> syncHistoryTsFile(long dataStartTime) {
+    return new ArrayList<>(this.dataRegion.collectHistoryTsFileForSync(this, dataStartTime));
+  }
+
+  @Override
+  public File createHardlink(File tsFile, long modsOffset) {
+    return syncPipe.createHistoryTsFileHardlink(tsFile, modsOffset);
+  }
+
+  @Override
+  public void delete() {
+    // TODO(sync): parse to delete operation and sync
+    // 1、get timeseries
+    // 2、get time partition
+    //    syncPipe.collectRealTimeDeletion();
+  }
+
+  public static List<PartialPath> splitPathPatternByDevice(PartialPath pathPattern)
+      throws MetadataException {
+    Set<PartialPath> devices = LocalSchemaProcessor.getInstance().getBelongedDevices(pathPattern);
+    List<PartialPath> resultPathPattern = new LinkedList<>();
+    for (PartialPath device : devices) {
+      pathPattern.alterPrefixPath(device).stream()
+          .filter(i -> !i.equals(device))
+          .forEach(resultPathPattern::add);
+    }
+    return resultPathPattern;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/manager/SchemaSyncManager.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/manager/SchemaSyncManager.java
deleted file mode 100644
index f04ea87..0000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/sender/manager/SchemaSyncManager.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.sync.sender.manager;
-
-import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.metadata.path.MeasurementPath;
-import org.apache.iotdb.db.metadata.schemaregion.ISchemaRegion;
-import org.apache.iotdb.db.metadata.schemaregion.SchemaEngine;
-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.DeleteTimeSeriesPlan;
-import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
-import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.sync.sender.pipe.TsFilePipe;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Set;
-
-import static org.apache.iotdb.db.metadata.MetadataConstant.ALL_RESULT_NODES;
-
-/**
- * SchemaSyncManager is designed to collect history metadata(i.e. all storage group and all
- * timeseries in IoTDB), and realtime metadata(i.e. create storage group, timeseries and delete
- * timeseries operations) for all registered {@linkplain TsFilePipe}.
- */
-public class SchemaSyncManager {
-
-  private static final Logger logger = LoggerFactory.getLogger(SchemaSyncManager.class);
-
-  private TsFilePipe syncPipe = null;
-
-  private static class SchemaSyncManagerHolder {
-
-    private SchemaSyncManagerHolder() {
-      // allowed to do nothing
-    }
-
-    private static final SchemaSyncManager INSTANCE = new SchemaSyncManager();
-  }
-
-  public static SchemaSyncManager getInstance() {
-    return SchemaSyncManagerHolder.INSTANCE;
-  }
-
-  public void registerSyncTask(TsFilePipe syncPipe) {
-    this.syncPipe = syncPipe;
-  }
-
-  public void deregisterSyncTask() {
-    this.syncPipe = null;
-  }
-
-  public boolean isEnableSync() {
-    return syncPipe != null;
-  }
-
-  public void syncMetadataPlan(PhysicalPlan plan) {
-    syncPipe.collectRealTimeMetaData(plan);
-  }
-
-  public void clear() {
-    this.syncPipe = null;
-  }
-
-  /** only support for SchemaRegion */
-  public List<PhysicalPlan> collectHistoryMetadata() {
-    List<PhysicalPlan> historyMetadata = new ArrayList<>();
-    List<SetStorageGroupPlan> storageGroupPlanList = getStorageGroupAsPlan();
-    for (SetStorageGroupPlan storageGroupPlan : storageGroupPlanList) {
-      historyMetadata.add(storageGroupPlan);
-    }
-
-    for (ISchemaRegion schemaRegion : SchemaEngine.getInstance().getAllSchemaRegions()) {
-      try {
-        for (MeasurementPath measurementPath :
-            schemaRegion.getMeasurementPaths(new PartialPath(ALL_RESULT_NODES), false)) {
-          if (measurementPath.isUnderAlignedEntity()) {
-            historyMetadata.add(
-                new CreateAlignedTimeSeriesPlan(
-                    measurementPath.getDevicePath(),
-                    measurementPath.getMeasurement(),
-                    (MeasurementSchema) measurementPath.getMeasurementSchema()));
-          } else {
-            historyMetadata.add(
-                new CreateTimeSeriesPlan(
-                    measurementPath, (MeasurementSchema) measurementPath.getMeasurementSchema()));
-          }
-        }
-      } catch (MetadataException e) {
-        logger.warn(
-            String.format(
-                "Collect history schema from schemaRegion: %s of sg %s error. Skip this schemaRegion.",
-                schemaRegion.getSchemaRegionId(), schemaRegion.getStorageGroupFullPath()));
-      }
-    }
-
-    return historyMetadata;
-  }
-
-  private List<SetStorageGroupPlan> getStorageGroupAsPlan() {
-    List<PartialPath> allStorageGroups = IoTDB.configManager.getAllStorageGroupPaths();
-    List<SetStorageGroupPlan> result = new LinkedList<>();
-    for (PartialPath sgPath : allStorageGroups) {
-      result.add(new SetStorageGroupPlan(sgPath));
-    }
-    return result;
-  }
-
-  public DeleteTimeSeriesPlan splitDeleteTimeseriesPlanByDevice(PartialPath pathPattern)
-      throws MetadataException {
-    return new DeleteTimeSeriesPlan(splitPathPatternByDevice(pathPattern));
-  }
-
-  public List<PartialPath> splitPathPatternByDevice(PartialPath pathPattern)
-      throws MetadataException {
-    Set<PartialPath> devices = IoTDB.schemaProcessor.getBelongedDevices(pathPattern);
-    List<PartialPath> resultPathPattern = new LinkedList<>();
-    for (PartialPath device : devices) {
-      pathPattern.alterPrefixPath(device).stream()
-          .filter(i -> !i.equals(device))
-          .forEach(resultPathPattern::add);
-    }
-    return resultPathPattern;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/manager/TsFileSyncManager.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/manager/TsFileSyncManager.java
deleted file mode 100644
index b688681..0000000
--- a/server/src/main/java/org/apache/iotdb/db/sync/sender/manager/TsFileSyncManager.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.sync.sender.manager;
-
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.engine.modification.Deletion;
-import org.apache.iotdb.db.engine.storagegroup.dataregion.StorageGroupManager;
-import org.apache.iotdb.db.sync.sender.pipe.TsFilePipe;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-/**
- * TsFileSyncManager is designed for collect all history TsFiles(i.e. before the pipe start time,
- * all tsfiles whose memtable is set to null.), and realtime tsfiles for registered {@linkplain
- * TsFilePipe}.
- */
-public class TsFileSyncManager {
-  private static final Logger logger = LoggerFactory.getLogger(TsFileSyncManager.class);
-
-  private TsFilePipe syncPipe;
-
-  /** singleton */
-  private TsFileSyncManager() {}
-
-  private static class TsFileSyncManagerHolder {
-    private static final TsFileSyncManager INSTANCE = new TsFileSyncManager();
-
-    private TsFileSyncManagerHolder() {}
-  }
-
-  public static TsFileSyncManager getInstance() {
-    return TsFileSyncManager.TsFileSyncManagerHolder.INSTANCE;
-  }
-
-  /** register */
-  public void registerSyncTask(TsFilePipe syncPipe) {
-    this.syncPipe = syncPipe;
-  }
-
-  public void deregisterSyncTask() {
-    this.syncPipe = null;
-  }
-
-  public boolean isEnableSync() {
-    return syncPipe != null;
-  }
-
-  public void clear() {
-    syncPipe = null;
-  }
-
-  /** tsfile */
-  public void collectRealTimeDeletion(Deletion deletion, String sgName) {
-    syncPipe.collectRealTimeDeletion(deletion, sgName);
-  }
-
-  public void collectRealTimeTsFile(File tsFile) {
-    syncPipe.collectRealTimeTsFile(tsFile);
-  }
-
-  public void collectRealTimeResource(File tsFile) {
-    syncPipe.collectRealTimeResource(tsFile);
-  }
-
-  public List<File> registerAndCollectHistoryTsFile(TsFilePipe syncPipe, long dataStartTime) {
-    registerSyncTask(syncPipe);
-
-    List<File> historyTsFiles = new ArrayList<>();
-    Iterator<Map.Entry<PartialPath, StorageGroupManager>> sgIterator =
-        StorageEngine.getInstance().getProcessorMap().entrySet().iterator();
-    while (sgIterator.hasNext()) {
-      historyTsFiles.addAll(
-          sgIterator.next().getValue().collectHistoryTsFileForSync(dataStartTime));
-    }
-
-    return historyTsFiles;
-  }
-
-  public File createHardlink(File tsFile, long modsOffset) {
-    return syncPipe.createHistoryTsFileHardlink(tsFile, modsOffset);
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/pipe/Pipe.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/pipe/Pipe.java
index f1afd93..773e591 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/sender/pipe/Pipe.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/pipe/Pipe.java
@@ -23,6 +23,7 @@
 import org.apache.iotdb.db.exception.sync.PipeException;
 import org.apache.iotdb.db.sync.SyncService;
 import org.apache.iotdb.db.sync.pipedata.PipeData;
+import org.apache.iotdb.db.sync.sender.manager.ISyncManager;
 import org.apache.iotdb.db.sync.transport.client.ISyncClient;
 
 /**
@@ -107,6 +108,17 @@
    */
   void commit();
 
+  /**
+   * Get {@linkplain ISyncManager} by dataRegionId. If ISyncManager does not exist, it will be
+   * created automatically.
+   *
+   * @param dataRegionId string of {@linkplain org.apache.iotdb.commons.consensus.DataRegionId}
+   * @return ISyncManager
+   */
+  ISyncManager getOrCreateSyncManager(String dataRegionId);
+
+  void deleteSyncManager(String dataRegionId);
+
   // a new pipe should be stop status
   enum PipeStatus {
     RUNNING,
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/pipe/TsFilePipe.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/pipe/TsFilePipe.java
index 55f0551..03d1bb8 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/sender/pipe/TsFilePipe.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/pipe/TsFilePipe.java
@@ -19,19 +19,20 @@
  */
 package org.apache.iotdb.db.sync.sender.pipe;
 
+import org.apache.iotdb.commons.consensus.DataRegionId;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.sync.SyncPathUtil;
+import org.apache.iotdb.db.engine.StorageEngineV2;
 import org.apache.iotdb.db.engine.modification.Deletion;
+import org.apache.iotdb.db.engine.storagegroup.DataRegion;
 import org.apache.iotdb.db.exception.sync.PipeException;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 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.sender.manager.SchemaSyncManager;
-import org.apache.iotdb.db.sync.sender.manager.TsFileSyncManager;
+import org.apache.iotdb.db.sync.sender.manager.ISyncManager;
+import org.apache.iotdb.db.sync.sender.manager.LocalSyncManager;
 import org.apache.iotdb.db.sync.sender.recovery.TsFilePipeLogger;
 
 import org.slf4j.Logger;
@@ -41,13 +42,15 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.locks.ReentrantLock;
 
 public class TsFilePipe implements Pipe {
   private static final Logger logger = LoggerFactory.getLogger(TsFilePipe.class);
-  private final SchemaSyncManager schemaSyncManager = SchemaSyncManager.getInstance();
-  private final TsFileSyncManager tsFileSyncManager = TsFileSyncManager.getInstance();
+  // <dataNodeId, ISyncManager>
+  private final Map<String, ISyncManager> syncManagerMap = new ConcurrentHashMap<>();
 
   private final long createTime;
   private final String name;
@@ -60,7 +63,6 @@
   private final TsFilePipeLogger pipeLog;
   private final ReentrantLock collectRealTimeDataLock;
 
-  private boolean isCollectingRealTimeData;
   private long maxSerialNumber;
 
   private PipeStatus status;
@@ -80,7 +82,6 @@
     this.pipeLog = new TsFilePipeLogger(this);
     this.collectRealTimeDataLock = new ReentrantLock();
 
-    this.isCollectingRealTimeData = false;
     this.maxSerialNumber = Math.max(0L, realTimeQueue.getLastMaxSerialNumber());
 
     this.status = PipeStatus.STOP;
@@ -95,42 +96,40 @@
       return;
     }
 
+    // init sync manager
+    List<DataRegion> dataRegions = StorageEngineV2.getInstance().getAllDataRegions();
+    for (DataRegion dataRegion : dataRegions) {
+      logger.info(
+          logFormat(
+              "init syncManager for %s-%s",
+              dataRegion.getStorageGroupName(), dataRegion.getDataRegionId()));
+      syncManagerMap.put(dataRegion.getDataRegionId(), new LocalSyncManager(dataRegion, this));
+    }
     try {
       if (!pipeLog.isCollectFinished()) {
         pipeLog.clear();
-        collectData();
+        collectHistoryData();
         pipeLog.finishCollect();
       }
-      if (!isCollectingRealTimeData) {
-        registerMetadata();
-        registerTsFile();
-        isCollectingRealTimeData = true;
-      }
 
       status = PipeStatus.RUNNING;
     } catch (IOException e) {
       logger.error(
-          String.format(
-              "Clear pipe dir %s error.", SyncPathUtil.getSenderPipeDir(name, createTime)),
+          logFormat("Clear pipe dir %s error.", SyncPathUtil.getSenderPipeDir(name, createTime)),
           e);
       throw new PipeException("Start error, can not clear pipe log.");
     }
   }
 
   /** collect data * */
-  private void collectData() {
-    registerMetadata();
-    List<PhysicalPlan> historyMetadata = collectHistoryMetadata();
-    List<File> historyTsFiles = registerAndCollectHistoryTsFile();
-    isCollectingRealTimeData = true;
-
-    // get all history data
-    int historyMetadataSize = historyMetadata.size();
-    int historyTsFilesSize = historyTsFiles.size();
-    for (int i = 0; i < historyMetadataSize; i++) {
-      long serialNumber = 1 - historyTsFilesSize - historyMetadataSize + i;
-      historyQueue.offer(new SchemaPipeData(historyMetadata.get(i), serialNumber));
+  private void collectHistoryData() {
+    // collect history TsFile
+    List<File> historyTsFiles = new ArrayList<>();
+    for (ISyncManager syncManager : syncManagerMap.values()) {
+      historyTsFiles.addAll(syncManager.syncHistoryTsFile(dataStartTime));
     }
+    // put history data into PipeDataQueue
+    int historyTsFilesSize = historyTsFiles.size();
     for (int i = 0; i < historyTsFilesSize; i++) {
       long serialNumber = 1 - historyTsFilesSize + i;
       File tsFile = historyTsFiles.get(i);
@@ -138,41 +137,6 @@
     }
   }
 
-  private void registerMetadata() {
-    schemaSyncManager.registerSyncTask(this);
-  }
-
-  private void deregisterMetadata() {
-    schemaSyncManager.deregisterSyncTask();
-  }
-
-  private List<PhysicalPlan> collectHistoryMetadata() {
-    return schemaSyncManager.collectHistoryMetadata();
-  }
-
-  public void collectRealTimeMetaData(PhysicalPlan plan) {
-    collectRealTimeDataLock.lock();
-    try {
-      maxSerialNumber += 1L;
-      PipeData metaData = new SchemaPipeData(plan, maxSerialNumber);
-      realTimeQueue.offer(metaData);
-    } finally {
-      collectRealTimeDataLock.unlock();
-    }
-  }
-
-  private void registerTsFile() {
-    tsFileSyncManager.registerSyncTask(this);
-  }
-
-  private void deregisterTsFile() {
-    tsFileSyncManager.deregisterSyncTask();
-  }
-
-  private List<File> registerAndCollectHistoryTsFile() {
-    return tsFileSyncManager.registerAndCollectHistoryTsFile(this, dataStartTime);
-  }
-
   public File createHistoryTsFileHardlink(File tsFile, long modsOffset) {
     collectRealTimeDataLock.lock(); // synchronize the pipeLog.isHardlinkExist
     try {
@@ -182,8 +146,7 @@
 
       return pipeLog.createTsFileAndModsHardlink(tsFile, modsOffset);
     } catch (IOException e) {
-      logger.error(
-          String.format("Create hardlink for history tsfile %s error.", tsFile.getPath()), e);
+      logger.error(logFormat("Create hardlink for history tsfile %s error.", tsFile.getPath()), e);
       return null;
     } finally {
       collectRealTimeDataLock.unlock();
@@ -197,8 +160,7 @@
         return;
       }
 
-      for (PartialPath deletePath :
-          schemaSyncManager.splitPathPatternByDevice(deletion.getPath())) {
+      for (PartialPath deletePath : LocalSyncManager.splitPathPatternByDevice(deletion.getPath())) {
         Deletion splitDeletion =
             new Deletion(
                 deletePath,
@@ -210,7 +172,7 @@
         realTimeQueue.offer(deletionData);
       }
     } catch (MetadataException e) {
-      logger.warn(String.format("Collect deletion %s error.", deletion), e);
+      logger.warn(logFormat("Collect deletion %s error.", deletion), e);
     } finally {
       collectRealTimeDataLock.unlock();
     }
@@ -230,7 +192,7 @@
       realTimeQueue.offer(tsFileData);
     } catch (IOException e) {
       logger.warn(
-          String.format(
+          logFormat(
               "Create Hardlink tsfile %s on disk error, serial number is %d.",
               tsFile.getPath(), maxSerialNumber),
           e);
@@ -243,13 +205,14 @@
     try {
       pipeLog.createTsFileResourceHardlink(tsFile);
     } catch (IOException e) {
-      logger.warn(String.format("Record tsfile resource %s on disk error.", tsFile.getPath()), e);
+      logger.warn(logFormat("Record tsfile resource %s on disk error.", tsFile.getPath()), e);
     }
   }
 
   /** transport data * */
   @Override
   public PipeData take() throws InterruptedException {
+    // TODO:should judge isCollectingRealTimeData here
     if (!historyQueue.isEmpty()) {
       return historyQueue.take();
     }
@@ -275,6 +238,23 @@
     realTimeQueue.commit();
   }
 
+  @Override
+  public ISyncManager getOrCreateSyncManager(String dataRegionId) {
+    return syncManagerMap.computeIfAbsent(
+        dataRegionId,
+        id ->
+            new LocalSyncManager(
+                StorageEngineV2.getInstance().getDataRegion(new DataRegionId(Integer.parseInt(id))),
+                this));
+  }
+
+  @Override
+  public void deleteSyncManager(String dataRegionId) {
+    if (syncManagerMap.containsKey(dataRegionId)) {
+      syncManagerMap.remove(dataRegionId).delete();
+    }
+  }
+
   public void commit(long serialNumber) {
     if (!historyQueue.isEmpty()) {
       historyQueue.commit(serialNumber);
@@ -290,13 +270,6 @@
       throw new PipeException(
           String.format("Can not stop pipe %s, because the pipe is drop.", name));
     }
-
-    if (!isCollectingRealTimeData) {
-      registerMetadata();
-      registerTsFile();
-      isCollectingRealTimeData = true;
-    }
-
     status = PipeStatus.STOP;
   }
 
@@ -311,29 +284,24 @@
   }
 
   private void clear() {
-    deregisterMetadata();
-    deregisterTsFile();
-    isCollectingRealTimeData = false;
-
     try {
       historyQueue.clear();
       realTimeQueue.clear();
       pipeLog.clear();
     } catch (IOException e) {
-      logger.warn(String.format("Clear pipe %s %d error.", name, createTime), e);
+      logger.warn(logFormat("Clear pipe %s %d error.", name, createTime), e);
     }
   }
 
+  private String logFormat(String format, Object... arguments) {
+    return String.format(String.format("[%s-%s] ", this.name, this.createTime) + format, arguments);
+  }
+
   @Override
   public void close() throws PipeException {
     if (status == PipeStatus.DROP) {
       return;
     }
-
-    deregisterMetadata();
-    deregisterTsFile();
-    isCollectingRealTimeData = false;
-
     historyQueue.close();
     realTimeQueue.close();
   }
@@ -374,8 +342,6 @@
         + syncDelOp
         + ", pipeLog="
         + pipeLog
-        + ", isCollectingRealTimeData="
-        + isCollectingRealTimeData
         + ", maxSerialNumber="
         + maxSerialNumber
         + ", status="
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/transport/client/SenderManager.java b/server/src/main/java/org/apache/iotdb/db/sync/transport/client/SenderManager.java
index 1979460..3705e5b 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/transport/client/SenderManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/transport/client/SenderManager.java
@@ -96,7 +96,6 @@
                       PipeMessage.MsgType.WARN,
                       String.format(
                           "Transfer piepdata %s error, skip it.", pipeData.getSerialNumber()));
-              continue;
             }
             pipe.commit();
           }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/ErrorHandlingUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/ErrorHandlingUtils.java
index 4b90e70..f45a09f 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/ErrorHandlingUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/ErrorHandlingUtils.java
@@ -76,7 +76,7 @@
     return e;
   }
 
-  public static TSStatus onQueryException(Exception e, String operation) {
+  public static TSStatus onQueryException(Exception e, String operation, TSStatusCode statusCode) {
     TSStatus status = tryCatchQueryException(e);
     if (status != null) {
       // ignore logging sg not ready exception
@@ -86,10 +86,14 @@
       }
       return status;
     } else {
-      return onNPEOrUnexpectedException(e, operation, TSStatusCode.INTERNAL_SERVER_ERROR);
+      return onNPEOrUnexpectedException(e, operation, statusCode);
     }
   }
 
+  public static TSStatus onQueryException(Exception e, String operation) {
+    return onQueryException(e, operation, TSStatusCode.INTERNAL_SERVER_ERROR);
+  }
+
   public static TSStatus onQueryException(Exception e, OperationType operation) {
     return onQueryException(e, operation.getName());
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/sync/SyncPipeUtil.java b/server/src/main/java/org/apache/iotdb/db/utils/sync/SyncPipeUtil.java
index 145130c..704600a 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/sync/SyncPipeUtil.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/sync/SyncPipeUtil.java
@@ -21,6 +21,7 @@
 import org.apache.iotdb.db.exception.sync.PipeException;
 import org.apache.iotdb.db.exception.sync.PipeSinkException;
 import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeSinkStatement;
+import org.apache.iotdb.db.mpp.plan.statement.sys.sync.CreatePipeStatement;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipeSinkPlan;
 import org.apache.iotdb.db.sync.sender.pipe.Pipe;
@@ -30,6 +31,8 @@
 import org.apache.iotdb.db.sync.sender.pipe.TsFilePipeInfo;
 import org.apache.iotdb.tsfile.utils.Pair;
 
+import java.util.Map;
+
 public class SyncPipeUtil {
 
   // TODO(sync): delete this in new-standalone version
@@ -61,6 +64,7 @@
     return pipeSink;
   }
 
+  // TODO(sync): delete this in new-standalone version
   public static Pipe parseCreatePipePlanAsPipe(
       CreatePipePlan plan, PipeSink pipeSink, long pipeCreateTime) throws PipeException {
     boolean syncDelOp = true;
@@ -77,6 +81,28 @@
         pipeCreateTime, plan.getPipeName(), pipeSink, plan.getDataStartTimestamp(), syncDelOp);
   }
 
+  public static Pipe parseCreatePipePlanAsPipe(
+      CreatePipeStatement createPipeStatement, PipeSink pipeSink, long pipeCreateTime)
+      throws PipeException {
+    boolean syncDelOp = true;
+    for (Map.Entry<String, String> entry : createPipeStatement.getPipeAttributes().entrySet()) {
+      String attributeKey = entry.getKey().toLowerCase();
+      if ("syncdelop".equals(attributeKey)) {
+        syncDelOp = Boolean.parseBoolean(entry.getValue());
+      } else {
+        throw new PipeException(String.format("Can not recognition attribute %s", entry.getKey()));
+      }
+    }
+
+    return new TsFilePipe(
+        pipeCreateTime,
+        createPipeStatement.getPipeName(),
+        pipeSink,
+        createPipeStatement.getStartTime(),
+        syncDelOp);
+  }
+
+  // TODO(sync): delete this in new-standalone version
   public static PipeInfo parseCreatePipePlanAsPipeInfo(
       CreatePipePlan plan, PipeSink pipeSink, long pipeCreateTime) throws PipeException {
     boolean syncDelOp = true;
@@ -97,6 +123,27 @@
         syncDelOp);
   }
 
+  public static PipeInfo parseCreatePipePlanAsPipeInfo(
+      CreatePipeStatement createPipeStatement, PipeSink pipeSink, long pipeCreateTime)
+      throws PipeException {
+    boolean syncDelOp = true;
+    for (Map.Entry<String, String> entry : createPipeStatement.getPipeAttributes().entrySet()) {
+      String attributeKey = entry.getKey().toLowerCase();
+      if ("syncdelop".equals(attributeKey)) {
+        syncDelOp = Boolean.parseBoolean(entry.getValue());
+      } else {
+        throw new PipeException(String.format("Can not recognition attribute %s", entry.getKey()));
+      }
+    }
+
+    return new TsFilePipeInfo(
+        createPipeStatement.getPipeName(),
+        pipeSink.getPipeSinkName(),
+        pipeCreateTime,
+        createPipeStatement.getStartTime(),
+        syncDelOp);
+  }
+
   /** parse PipeInfo ass Pipe, ignore status */
   public static Pipe parsePipeInfoAsPipe(PipeInfo pipeInfo, PipeSink pipeSink)
       throws PipeException {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
index b7c8ed1..5b102d4 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
@@ -71,7 +71,8 @@
 
   private TsFileProcessor processor;
   private String storageGroup = "root.vehicle";
-  private StorageGroupInfo sgInfo = new StorageGroupInfo(null);
+  private final String systemDir = TestConstant.OUTPUT_DATA_DIR.concat("info");
+  private StorageGroupInfo sgInfo;
   private String filePath = TestConstant.getTestTsFilePath("root.vehicle", 0, 0, 0);
   private String deviceId = "root.vehicle.d0";
   private String measurementId = "s0";
@@ -82,12 +83,13 @@
   private static Logger logger = LoggerFactory.getLogger(TsFileProcessorTest.class);
 
   @Before
-  public void setUp() {
+  public void setUp() throws Exception {
     File file = new File(filePath);
     if (!file.getParentFile().exists()) {
       Assert.assertTrue(file.getParentFile().mkdirs());
     }
     EnvironmentUtils.envSetUp();
+    sgInfo = new StorageGroupInfo(new DataRegionTest.DummyDataRegion(systemDir, storageGroup));
     MetadataManagerHelper.initMetadata();
     context = EnvironmentUtils.TEST_QUERY_CONTEXT;
   }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/trigger_example/Counter.java b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/trigger_example/Counter.java
index 3c86592..77145e9 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/trigger_example/Counter.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/trigger_example/Counter.java
@@ -21,7 +21,7 @@
 
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.engine.trigger.api.Trigger;
-import org.apache.iotdb.db.engine.trigger.api.TriggerAttributes;
+import org.apache.iotdb.trigger.api.TriggerAttributes;
 import org.apache.iotdb.tsfile.utils.Binary;
 
 public class Counter implements Trigger {
diff --git a/server/src/test/java/org/apache/iotdb/db/protocol/mqtt/PublishHandlerTest.java b/server/src/test/java/org/apache/iotdb/db/protocol/mqtt/PublishHandlerTest.java
index 3ddc0b0..f1b1a20 100644
--- a/server/src/test/java/org/apache/iotdb/db/protocol/mqtt/PublishHandlerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/protocol/mqtt/PublishHandlerTest.java
@@ -63,6 +63,7 @@
   public void onPublish() throws ClassNotFoundException {
     PayloadFormatter payloadFormat = PayloadFormatManager.getPayloadFormat("json");
     PublishHandler handler = new PublishHandler(payloadFormat);
+    String clientId = "clientId";
 
     String payload =
         "{\n"
@@ -77,7 +78,7 @@
     // connect
     MqttConnectPayload mqttConnectPayload =
         new MqttConnectPayload(
-            null,
+            clientId,
             null,
             "test".getBytes(StandardCharsets.UTF_8),
             "root",
@@ -92,12 +93,12 @@
     MqttFixedHeader fixedHeader =
         new MqttFixedHeader(MqttMessageType.PUBLISH, false, MqttQoS.AT_LEAST_ONCE, false, 1);
     MqttPublishMessage publishMessage = new MqttPublishMessage(fixedHeader, variableHeader, buf);
-    InterceptPublishMessage message = new InterceptPublishMessage(publishMessage, null, null);
+    InterceptPublishMessage message = new InterceptPublishMessage(publishMessage, clientId, null);
     handler.onPublish(message);
 
     // disconnect
     InterceptDisconnectMessage interceptDisconnectMessage =
-        new InterceptDisconnectMessage(null, null);
+        new InterceptDisconnectMessage(clientId, null);
     handler.onDisconnect(interceptDisconnectMessage);
 
     String[] retArray = new String[] {"1586076045524,0.530635,"};
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/receiver/manager/SyncInfoTest.java b/server/src/test/java/org/apache/iotdb/db/sync/receiver/manager/SyncInfoTest.java
index 6ea46e9..7ac36ca 100644
--- a/server/src/test/java/org/apache/iotdb/db/sync/receiver/manager/SyncInfoTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/sync/receiver/manager/SyncInfoTest.java
@@ -20,7 +20,7 @@
 
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.sync.PipeException;
-import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.mpp.plan.constant.StatementType;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipeSinkPlan;
 import org.apache.iotdb.db.sync.common.SyncInfo;
@@ -72,10 +72,10 @@
       } catch (PipeException e) {
         // throw exception because only one pipe is allowed now
       }
-      syncInfo.operatePipe(pipe1, Operator.OperatorType.DROP_PIPE);
+      syncInfo.operatePipe(pipe1, StatementType.DROP_PIPE);
       syncInfo.addPipe(new CreatePipePlan(pipe2, "demo"), createdTime2);
-      syncInfo.operatePipe(pipe2, Operator.OperatorType.STOP_PIPE);
-      syncInfo.operatePipe(pipe2, Operator.OperatorType.START_PIPE);
+      syncInfo.operatePipe(pipe2, StatementType.STOP_PIPE);
+      syncInfo.operatePipe(pipe2, StatementType.START_PIPE);
       Assert.assertEquals(2, syncInfo.getAllPipeInfos().size());
       Assert.assertEquals(1, syncInfo.getAllPipeSink().size());
       PipeMessage info = new PipeMessage(PipeMessage.MsgType.INFO, "info");
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/receiver/recovery/SyncLogTest.java b/server/src/test/java/org/apache/iotdb/db/sync/receiver/recovery/SyncLogTest.java
index 26919b9..d072fe1 100644
--- a/server/src/test/java/org/apache/iotdb/db/sync/receiver/recovery/SyncLogTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/sync/receiver/recovery/SyncLogTest.java
@@ -20,7 +20,7 @@
 
 import org.apache.iotdb.commons.sync.SyncPathUtil;
 import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.mpp.plan.constant.StatementType;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreatePipeSinkPlan;
 import org.apache.iotdb.db.sync.common.persistence.SyncLogReader;
@@ -69,11 +69,11 @@
       createPipeSinkPlan.addPipeSinkAttribute("port", "6670");
       log.addPipeSink(createPipeSinkPlan);
       log.addPipe(new CreatePipePlan(pipe1, "demo"), 1);
-      log.operatePipe(pipe1, Operator.OperatorType.DROP_PIPE);
+      log.operatePipe(pipe1, StatementType.DROP_PIPE);
 
       log.addPipe(new CreatePipePlan(pipe2, "demo"), 2);
-      log.operatePipe(pipe1, Operator.OperatorType.STOP_PIPE);
-      log.operatePipe(pipe1, Operator.OperatorType.START_PIPE);
+      log.operatePipe(pipe1, StatementType.STOP_PIPE);
+      log.operatePipe(pipe1, StatementType.START_PIPE);
       log.close();
       SyncLogReader syncLogReader = new SyncLogReader();
       syncLogReader.recover();
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java b/service-rpc/src/main/java/org/apache/iotdb/rpc/NoValidValueException.java
similarity index 66%
copy from node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java
copy to service-rpc/src/main/java/org/apache/iotdb/rpc/NoValidValueException.java
index ec0c375..c329765 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFExecutableResource.java
+++ b/service-rpc/src/main/java/org/apache/iotdb/rpc/NoValidValueException.java
@@ -16,24 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.rpc;
 
-package org.apache.iotdb.commons.udf.service;
-
-public class UDFExecutableResource {
-
-  private final long requestId;
-  private final String resourceDir;
-
-  public UDFExecutableResource(long requestId, String resourceDir) {
-    this.requestId = requestId;
-    this.resourceDir = resourceDir;
+public class NoValidValueException extends RuntimeException {
+  public NoValidValueException() {
+    super();
   }
 
-  public long getRequestId() {
-    return requestId;
-  }
-
-  public String getResourceDir() {
-    return resourceDir;
+  public NoValidValueException(String message) {
+    super(message);
   }
 }
diff --git a/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
index b7d095e..edb1675 100644
--- a/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
+++ b/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
@@ -103,6 +103,7 @@
   START_UP_ERROR(504),
   SHUT_DOWN_ERROR(505),
   MULTIPLE_ERROR(506),
+  SESSION_EXPIRED(507),
 
   WRONG_LOGIN_PASSWORD_ERROR(600),
   NOT_LOGIN_ERROR(601),
diff --git a/session/src/main/java/org/apache/iotdb/session/Session.java b/session/src/main/java/org/apache/iotdb/session/Session.java
index 6c270f1..3a07b2c 100644
--- a/session/src/main/java/org/apache/iotdb/session/Session.java
+++ b/session/src/main/java/org/apache/iotdb/session/Session.java
@@ -21,6 +21,7 @@
 import org.apache.iotdb.common.rpc.thrift.TEndPoint;
 import org.apache.iotdb.rpc.BatchExecutionException;
 import org.apache.iotdb.rpc.IoTDBConnectionException;
+import org.apache.iotdb.rpc.NoValidValueException;
 import org.apache.iotdb.rpc.RedirectException;
 import org.apache.iotdb.rpc.StatementExecutionException;
 import org.apache.iotdb.service.rpc.thrift.TSAppendSchemaTemplateReq;
@@ -780,8 +781,20 @@
       List<TSDataType> types,
       Object... values)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertRecordReq request =
-        genTSInsertRecordReq(deviceId, time, measurements, types, Arrays.asList(values), false);
+    TSInsertRecordReq request;
+    try {
+      request =
+          filterAndGenTSInsertRecordReq(
+              deviceId, time, measurements, types, Arrays.asList(values), false);
+    } catch (NoValidValueException e) {
+      logger.warn(
+          "All values are null and this submission is ignored,deviceId is [{}],time is [{}],measurements is [{}]",
+          deviceId,
+          time,
+          measurements.toString());
+      return;
+    }
+
     insertRecord(deviceId, request);
   }
 
@@ -906,8 +919,18 @@
       List<Object> values)
       throws IoTDBConnectionException, StatementExecutionException {
     // not vector by default
-    TSInsertRecordReq request =
-        genTSInsertRecordReq(deviceId, time, measurements, types, values, false);
+    TSInsertRecordReq request;
+    try {
+      request = filterAndGenTSInsertRecordReq(deviceId, time, measurements, types, values, false);
+    } catch (NoValidValueException e) {
+      logger.warn(
+          "All values are null and this submission is ignored,deviceId is [{}],time is [{}],measurements is [{}]",
+          deviceId,
+          time,
+          measurements.toString());
+      return;
+    }
+
     insertRecord(deviceId, request);
   }
 
@@ -925,11 +948,41 @@
       List<TSDataType> types,
       List<Object> values)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertRecordReq request =
-        genTSInsertRecordReq(deviceId, time, measurements, types, values, true);
+    TSInsertRecordReq request;
+    try {
+      request = filterAndGenTSInsertRecordReq(deviceId, time, measurements, types, values, true);
+    } catch (NoValidValueException e) {
+      logger.warn(
+          "All values are null and this submission is ignored,deviceId is [{}],time is [{}],measurements is [{}]",
+          deviceId,
+          time,
+          measurements.toString());
+      return;
+    }
     insertRecord(deviceId, request);
   }
 
+  private TSInsertRecordReq filterAndGenTSInsertRecordReq(
+      String prefixPath,
+      long time,
+      List<String> measurements,
+      List<TSDataType> types,
+      List<Object> values,
+      boolean isAligned)
+      throws IoTDBConnectionException {
+    if (hasNull(values)) {
+      measurements = new ArrayList<>(measurements);
+      values = new ArrayList<>(values);
+      types = new ArrayList<>(types);
+      boolean isAllValuesNull =
+          filterNullValueAndMeasurement(prefixPath, measurements, types, values);
+      if (isAllValuesNull) {
+        throw new NoValidValueException("All inserted data is null.");
+      }
+    }
+    return genTSInsertRecordReq(prefixPath, time, measurements, types, values, isAligned);
+  }
+
   private TSInsertRecordReq genTSInsertRecordReq(
       String prefixPath,
       long time,
@@ -958,8 +1011,17 @@
   public void insertRecord(
       String deviceId, long time, List<String> measurements, List<String> values)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertStringRecordReq request =
-        genTSInsertStringRecordReq(deviceId, time, measurements, values, false);
+    TSInsertStringRecordReq request;
+    try {
+      request = filterAndGenTSInsertStringRecordReq(deviceId, time, measurements, values, false);
+    } catch (NoValidValueException e) {
+      logger.warn(
+          "All values are null and this submission is ignored,deviceId is [{}],time is [{}],measurements is [{}]",
+          deviceId,
+          time,
+          measurements.toString());
+      return;
+    }
     insertRecord(deviceId, request);
   }
 
@@ -973,11 +1035,38 @@
   public void insertAlignedRecord(
       String deviceId, long time, List<String> measurements, List<String> values)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertStringRecordReq request =
-        genTSInsertStringRecordReq(deviceId, time, measurements, values, true);
+    TSInsertStringRecordReq request;
+    try {
+      request = filterAndGenTSInsertStringRecordReq(deviceId, time, measurements, values, true);
+    } catch (NoValidValueException e) {
+      logger.warn(
+          "All values are null and this submission is ignored,deviceId is [{}],time is [{}],measurements is [{}]",
+          deviceId,
+          time,
+          measurements.toString());
+      return;
+    }
     insertRecord(deviceId, request);
   }
 
+  private TSInsertStringRecordReq filterAndGenTSInsertStringRecordReq(
+      String prefixPath,
+      long time,
+      List<String> measurements,
+      List<String> values,
+      boolean isAligned) {
+    if (hasNull(values)) {
+      measurements = new ArrayList<>(measurements);
+      values = new ArrayList<>(values);
+      boolean isAllValueNull =
+          filterNullValueAndMeasurementWithStringType(values, prefixPath, measurements);
+      if (isAllValueNull) {
+        throw new NoValidValueException("All inserted data is null.");
+      }
+    }
+    return genTSInsertStringRecordReq(prefixPath, time, measurements, values, isAligned);
+  }
+
   private TSInsertStringRecordReq genTSInsertStringRecordReq(
       String prefixPath,
       long time,
@@ -1016,8 +1105,19 @@
     if (enableCacheLeader) {
       insertStringRecordsWithLeaderCache(deviceIds, times, measurementsList, valuesList, false);
     } else {
-      TSInsertStringRecordsReq request =
-          genTSInsertStringRecordsReq(deviceIds, times, measurementsList, valuesList, false);
+      TSInsertStringRecordsReq request;
+      try {
+        request =
+            filterAndGenTSInsertStringRecordsReq(
+                deviceIds, times, measurementsList, valuesList, false);
+      } catch (NoValidValueException e) {
+        logger.warn(
+            "All values are null and this submission is ignored,deviceIds are [{}],times are [{}],measurements are [{}]",
+            deviceIds.toString(),
+            times.toString(),
+            measurementsList.toString());
+        return;
+      }
       try {
         defaultSessionConnection.insertRecords(request);
       } catch (RedirectException e) {
@@ -1030,6 +1130,210 @@
   }
 
   /**
+   * When the value is null,filter this,don't use this measurement.
+   *
+   * @param times
+   * @param measurementsList
+   * @param valuesList
+   * @param typesList
+   */
+  private void filterNullValueAndMeasurement(
+      List<String> deviceIds,
+      List<Long> times,
+      List<List<String>> measurementsList,
+      List<List<Object>> valuesList,
+      List<List<TSDataType>> typesList) {
+    for (int i = valuesList.size() - 1; i >= 0; i--) {
+      List<Object> values = valuesList.get(i);
+      List<String> measurements = measurementsList.get(i);
+      List<TSDataType> types = typesList.get(i);
+      boolean isAllValuesNull =
+          filterNullValueAndMeasurement(deviceIds.get(i), measurements, types, values);
+      if (isAllValuesNull) {
+        valuesList.remove(i);
+        measurementsList.remove(i);
+        deviceIds.remove(i);
+        times.remove(i);
+        typesList.remove(i);
+      }
+    }
+    if (valuesList.size() == 0) {
+      throw new NoValidValueException("All inserted data is null.");
+    }
+  }
+
+  /**
+   * Filter the null value of list。
+   *
+   * @param deviceId
+   * @param times
+   * @param measurementsList
+   * @param typesList
+   * @param valuesList
+   */
+  private void filterNullValueAndMeasurementOfOneDevice(
+      String deviceId,
+      List<Long> times,
+      List<List<String>> measurementsList,
+      List<List<TSDataType>> typesList,
+      List<List<Object>> valuesList) {
+    for (int i = valuesList.size() - 1; i >= 0; i--) {
+      List<Object> values = valuesList.get(i);
+      List<String> measurements = measurementsList.get(i);
+      List<TSDataType> types = typesList.get(i);
+      boolean isAllValuesNull =
+          filterNullValueAndMeasurement(deviceId, measurements, types, values);
+      if (isAllValuesNull) {
+        valuesList.remove(i);
+        measurementsList.remove(i);
+        typesList.remove(i);
+        times.remove(i);
+      }
+    }
+    if (valuesList.size() == 0) {
+      throw new NoValidValueException("All inserted data is null.");
+    }
+  }
+
+  /**
+   * Filter the null value of list。
+   *
+   * @param times
+   * @param deviceId
+   * @param measurementsList
+   * @param valuesList
+   */
+  private void filterNullValueAndMeasurementWithStringTypeOfOneDevice(
+      List<Long> times,
+      String deviceId,
+      List<List<String>> measurementsList,
+      List<List<String>> valuesList) {
+    for (int i = valuesList.size() - 1; i >= 0; i--) {
+      List<String> values = valuesList.get(i);
+      List<String> measurements = measurementsList.get(i);
+      boolean isAllValuesNull =
+          filterNullValueAndMeasurementWithStringType(values, deviceId, measurements);
+      if (isAllValuesNull) {
+        valuesList.remove(i);
+        measurementsList.remove(i);
+        times.remove(i);
+      }
+    }
+    if (valuesList.size() == 0) {
+      throw new NoValidValueException("All inserted data is null.");
+    }
+  }
+
+  /**
+   * Filter the null object of list。
+   *
+   * @param deviceId
+   * @param measurementsList
+   * @param types
+   * @param valuesList
+   * @return true:all value is null;false:not all null value is null.
+   */
+  private boolean filterNullValueAndMeasurement(
+      String deviceId,
+      List<String> measurementsList,
+      List<TSDataType> types,
+      List<Object> valuesList) {
+    Map<String, Object> nullMap = new HashMap<>();
+    for (int i = valuesList.size() - 1; i >= 0; i--) {
+      if (valuesList.get(i) == null) {
+        nullMap.put(measurementsList.get(i), valuesList.get(i));
+        valuesList.remove(i);
+        measurementsList.remove(i);
+        types.remove(i);
+      }
+    }
+    if (valuesList.size() == 0) {
+      logger.info("All values of the {} are null,null values are {}", deviceId, nullMap.toString());
+      return true;
+    } else {
+      logger.info("Some values of {} are null,null values are {}", deviceId, nullMap.toString());
+    }
+    return false;
+  }
+
+  /**
+   * Filter the null object of list。
+   *
+   * @param prefixPaths devices path。
+   * @param times
+   * @param measurementsList
+   * @param valuesList
+   * @return true:all values of valuesList are null;false:Not all values of valuesList are null.
+   */
+  private void filterNullValueAndMeasurementWithStringType(
+      List<String> prefixPaths,
+      List<Long> times,
+      List<List<String>> measurementsList,
+      List<List<String>> valuesList) {
+    for (int i = valuesList.size() - 1; i >= 0; i--) {
+      List<String> values = valuesList.get(i);
+      List<String> measurements = measurementsList.get(i);
+      boolean isAllValueNull =
+          filterNullValueAndMeasurementWithStringType(values, prefixPaths.get(i), measurements);
+      if (isAllValueNull) {
+        valuesList.remove(i);
+        measurementsList.remove(i);
+        times.remove(i);
+        prefixPaths.remove(i);
+      }
+    }
+    if (valuesList.size() == 0) {
+      throw new NoValidValueException("All inserted data is null.");
+    }
+  }
+
+  /**
+   * When the value is null,filter this,don't use this measurement.
+   *
+   * @param valuesList
+   * @param measurementsList
+   * @return true:all value is null;false:not all null value is null.
+   */
+  private boolean filterNullValueAndMeasurementWithStringType(
+      List<String> valuesList, String deviceId, List<String> measurementsList) {
+    Map<String, Object> nullMap = new HashMap<>();
+    for (int i = valuesList.size() - 1; i >= 0; i--) {
+      if (valuesList.get(i) == null) {
+        nullMap.put(measurementsList.get(i), valuesList.get(i));
+        valuesList.remove(i);
+        measurementsList.remove(i);
+      }
+    }
+    if (valuesList.size() == 0) {
+      logger.info("All values of the {} are null,null values are {}", deviceId, nullMap.toString());
+      return true;
+    } else {
+      logger.info("Some values of {} are null,null values are {}", deviceId, nullMap.toString());
+    }
+    return false;
+  }
+
+  private boolean hasNull(List valuesList) {
+    boolean haveNull = false;
+    for (int i1 = 0; i1 < valuesList.size(); i1++) {
+      Object o = valuesList.get(i1);
+      if (o instanceof List) {
+        List o1 = (List) o;
+        if (hasNull(o1)) {
+          haveNull = true;
+          break;
+        }
+      } else {
+        if (o == null) {
+          haveNull = true;
+          break;
+        }
+      }
+    }
+    return haveNull;
+  }
+
+  /**
    * Insert aligned multiple rows, which can reduce the overhead of network. This method is just
    * like jdbc executeBatch, we pack some insert request in batch and send them to server. If you
    * want improve your performance, please see insertTablet method
@@ -1053,8 +1357,20 @@
     if (enableCacheLeader) {
       insertStringRecordsWithLeaderCache(deviceIds, times, measurementsList, valuesList, true);
     } else {
-      TSInsertStringRecordsReq request =
-          genTSInsertStringRecordsReq(deviceIds, times, measurementsList, valuesList, true);
+      TSInsertStringRecordsReq request;
+      try {
+        request =
+            filterAndGenTSInsertStringRecordsReq(
+                deviceIds, times, measurementsList, valuesList, true);
+      } catch (NoValidValueException e) {
+        logger.warn(
+            "All values are null and this submission is ignored,deviceIds are [{}],times are [{}],measurements are [{}]",
+            deviceIds.toString(),
+            times.toString(),
+            measurementsList.toString());
+        return;
+      }
+
       try {
         defaultSessionConnection.insertRecords(request);
       } catch (RedirectException e) {
@@ -1079,13 +1395,77 @@
       TSInsertStringRecordsReq request =
           recordsGroup.computeIfAbsent(connection, k -> new TSInsertStringRecordsReq());
       request.setIsAligned(isAligned);
-      updateTSInsertStringRecordsReq(
-          request, deviceIds.get(i), times.get(i), measurementsList.get(i), valuesList.get(i));
+      try {
+        filterAndUpdateTSInsertStringRecordsReq(
+            request, deviceIds.get(i), times.get(i), measurementsList.get(i), valuesList.get(i));
+      } catch (NoValidValueException e) {
+        logger.warn(
+            "All values are null and this submission is ignored,deviceId is [{}],time is [{}],measurements is [{}]",
+            deviceIds.get(i),
+            times.get(i),
+            measurementsList.get(i).toString());
+        continue;
+      }
     }
 
     insertByGroup(recordsGroup, SessionConnection::insertRecords);
   }
 
+  private TSInsertStringRecordsReq filterAndGenTSInsertStringRecordsReq(
+      List<String> prefixPaths,
+      List<Long> time,
+      List<List<String>> measurements,
+      List<List<String>> values,
+      boolean isAligned) {
+    if (hasNull(values)) {
+      values = changeToArrayListWithStringType(values);
+      measurements = changeToArrayListWithStringType(measurements);
+      prefixPaths = new ArrayList<>(prefixPaths);
+      time = new ArrayList<>(time);
+      filterNullValueAndMeasurementWithStringType(prefixPaths, time, measurements, values);
+    }
+    return genTSInsertStringRecordsReq(prefixPaths, time, measurements, values, isAligned);
+  }
+
+  private List<List<String>> changeToArrayListWithStringType(List<List<String>> values) {
+    if (!(values instanceof ArrayList)) {
+      values = new ArrayList<>(values);
+    }
+    for (int i = 0; i < values.size(); i++) {
+      List<String> currentValue = values.get(i);
+      if (!(currentValue instanceof ArrayList)) {
+        values.set(i, new ArrayList<>(currentValue));
+      }
+    }
+    return values;
+  }
+
+  private List<List<Object>> changeToArrayList(List<List<Object>> values) {
+    if (!(values instanceof ArrayList)) {
+      values = new ArrayList<>(values);
+    }
+    for (int i = 0; i < values.size(); i++) {
+      List<Object> currentValue = values.get(i);
+      if (!(currentValue instanceof ArrayList)) {
+        values.set(i, new ArrayList<>(currentValue));
+      }
+    }
+    return values;
+  }
+
+  private List<List<TSDataType>> changeToArrayListWithTSDataType(List<List<TSDataType>> values) {
+    if (!(values instanceof ArrayList)) {
+      values = new ArrayList<>(values);
+    }
+    for (int i = 0; i < values.size(); i++) {
+      List<TSDataType> currentValue = values.get(i);
+      if (!(currentValue instanceof ArrayList)) {
+        values.set(i, new ArrayList<>(currentValue));
+      }
+    }
+    return values;
+  }
+
   private TSInsertStringRecordsReq genTSInsertStringRecordsReq(
       List<String> prefixPaths,
       List<Long> time,
@@ -1102,6 +1482,24 @@
     return request;
   }
 
+  private void filterAndUpdateTSInsertStringRecordsReq(
+      TSInsertStringRecordsReq request,
+      String deviceId,
+      long time,
+      List<String> measurements,
+      List<String> values) {
+    if (hasNull(values)) {
+      measurements = new ArrayList<>(measurements);
+      values = new ArrayList<>(values);
+      boolean isAllValueNull =
+          filterNullValueAndMeasurementWithStringType(values, deviceId, measurements);
+      if (isAllValueNull) {
+        throw new NoValidValueException("All inserted data is null.");
+      }
+    }
+    updateTSInsertStringRecordsReq(request, deviceId, time, measurements, values);
+  }
+
   private void updateTSInsertStringRecordsReq(
       TSInsertStringRecordsReq request,
       String deviceId,
@@ -1139,8 +1537,19 @@
       insertRecordsWithLeaderCache(
           deviceIds, times, measurementsList, typesList, valuesList, false);
     } else {
-      TSInsertRecordsReq request =
-          genTSInsertRecordsReq(deviceIds, times, measurementsList, typesList, valuesList, false);
+      TSInsertRecordsReq request;
+      try {
+        request =
+            filterAndGenTSInsertRecordsReq(
+                deviceIds, times, measurementsList, typesList, valuesList, false);
+      } catch (NoValidValueException e) {
+        logger.warn(
+            "All values are null and this submission is ignored,deviceIds are [{}],times are [{}],measurements are [{}]",
+            deviceIds.toString(),
+            times.toString(),
+            measurementsList.toString());
+        return;
+      }
       try {
         defaultSessionConnection.insertRecords(request);
       } catch (RedirectException e) {
@@ -1177,8 +1586,19 @@
     if (enableCacheLeader) {
       insertRecordsWithLeaderCache(deviceIds, times, measurementsList, typesList, valuesList, true);
     } else {
-      TSInsertRecordsReq request =
-          genTSInsertRecordsReq(deviceIds, times, measurementsList, typesList, valuesList, true);
+      TSInsertRecordsReq request;
+      try {
+        request =
+            filterAndGenTSInsertRecordsReq(
+                deviceIds, times, measurementsList, typesList, valuesList, true);
+      } catch (NoValidValueException e) {
+        logger.warn(
+            "All values are null and this submission is ignored,deviceIds are [{}],times are [{}],measurements are [{}]",
+            deviceIds.toString(),
+            times.toString(),
+            measurementsList.toString());
+        return;
+      }
       try {
         defaultSessionConnection.insertRecords(request);
       } catch (RedirectException e) {
@@ -1232,9 +1652,19 @@
       throw new IllegalArgumentException(
           "times, measurementsList and valuesList's size should be equal");
     }
-    TSInsertRecordsOfOneDeviceReq request =
-        genTSInsertRecordsOfOneDeviceReq(
-            deviceId, times, measurementsList, typesList, valuesList, haveSorted, false);
+    TSInsertRecordsOfOneDeviceReq request;
+    try {
+      request =
+          filterAndGenTSInsertRecordsOfOneDeviceReq(
+              deviceId, times, measurementsList, typesList, valuesList, haveSorted, false);
+    } catch (NoValidValueException e) {
+      logger.warn(
+          "All values are null and this submission is ignored,deviceId is [{}],times are [{}],measurements are [{}]",
+          deviceId,
+          times.toString(),
+          measurementsList.toString());
+      return;
+    }
     try {
       getSessionConnection(deviceId).insertRecordsOfOneDevice(request);
     } catch (RedirectException e) {
@@ -1264,9 +1694,19 @@
       throw new IllegalArgumentException(
           "times, measurementsList and valuesList's size should be equal");
     }
-    TSInsertStringRecordsOfOneDeviceReq req =
-        genTSInsertStringRecordsOfOneDeviceReq(
-            deviceId, times, measurementsList, valuesList, haveSorted, false);
+    TSInsertStringRecordsOfOneDeviceReq req;
+    try {
+      req =
+          filterAndGenTSInsertStringRecordsOfOneDeviceReq(
+              deviceId, times, measurementsList, valuesList, haveSorted, false);
+    } catch (NoValidValueException e) {
+      logger.warn(
+          "All values are null and this submission is ignored,deviceId is [{}],times are [{}],measurements are [{}]",
+          deviceId,
+          times.toString(),
+          measurementsList.toString());
+      return;
+    }
     try {
       getSessionConnection(deviceId).insertStringRecordsOfOneDevice(req);
     } catch (RedirectException e) {
@@ -1334,9 +1774,19 @@
       throw new IllegalArgumentException(
           "times, subMeasurementsList and valuesList's size should be equal");
     }
-    TSInsertRecordsOfOneDeviceReq request =
-        genTSInsertRecordsOfOneDeviceReq(
-            deviceId, times, measurementsList, typesList, valuesList, haveSorted, true);
+    TSInsertRecordsOfOneDeviceReq request;
+    try {
+      request =
+          filterAndGenTSInsertRecordsOfOneDeviceReq(
+              deviceId, times, measurementsList, typesList, valuesList, haveSorted, true);
+    } catch (NoValidValueException e) {
+      logger.warn(
+          "All values are null and this submission is ignored,deviceId is [{}],times are [{}],measurements are [{}]",
+          deviceId,
+          times.toString(),
+          measurementsList.toString());
+      return;
+    }
     try {
       getSessionConnection(deviceId).insertRecordsOfOneDevice(request);
     } catch (RedirectException e) {
@@ -1366,9 +1816,19 @@
       throw new IllegalArgumentException(
           "times, measurementsList and valuesList's size should be equal");
     }
-    TSInsertStringRecordsOfOneDeviceReq req =
-        genTSInsertStringRecordsOfOneDeviceReq(
-            deviceId, times, measurementsList, valuesList, haveSorted, true);
+    TSInsertStringRecordsOfOneDeviceReq req;
+    try {
+      req =
+          filterAndGenTSInsertStringRecordsOfOneDeviceReq(
+              deviceId, times, measurementsList, valuesList, haveSorted, true);
+    } catch (NoValidValueException e) {
+      logger.warn(
+          "All values are null and this submission is ignored,deviceId is [{}],times are [{}],measurements are [{}]",
+          deviceId,
+          times.toString(),
+          measurementsList.toString());
+      return;
+    }
     try {
       getSessionConnection(deviceId).insertStringRecordsOfOneDevice(req);
     } catch (RedirectException e) {
@@ -1395,6 +1855,27 @@
     insertAlignedStringRecordsOfOneDevice(deviceId, times, measurementsList, valuesList, false);
   }
 
+  private TSInsertRecordsOfOneDeviceReq filterAndGenTSInsertRecordsOfOneDeviceReq(
+      String prefixPath,
+      List<Long> times,
+      List<List<String>> measurementsList,
+      List<List<TSDataType>> typesList,
+      List<List<Object>> valuesList,
+      boolean haveSorted,
+      boolean isAligned)
+      throws IoTDBConnectionException, BatchExecutionException {
+    if (hasNull(valuesList)) {
+      measurementsList = changeToArrayListWithStringType(measurementsList);
+      valuesList = changeToArrayList(valuesList);
+      typesList = changeToArrayListWithTSDataType(typesList);
+      times = new ArrayList<>(times);
+      filterNullValueAndMeasurementOfOneDevice(
+          prefixPath, times, measurementsList, typesList, valuesList);
+    }
+    return genTSInsertRecordsOfOneDeviceReq(
+        prefixPath, times, measurementsList, typesList, valuesList, haveSorted, isAligned);
+  }
+
   private TSInsertRecordsOfOneDeviceReq genTSInsertRecordsOfOneDeviceReq(
       String prefixPath,
       List<Long> times,
@@ -1437,6 +1918,24 @@
     return request;
   }
 
+  private TSInsertStringRecordsOfOneDeviceReq filterAndGenTSInsertStringRecordsOfOneDeviceReq(
+      String prefixPath,
+      List<Long> times,
+      List<List<String>> measurementsList,
+      List<List<String>> valuesList,
+      boolean haveSorted,
+      boolean isAligned) {
+    if (hasNull(valuesList)) {
+      measurementsList = changeToArrayListWithStringType(measurementsList);
+      valuesList = changeToArrayListWithStringType(valuesList);
+      times = new ArrayList<>(times);
+      filterNullValueAndMeasurementWithStringTypeOfOneDevice(
+          times, prefixPath, measurementsList, valuesList);
+    }
+    return genTSInsertStringRecordsOfOneDeviceReq(
+        prefixPath, times, measurementsList, valuesList, haveSorted, isAligned);
+  }
+
   private TSInsertStringRecordsOfOneDeviceReq genTSInsertStringRecordsOfOneDeviceReq(
       String prefixPath,
       List<Long> times,
@@ -1512,17 +2011,46 @@
       TSInsertRecordsReq request =
           recordsGroup.computeIfAbsent(connection, k -> new TSInsertRecordsReq());
       request.setIsAligned(isAligned);
-      updateTSInsertRecordsReq(
-          request,
-          deviceIds.get(i),
-          times.get(i),
-          measurementsList.get(i),
-          typesList.get(i),
-          valuesList.get(i));
+      try {
+        filterAndUpdateTSInsertRecordsReq(
+            request,
+            deviceIds.get(i),
+            times.get(i),
+            measurementsList.get(i),
+            typesList.get(i),
+            valuesList.get(i));
+      } catch (NoValidValueException e) {
+        logger.warn(
+            "All values are null and this submission is ignored,deviceId is [{}],time is [{}],measurements are [{}]",
+            deviceIds.get(i),
+            times.get(i),
+            measurementsList.get(i).toString());
+        continue;
+      }
     }
     insertByGroup(recordsGroup, SessionConnection::insertRecords);
   }
 
+  private TSInsertRecordsReq filterAndGenTSInsertRecordsReq(
+      List<String> deviceIds,
+      List<Long> times,
+      List<List<String>> measurementsList,
+      List<List<TSDataType>> typesList,
+      List<List<Object>> valuesList,
+      boolean isAligned)
+      throws IoTDBConnectionException {
+    if (hasNull(valuesList)) {
+      measurementsList = changeToArrayListWithStringType(measurementsList);
+      valuesList = changeToArrayList(valuesList);
+      deviceIds = new ArrayList<>(deviceIds);
+      times = new ArrayList<>(times);
+      typesList = changeToArrayListWithTSDataType(typesList);
+      filterNullValueAndMeasurement(deviceIds, times, measurementsList, valuesList, typesList);
+    }
+    return genTSInsertRecordsReq(
+        deviceIds, times, measurementsList, typesList, valuesList, isAligned);
+  }
+
   private TSInsertRecordsReq genTSInsertRecordsReq(
       List<String> deviceIds,
       List<Long> times,
@@ -1541,6 +2069,27 @@
     return request;
   }
 
+  private void filterAndUpdateTSInsertRecordsReq(
+      TSInsertRecordsReq request,
+      String deviceId,
+      Long time,
+      List<String> measurements,
+      List<TSDataType> types,
+      List<Object> values)
+      throws IoTDBConnectionException {
+    if (hasNull(values)) {
+      measurements = new ArrayList<>(measurements);
+      types = new ArrayList<>(types);
+      values = new ArrayList<>(values);
+      boolean isAllValuesNull =
+          filterNullValueAndMeasurement(deviceId, measurements, types, values);
+      if (isAllValuesNull) {
+        throw new NoValidValueException("All inserted data is null.");
+      }
+    }
+    updateTSInsertRecordsReq(request, deviceId, time, measurements, types, values);
+  }
+
   private void updateTSInsertRecordsReq(
       TSInsertRecordsReq request,
       String deviceId,
@@ -1810,8 +2359,20 @@
       List<List<String>> measurementsList,
       List<List<String>> valuesList)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertStringRecordsReq request =
-        genTSInsertStringRecordsReq(deviceIds, times, measurementsList, valuesList, false);
+    TSInsertStringRecordsReq request;
+    try {
+      request =
+          filterAndGenTSInsertStringRecordsReq(
+              deviceIds, times, measurementsList, valuesList, false);
+    } catch (NoValidValueException e) {
+      logger.warn(
+          "All values are null and this submission is ignored,deviceIds are [{}],times are [{}],measurements are [{}]",
+          deviceIds.toString(),
+          times.toString(),
+          measurementsList.toString());
+      return;
+    }
+
     defaultSessionConnection.testInsertRecords(request);
   }
 
@@ -1826,8 +2387,20 @@
       List<List<TSDataType>> typesList,
       List<List<Object>> valuesList)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertRecordsReq request =
-        genTSInsertRecordsReq(deviceIds, times, measurementsList, typesList, valuesList, false);
+    TSInsertRecordsReq request;
+    try {
+      request =
+          filterAndGenTSInsertRecordsReq(
+              deviceIds, times, measurementsList, typesList, valuesList, false);
+    } catch (NoValidValueException e) {
+      logger.warn(
+          "All values are null and this submission is ignored,deviceIds are [{}],times are [{}],measurements are [{}]",
+          deviceIds.toString(),
+          times.toString(),
+          measurementsList.toString());
+      return;
+    }
+
     defaultSessionConnection.testInsertRecords(request);
   }
 
@@ -1838,8 +2411,17 @@
   public void testInsertRecord(
       String deviceId, long time, List<String> measurements, List<String> values)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertStringRecordReq request =
-        genTSInsertStringRecordReq(deviceId, time, measurements, values, false);
+    TSInsertStringRecordReq request;
+    try {
+      request = filterAndGenTSInsertStringRecordReq(deviceId, time, measurements, values, false);
+    } catch (NoValidValueException e) {
+      logger.warn(
+          "All values are null and this submission is ignored,deviceId is [{}],time is [{}],measurements is [{}]",
+          deviceId,
+          time,
+          measurements.toString());
+      return;
+    }
     defaultSessionConnection.testInsertRecord(request);
   }
 
@@ -1854,8 +2436,17 @@
       List<TSDataType> types,
       List<Object> values)
       throws IoTDBConnectionException, StatementExecutionException {
-    TSInsertRecordReq request =
-        genTSInsertRecordReq(deviceId, time, measurements, types, values, false);
+    TSInsertRecordReq request;
+    try {
+      request = filterAndGenTSInsertRecordReq(deviceId, time, measurements, types, values, false);
+    } catch (NoValidValueException e) {
+      logger.warn(
+          "All values are null and this submission is ignored,deviceId is [{}],time is [{}],measurements is [{}]",
+          deviceId,
+          time,
+          measurements.toString());
+      return;
+    }
     defaultSessionConnection.testInsertRecord(request);
   }
 
diff --git a/site/src/main/.vuepress/config.js b/site/src/main/.vuepress/config.js
index 91a3838..d5ee8db 100644
--- a/site/src/main/.vuepress/config.js
+++ b/site/src/main/.vuepress/config.js
@@ -962,7 +962,7 @@
 							['Ecosystem-Integration/Grafana-Connector','Grafana-Connector (Not Recommended)'],
 							['Ecosystem-Integration/Zeppelin-IoTDB','Zeppelin-IoTDB'],
 							['Ecosystem-Integration/DBeaver','DBeaver-IoTDB'],
-							['Ecosystem-Integration/MapReduce TsFile','MapReduce-TsFile'],
+							['Ecosystem-Integration/MapReduce-TsFile','MapReduce-TsFile'],
 							['Ecosystem-Integration/Spark-TsFile','Spark-TsFile'],
 							['Ecosystem-Integration/Spark-IoTDB','Spark-IoTDB'],
 							['Ecosystem-Integration/Hive-TsFile','Hive-TsFile'],
diff --git a/thrift-confignode/src/main/thrift/confignode.thrift b/thrift-confignode/src/main/thrift/confignode.thrift
index 1e6d62f..e177f8b 100644
--- a/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/thrift-confignode/src/main/thrift/confignode.thrift
@@ -231,12 +231,27 @@
 }
 
 // Trigger
+enum TTriggerState {
+  // The intermediate state of Create trigger, the trigger need to create has not yet activated on any DataNodes.
+  INACTIVE
+  // The intermediate state of Create trigger, the trigger need to create has activated on some DataNodes.
+  PARTIAL_ACTIVE
+  // Triggers on all DataNodes are available.
+  ACTIVE
+  // The intermediate state of Drop trigger, the cluster is in the process of removing the trigger.
+  DROPPING
+}
+
 struct TCreateTriggerReq {
   1: required string triggerName
   2: required binary triggerInformation
   3: required common.TFile jarFile
 }
 
+struct TDropTriggerReq {
+  1: required string triggerName
+}
+
 // Show cluster
 struct TShowClusterResp {
   1: required common.TSStatus status
@@ -349,6 +364,21 @@
   2: optional list<string> pathList
 }
 
+// Show pipe
+struct TPipeInfo {
+  1: required i64 createTime
+  2: required string pipeName
+  3: required string role
+  4: required string remote
+  5: required string status
+  6: required string message
+}
+
+struct TShowPipeResp {
+  1: required common.TSStatus status
+  2: optional list<TPipeInfo> pipeInfoList
+}
+
 service IConfigNodeRPCService {
 
   // ======================================================
@@ -604,6 +634,14 @@
       */
   common.TSStatus createTrigger(TCreateTriggerReq req)
 
+  /**
+       * Remove a trigger on all online ConfigNodes and DataNodes
+       *
+       * @return SUCCESS_STATUS if the function was removed successfully
+       *         EXECUTE_STATEMENT_ERROR if operations on any node failed
+       */
+    common.TSStatus dropTrigger(TDropTriggerReq req)
+
   // ======================================================
   // Maintenance Tools
   // ======================================================
diff --git a/thrift/src/main/thrift/datanode.thrift b/thrift/src/main/thrift/datanode.thrift
index 8cfc0f9..a84cf62 100644
--- a/thrift/src/main/thrift/datanode.thrift
+++ b/thrift/src/main/thrift/datanode.thrift
@@ -176,7 +176,7 @@
   2: required binary jarFile
 }
 
-struct TremoveTriggerInstanceReq {
+struct TDropTriggerInstanceReq {
   1: required string triggerName
   2: required bool needToDeleteJarFile
 }
@@ -355,6 +355,13 @@
   common.TSStatus createTriggerInstance(TcreateTriggerInstanceReq req)
 
   /**
+     * Config node will drop a trigger on all online config nodes and data nodes.
+     *
+     * @param trigger name, whether need to delete jar
+     **/
+    common.TSStatus dropTriggerInstance(TDropTriggerInstanceReq req)
+
+  /**
    * Config node will invalidate permission Info cache.
    *
    * @param string:username, list<string>:roleList
diff --git a/trigger-api/pom.xml b/trigger-api/pom.xml
new file mode 100644
index 0000000..4d93a3d
--- /dev/null
+++ b/trigger-api/pom.xml
@@ -0,0 +1,73 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+        http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>iotdb-parent</artifactId>
+        <groupId>org.apache.iotdb</groupId>
+        <version>0.14.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>trigger-api</artifactId>
+    <profiles>
+        <profile>
+            <id>get-jar-with-dependencies</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <artifactId>maven-assembly-plugin</artifactId>
+                        <version>${maven.assembly.version}</version>
+                        <configuration>
+                            <descriptorRefs>
+                                <descriptorRef>jar-with-dependencies</descriptorRef>
+                            </descriptorRefs>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <id>make-assembly</id>
+                                <!-- this is used for inheritance merges -->
+                                <phase>package</phase>
+                                <!-- bind to the packaging phase -->
+                                <goals>
+                                    <goal>single</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.iotdb</groupId>
+            <artifactId>tsfile</artifactId>
+            <version>${project.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+    </dependencies>
+</project>
diff --git a/trigger-api/src/main/java/org/apache/iotdb/trigger/api/Trigger.java b/trigger-api/src/main/java/org/apache/iotdb/trigger/api/Trigger.java
new file mode 100644
index 0000000..7ed99bc
--- /dev/null
+++ b/trigger-api/src/main/java/org/apache/iotdb/trigger/api/Trigger.java
@@ -0,0 +1,68 @@
+/*
+ * 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.trigger.api;
+
+import org.apache.iotdb.tsfile.write.record.Tablet;
+
+public interface Trigger {
+
+  /**
+   * This method is mainly used to validate {@link TriggerAttributes} before calling {@link
+   * Trigger#onCreate(TriggerAttributes)}.
+   *
+   * @param attributes TriggerAttributes
+   * @throws Exception e
+   */
+  default void validate(TriggerAttributes attributes) throws Exception {}
+
+  /**
+   * This method will be called when creating a trigger after validation.
+   *
+   * @param attributes TriggerAttributes
+   * @throws Exception e
+   */
+  default void onCreate(TriggerAttributes attributes) throws Exception {}
+
+  /**
+   * This method will be called when dropping a trigger.
+   *
+   * @throws Exception e
+   */
+  default void onDrop() throws Exception {}
+
+  /**
+   * When restarting a DataNode, Triggers that have been registered will be restored and this method
+   * will be called during the process of restoring.
+   *
+   * @throws Exception e
+   */
+  default void restore() throws Exception {}
+
+  /**
+   * @param tablet see {@link Tablet} for detailed information of data structure. Data that is
+   *     inserted will be constructed as a Tablet and you can define process logic with {@link
+   *     Tablet}.
+   * @return true if successfully fired
+   * @throws Exception e
+   */
+  default boolean fire(Tablet tablet) throws Exception {
+    return true;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/trigger/api/TriggerAttributes.java b/trigger-api/src/main/java/org/apache/iotdb/trigger/api/TriggerAttributes.java
similarity index 98%
rename from server/src/main/java/org/apache/iotdb/db/engine/trigger/api/TriggerAttributes.java
rename to trigger-api/src/main/java/org/apache/iotdb/trigger/api/TriggerAttributes.java
index 2fd69d8..64da3a9 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/trigger/api/TriggerAttributes.java
+++ b/trigger-api/src/main/java/org/apache/iotdb/trigger/api/TriggerAttributes.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.engine.trigger.api;
+package org.apache.iotdb.trigger.api;
 
 import java.util.Map;