You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ro...@apache.org on 2024/03/25 02:07:31 UTC

(iotdb) branch master updated: Pipe: Introduced aggregate processor as the universal aggregation framework (#12210)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 945112d642d Pipe: Introduced aggregate processor as the universal aggregation framework (#12210)
945112d642d is described below

commit 945112d642d0d61942a7e622bf88252a95688b26
Author: Caideyipi <87...@users.noreply.github.com>
AuthorDate: Mon Mar 25 10:07:25 2024 +0800

    Pipe: Introduced aggregate processor as the universal aggregation framework (#12210)
    
    This commit introduced aggregate processor and filled in some operators and tumbling windowing tactics. This processor can dynamically adopt windowing processor and operator processors as its plugins to implement custom aggregation.
    
    ---------
    
    Co-authored-by: Steve Yurong Su <ro...@apache.org>
---
 .../apache/iotdb/opcua/ClientExampleRunner.java    |  13 +-
 .../java/org/apache/iotdb/opcua/ClientTest.java    |  19 +-
 .../iotdb/opcua/IoTDBKeyStoreLoaderClient.java     |  14 +-
 .../iotdb/pipe/it/single/AbstractPipeSingleIT.java |  25 +-
 .../iotdb/pipe/it/single/IoTDBPipeAggregateIT.java | 121 ++++
 .../it/{ => single}/IoTDBPipeSingleEnvDemoIT.java  |  20 +-
 .../api/customizer/parameter/PipeParameters.java   |  19 +
 .../flink/sql/client/IoTDBWebSocketClient.java     |   9 +-
 .../iotdb/confignode/manager/ProcedureManager.java | 138 ++--
 .../agent/plugin/PipeConfigNodePluginAgent.java    |  10 +-
 .../PipeConfigRegionConnectorConstructor.java      |   2 +-
 .../PipeConfigRegionExtractorConstructor.java      |   2 +-
 .../PipeConfigRegionProcessorConstructor.java      |   2 +-
 .../transfer/execution/PipeConfigNodeSubtask.java  |  21 +-
 .../persistence/pipe/PipePluginInfo.java           |   4 +-
 .../confignode/persistence/pipe/PipeTaskInfo.java  |   6 +-
 .../impl/pipe/task/CreatePipeProcedureV2.java      |   4 +-
 .../impl/pipe/task/DropPipeProcedureV2.java        |   2 +-
 .../impl/pipe/task/StartPipeProcedureV2.java       |   4 +-
 .../impl/pipe/task/StopPipeProcedureV2.java        |   4 +-
 .../dataregion/PipeDataRegionPluginAgent.java      |  10 +-
 .../PipeDataRegionProcessorConstructor.java        |  11 +
 .../PipeSchemaRegionConnectorConstructor.java      |   2 +-
 .../PipeSchemaRegionExtractorConstructor.java      |   2 +-
 .../schemaregion/PipeSchemaRegionPluginAgent.java  |  10 +-
 .../PipeSchemaRegionProcessorConstructor.java      |   2 +-
 .../protocol/legacy/IoTDBLegacyPipeConnector.java  |   6 +-
 .../connector/protocol/opcua/OpcUaConnector.java   |  16 +-
 .../protocol/opcua/OpcUaKeyStoreLoader.java        |  20 +-
 .../protocol/opcua/OpcUaServerBuilder.java         |  44 +-
 .../async/IoTDBDataRegionAsyncConnector.java       |   3 +-
 .../thrift/sync/IoTDBDataRegionSyncConnector.java  |   6 +-
 .../protocol/writeback/WriteBackConnector.java     |   2 +-
 .../pipe/event/common/row/PipeResetTabletRow.java  |  54 ++
 .../db/pipe/event/common/row/PipeRowCollector.java |   9 +-
 .../tablet/PipeInsertNodeTabletInsertionEvent.java |  14 +-
 .../common/tablet/PipeRawTabletInsertionEvent.java |  23 +-
 .../tablet/TabletInsertionDataContainer.java       |   2 +-
 .../common/tsfile/PipeTsFileInsertionEvent.java    |   4 +-
 .../PipeHistoricalDataRegionTsFileExtractor.java   |  29 +-
 .../realtime/PipeRealtimeDataRegionExtractor.java  |   6 +-
 .../aggregate/AbstractFormalProcessor.java         |  54 ++
 .../processor/aggregate/AggregateProcessor.java    | 719 +++++++++++++++++++++
 .../aggregate/TimeSeriesRuntimeState.java          | 396 ++++++++++++
 .../aggregatedresult/AggregatedResultOperator.java |  62 ++
 .../standardstatistics/AverageOperator.java        |  56 ++
 .../ClearanceFactorOperator.java                   |  60 ++
 .../standardstatistics/CrestFactorOperator.java    |  59 ++
 .../standardstatistics/FormFactorOperator.java     |  57 ++
 .../standardstatistics/KurtosisOperator.java       |  67 ++
 .../standardstatistics/PeakOperator.java           |  52 ++
 .../standardstatistics/PulseFactorOperator.java    |  57 ++
 .../standardstatistics/RootMeanSquareOperator.java |  57 ++
 .../standardstatistics/SkewnessOperator.java       |  63 ++
 .../standardstatistics/VarianceOperator.java       |  59 ++
 .../CustomizedReadableIntermediateResults.java     | 158 +++++
 .../IntermediateResultOperator.java                | 113 ++++
 .../sametype/numeric/AbsoluteMaxOperator.java      |  71 ++
 .../numeric/AbstractSameTypeNumericOperator.java   | 152 +++++
 .../numeric/IntegralPoweredSumOperator.java        |  77 +++
 .../doubletype/FractionPoweredSumOperator.java     | 137 ++++
 .../specifictype/integertype/CountOperator.java    | 125 ++++
 .../processor/AbstractOperatorProcessor.java       |  73 +++
 .../StandardStatisticsOperatorProcessor.java       |  76 +++
 .../window/datastructure/TimeSeriesWindow.java     | 495 ++++++++++++++
 .../window/datastructure/WindowOutput.java         |  71 ++
 .../window/datastructure/WindowState.java          |  75 +++
 .../AbstractSimpleTimeWindowingProcessor.java      | 102 +++
 .../processor/AbstractWindowingProcessor.java      | 119 ++++
 .../processor/TumblingWindowingProcessor.java      | 110 ++++
 .../sdt/SwingingDoorTrendingFilter.java            |   8 +-
 .../sdt/SwingingDoorTrendingSamplingProcessor.java |   6 +-
 .../receiver/thrift/IoTDBDataNodeReceiver.java     |   2 +-
 .../pipe/task/builder/PipeDataNodeTaskBuilder.java |   3 +-
 .../db/pipe/task/stage/PipeTaskProcessorStage.java |  47 +-
 .../subtask/connector/PipeConnectorSubtask.java    |   2 +-
 .../config/executor/ClusterConfigTaskExecutor.java | 121 ++--
 .../planner/plan/node/write/InsertTabletNode.java  |   3 +-
 .../dataregion/memtable/TsFileProcessor.java       |   2 +-
 .../dataregion/tsfile/TsFileManager.java           |   6 +-
 .../dataregion/tsfile/TsFileResource.java          |   2 +-
 .../dataregion/wal/utils/WALInsertNodeCache.java   |   2 +-
 .../tools/schema/SchemaRegionSnapshotParser.java   |  11 +-
 .../java/org/apache/iotdb/db/utils/MemUtils.java   |   9 +-
 .../plan/planner/FragmentInstanceSerdeTest.java    |   1 +
 .../commons/consensus/index/ProgressIndexType.java |   8 +
 .../index/impl/TimeWindowStateProgressIndex.java   | 280 ++++++++
 .../commons/pipe/agent/plugin/PipePluginAgent.java |  88 ++-
 .../pipe/agent/plugin/PipePluginConstructor.java   |   2 +-
 .../config/constant/PipeProcessorConstant.java     |  20 +
 .../env/PipeTaskProcessorRuntimeEnvironment.java   |  12 +-
 .../pipe/connector/client/IoTDBSyncClient.java     |   2 +-
 .../iotdb/commons/pipe/event/EnrichedEvent.java    |  14 +-
 .../pipe/plugin/builtin/BuiltinPipePlugin.java     |  11 +
 .../processor/aggregate/AggregateProcessor.java}   |  15 +-
 .../aggregate/StandardStatisticsProcessor.java}    |  15 +-
 .../aggregate/TumblingWindowingProcessor.java}     |  15 +-
 .../commons/pipe/receiver/IoTDBFileReceiver.java   |   2 +-
 .../commons/pipe/task/meta/PipeMetaDeSerTest.java  |  20 +
 .../org/apache/iotdb/tsfile/read/TsFileReader.java |   2 +-
 .../iotdb/tsfile/read/TsFileSequenceReader.java    |   2 +-
 .../iotdb/tsfile/utils/ReadWriteIOUtils.java       |  48 +-
 .../apache/iotdb/tsfile/write/record/Tablet.java   |  70 +-
 103 files changed, 4909 insertions(+), 428 deletions(-)

diff --git a/example/pipe-opc-ua-sink/src/main/java/org/apache/iotdb/opcua/ClientExampleRunner.java b/example/pipe-opc-ua-sink/src/main/java/org/apache/iotdb/opcua/ClientExampleRunner.java
index b9a90c2fee9..599afd23ec2 100644
--- a/example/pipe-opc-ua-sink/src/main/java/org/apache/iotdb/opcua/ClientExampleRunner.java
+++ b/example/pipe-opc-ua-sink/src/main/java/org/apache/iotdb/opcua/ClientExampleRunner.java
@@ -54,22 +54,23 @@ public class ClientExampleRunner {
   }
 
   private OpcUaClient createClient() throws Exception {
-    Path securityTempDir = Paths.get(System.getProperty("java.io.tmpdir"), "client", "security");
+    final Path securityTempDir =
+        Paths.get(System.getProperty("java.io.tmpdir"), "client", "security");
     Files.createDirectories(securityTempDir);
     if (!Files.exists(securityTempDir)) {
       throw new Exception("unable to create security dir: " + securityTempDir);
     }
 
-    File pkiDir = securityTempDir.resolve("pki").toFile();
+    final File pkiDir = securityTempDir.resolve("pki").toFile();
 
     System.out.println("security dir: " + securityTempDir.toAbsolutePath());
     LoggerFactory.getLogger(getClass()).info("security pki dir: {}", pkiDir.getAbsolutePath());
 
-    IoTDBKeyStoreLoaderClient loader = new IoTDBKeyStoreLoaderClient().load(securityTempDir);
+    final IoTDBKeyStoreLoaderClient loader = new IoTDBKeyStoreLoaderClient().load(securityTempDir);
 
-    DefaultTrustListManager trustListManager = new DefaultTrustListManager(pkiDir);
+    final DefaultTrustListManager trustListManager = new DefaultTrustListManager(pkiDir);
 
-    DefaultClientCertificateValidator certificateValidator =
+    final DefaultClientCertificateValidator certificateValidator =
         new DefaultClientCertificateValidator(trustListManager);
 
     return OpcUaClient.create(
@@ -90,7 +91,7 @@ public class ClientExampleRunner {
 
   public void run() {
     try {
-      OpcUaClient client = createClient();
+      final OpcUaClient client = createClient();
 
       future.whenCompleteAsync(
           (c, ex) -> {
diff --git a/example/pipe-opc-ua-sink/src/main/java/org/apache/iotdb/opcua/ClientTest.java b/example/pipe-opc-ua-sink/src/main/java/org/apache/iotdb/opcua/ClientTest.java
index 515de3ba24b..cc09a7dc7bb 100644
--- a/example/pipe-opc-ua-sink/src/main/java/org/apache/iotdb/opcua/ClientTest.java
+++ b/example/pipe-opc-ua-sink/src/main/java/org/apache/iotdb/opcua/ClientTest.java
@@ -47,7 +47,7 @@ import static org.eclipse.milo.opcua.stack.core.types.builtin.unsigned.Unsigned.
 public class ClientTest implements ClientExample {
 
   public static void main(String[] args) {
-    ClientTest example = new ClientTest();
+    final ClientTest example = new ClientTest();
 
     new ClientExampleRunner(example).run();
   }
@@ -60,16 +60,17 @@ public class ClientTest implements ClientExample {
     client.connect().get();
 
     // create a subscription and a monitored item
-    UaSubscription subscription = client.getSubscriptionManager().createSubscription(200.0).get();
+    final UaSubscription subscription =
+        client.getSubscriptionManager().createSubscription(200.0).get();
 
-    ReadValueId readValueId =
+    final ReadValueId readValueId =
         new ReadValueId(
             Identifiers.Server, AttributeId.EventNotifier.uid(), null, QualifiedName.NULL_VALUE);
 
     // client handle must be unique per item
-    UInteger clientHandle = uint(clientHandles.getAndIncrement());
+    final UInteger clientHandle = uint(clientHandles.getAndIncrement());
 
-    EventFilter eventFilter =
+    final EventFilter eventFilter =
         new EventFilter(
             new SimpleAttributeOperand[] {
               new SimpleAttributeOperand(
@@ -95,7 +96,7 @@ public class ClientTest implements ClientExample {
             },
             new ContentFilter(null));
 
-    MonitoringParameters parameters =
+    final MonitoringParameters parameters =
         new MonitoringParameters(
             clientHandle,
             0.0,
@@ -103,16 +104,16 @@ public class ClientTest implements ClientExample {
             uint(10000),
             true);
 
-    MonitoredItemCreateRequest request =
+    final MonitoredItemCreateRequest request =
         new MonitoredItemCreateRequest(readValueId, MonitoringMode.Reporting, parameters);
 
-    List<UaMonitoredItem> items =
+    final List<UaMonitoredItem> items =
         subscription
             .createMonitoredItems(TimestampsToReturn.Both, Collections.singletonList(request))
             .get();
 
     // do something with the value updates
-    UaMonitoredItem monitoredItem = items.get(0);
+    final UaMonitoredItem monitoredItem = items.get(0);
 
     final AtomicInteger eventCount = new AtomicInteger(0);
 
diff --git a/example/pipe-opc-ua-sink/src/main/java/org/apache/iotdb/opcua/IoTDBKeyStoreLoaderClient.java b/example/pipe-opc-ua-sink/src/main/java/org/apache/iotdb/opcua/IoTDBKeyStoreLoaderClient.java
index da1d4182e31..e3245994493 100644
--- a/example/pipe-opc-ua-sink/src/main/java/org/apache/iotdb/opcua/IoTDBKeyStoreLoaderClient.java
+++ b/example/pipe-opc-ua-sink/src/main/java/org/apache/iotdb/opcua/IoTDBKeyStoreLoaderClient.java
@@ -49,18 +49,18 @@ class IoTDBKeyStoreLoaderClient {
   private KeyPair clientKeyPair;
 
   IoTDBKeyStoreLoaderClient load(Path baseDir) throws Exception {
-    KeyStore keyStore = KeyStore.getInstance("PKCS12");
+    final KeyStore keyStore = KeyStore.getInstance("PKCS12");
 
-    Path serverKeyStore = baseDir.resolve("example-client.pfx");
+    final Path serverKeyStore = baseDir.resolve("example-client.pfx");
 
     System.out.println("Loading KeyStore at " + serverKeyStore);
 
     if (!Files.exists(serverKeyStore)) {
       keyStore.load(null, PASSWORD);
 
-      KeyPair keyPair = SelfSignedCertificateGenerator.generateRsaKeyPair(2048);
+      final KeyPair keyPair = SelfSignedCertificateGenerator.generateRsaKeyPair(2048);
 
-      SelfSignedCertificateBuilder builder =
+      final SelfSignedCertificateBuilder builder =
           new SelfSignedCertificateBuilder(keyPair)
               .setCommonName("Eclipse Milo Example Client")
               .setOrganization("digitalpetri")
@@ -81,7 +81,7 @@ class IoTDBKeyStoreLoaderClient {
         }
       }
 
-      X509Certificate certificate = builder.build();
+      final X509Certificate certificate = builder.build();
 
       keyStore.setKeyEntry(
           CLIENT_ALIAS, keyPair.getPrivate(), PASSWORD, new X509Certificate[] {certificate});
@@ -94,7 +94,7 @@ class IoTDBKeyStoreLoaderClient {
       }
     }
 
-    Key clientPrivateKey = keyStore.getKey(CLIENT_ALIAS, PASSWORD);
+    final Key clientPrivateKey = keyStore.getKey(CLIENT_ALIAS, PASSWORD);
     if (clientPrivateKey instanceof PrivateKey) {
       clientCertificate = (X509Certificate) keyStore.getCertificate(CLIENT_ALIAS);
 
@@ -103,7 +103,7 @@ class IoTDBKeyStoreLoaderClient {
               .map(X509Certificate.class::cast)
               .toArray(X509Certificate[]::new);
 
-      PublicKey serverPublicKey = clientCertificate.getPublicKey();
+      final PublicKey serverPublicKey = clientCertificate.getPublicKey();
       clientKeyPair = new KeyPair(serverPublicKey, (PrivateKey) clientPrivateKey);
     }
 
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/AbstractPipeSingleIT.java
similarity index 60%
copy from iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java
copy to integration-test/src/test/java/org/apache/iotdb/pipe/it/single/AbstractPipeSingleIT.java
index 47314d30a0b..cadac3cc7a3 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java
+++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/AbstractPipeSingleIT.java
@@ -17,11 +17,28 @@
  * under the License.
  */
 
-package org.apache.iotdb.commons.pipe.config.plugin.env;
+package org.apache.iotdb.pipe.it.single;
 
-public class PipeTaskProcessorRuntimeEnvironment extends PipeTaskRuntimeEnvironment {
+import org.apache.iotdb.it.env.MultiEnvFactory;
+import org.apache.iotdb.itbase.env.BaseEnv;
 
-  public PipeTaskProcessorRuntimeEnvironment(String pipeName, long creationTime, int regionId) {
-    super(pipeName, creationTime, regionId);
+import org.junit.After;
+import org.junit.Before;
+
+abstract class AbstractPipeSingleIT {
+
+  protected BaseEnv env;
+
+  @Before
+  public void setUp() {
+    MultiEnvFactory.createEnv(2);
+    env = MultiEnvFactory.getEnv(0);
+    env.getConfig().getCommonConfig().setAutoCreateSchemaEnabled(true);
+    env.initClusterEnvironment();
+  }
+
+  @After
+  public final void tearDown() {
+    env.cleanClusterEnvironment();
   }
 }
diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/IoTDBPipeAggregateIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/IoTDBPipeAggregateIT.java
new file mode 100644
index 00000000000..5c2394e0211
--- /dev/null
+++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/IoTDBPipeAggregateIT.java
@@ -0,0 +1,121 @@
+/*
+ * 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.pipe.it.single;
+
+import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient;
+import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq;
+import org.apache.iotdb.db.it.utils.TestUtils;
+import org.apache.iotdb.it.framework.IoTDBTestRunner;
+import org.apache.iotdb.itbase.category.MultiClusterIT1;
+import org.apache.iotdb.rpc.TSStatusCode;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+@RunWith(IoTDBTestRunner.class)
+@Category({MultiClusterIT1.class})
+public class IoTDBPipeAggregateIT extends AbstractPipeSingleIT {
+  @Test
+  public void testAggregator() throws Exception {
+    try (SyncConfigNodeIServiceClient client =
+        (SyncConfigNodeIServiceClient) env.getLeaderConfigNodeConnection()) {
+      // Test the mixture of historical and realtime data
+      // Do not fail if the failure has nothing to do with pipe
+      // Because the failures will randomly generate due to resource limitation
+      if (!TestUtils.tryExecuteNonQueriesWithRetry(
+          env,
+          Arrays.asList(
+              "create timeseries root.ln.wf01.wt01.temperature with datatype=FLOAT, encoding=RLE, compression=SNAPPY tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)",
+              "create timeseries root.ln.wf01.wt01.status with datatype=BOOLEAN, encoding=RLE, compression=SNAPPY",
+              "insert into root.ln.wf01.wt01(time, temperature, status) values (10000, 1, false)"))) {
+        return;
+      }
+
+      Map<String, String> extractorAttributes = new HashMap<>();
+      Map<String, String> processorAttributes = new HashMap<>();
+      Map<String, String> connectorAttributes = new HashMap<>();
+
+      extractorAttributes.put("pattern", "root.ln");
+
+      processorAttributes.put("processor", "aggregate-processor");
+      processorAttributes.put("output.database", "root.testdb");
+      processorAttributes.put(
+          "output.measurements", "Avg1, peak1, rms1, var1, skew1, kurt1, ff1, cf1, pf1");
+      processorAttributes.put("operators", "avg, peak, rms, var, skew, kurt, ff, cf, pf, cE");
+      processorAttributes.put("sliding.seconds", "60");
+
+      connectorAttributes.put("sink", "write-back-sink");
+
+      TSStatus status =
+          client.createPipe(
+              new TCreatePipeReq("testPipe", connectorAttributes)
+                  .setExtractorAttributes(extractorAttributes)
+                  .setProcessorAttributes(processorAttributes));
+      Assert.assertEquals(TSStatusCode.SUCCESS_STATUS.getStatusCode(), status.getCode());
+
+      if (!TestUtils.tryExecuteNonQueriesWithRetry(
+          env,
+          Arrays.asList(
+              "insert into root.ln.wf01.wt01(time, temperature, status) values (20000, 2, true)",
+              "insert into root.ln.wf01.wt01(time, temperature, status) values (30000, 3, false)",
+              "insert into root.ln.wf01.wt01(time, temperature, status) values (40000, 4, true)",
+              "insert into root.ln.wf01.wt01(time, temperature, status) values (50000, 5, false)",
+              "insert into root.ln.wf01.wt01(time, temperature, status) values (60000, 6, true)",
+              "insert into root.ln.wf01.wt01(time, temperature, status) values (70000, 7, false)",
+              "insert into root.ln.wf01.wt01(time, temperature, status) values (80000, 8, true)",
+              "insert into root.ln.wf01.wt01(time, temperature, status) values (90000, 9, false)",
+              "insert into root.ln.wf01.wt01(time, temperature, status) values (100000, 10, true)",
+              "insert into root.ln.wf01.wt01(time, temperature, status) values (110000, 11, false)",
+              "insert into root.ln.wf01.wt01(time, temperature, status) values (120000, 12, false)",
+              "flush"))) {
+        return;
+      }
+
+      // Test total number
+      TestUtils.assertDataEventuallyOnEnv(
+          env,
+          "select count(*) from root.testdb.** group by level=1",
+          "count(root.testdb.*.*.*.*),",
+          Collections.singleton("20,"));
+
+      // Test manually renamed timeSeries count
+      TestUtils.assertDataEventuallyOnEnv(
+          env,
+          "select count(Avg1) from root.testdb.wf01.wt01.temperature",
+          "count(root.testdb.wf01.wt01.temperature.Avg1),",
+          Collections.singleton("2,"));
+
+      // Test default renamed timeSeries count
+      TestUtils.assertDataEventuallyOnEnv(
+          env,
+          "select count(cE) from root.testdb.wf01.wt01.temperature",
+          "count(root.testdb.wf01.wt01.temperature.cE),",
+          Collections.singleton("2,"));
+    }
+  }
+}
diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/IoTDBPipeSingleEnvDemoIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/IoTDBPipeSingleEnvDemoIT.java
similarity index 81%
rename from integration-test/src/test/java/org/apache/iotdb/pipe/it/IoTDBPipeSingleEnvDemoIT.java
rename to integration-test/src/test/java/org/apache/iotdb/pipe/it/single/IoTDBPipeSingleEnvDemoIT.java
index 9e85c3bcf0f..184249b3069 100644
--- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/IoTDBPipeSingleEnvDemoIT.java
+++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/IoTDBPipeSingleEnvDemoIT.java
@@ -17,19 +17,16 @@
  * under the License.
  */
 
-package org.apache.iotdb.pipe.it;
+package org.apache.iotdb.pipe.it.single;
 
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.client.sync.SyncConfigNodeIServiceClient;
 import org.apache.iotdb.confignode.rpc.thrift.TCreatePipeReq;
-import org.apache.iotdb.it.env.MultiEnvFactory;
 import org.apache.iotdb.it.framework.IoTDBTestRunner;
 import org.apache.iotdb.itbase.category.MultiClusterIT1;
 import org.apache.iotdb.rpc.TSStatusCode;
 
-import org.junit.After;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -39,22 +36,11 @@ import java.util.Map;
 
 @RunWith(IoTDBTestRunner.class)
 @Category({MultiClusterIT1.class})
-public class IoTDBPipeSingleEnvDemoIT {
-  @Before
-  public void setUp() throws Exception {
-    MultiEnvFactory.createEnv(1);
-    MultiEnvFactory.getEnv(0).initClusterEnvironment(1, 1);
-  }
-
-  @After
-  public void tearDown() {
-    MultiEnvFactory.getEnv(0).cleanClusterEnvironment();
-  }
-
+public class IoTDBPipeSingleEnvDemoIT extends AbstractPipeSingleIT {
   @Test
   public void testSingleEnv() throws Exception {
     try (SyncConfigNodeIServiceClient client =
-        (SyncConfigNodeIServiceClient) MultiEnvFactory.getEnv(0).getLeaderConfigNodeConnection()) {
+        (SyncConfigNodeIServiceClient) env.getLeaderConfigNodeConnection()) {
       Map<String, String> extractorAttributes = new HashMap<>();
       Map<String, String> processorAttributes = new HashMap<>();
       Map<String, String> connectorAttributes = new HashMap<>();
diff --git a/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/customizer/parameter/PipeParameters.java b/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/customizer/parameter/PipeParameters.java
index 9077920726c..87461019a55 100644
--- a/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/customizer/parameter/PipeParameters.java
+++ b/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/customizer/parameter/PipeParameters.java
@@ -302,6 +302,25 @@ public class PipeParameters {
     return this;
   }
 
+  /**
+   * This method clones a new {@link PipeParameters} with equivalent attributes in this {@link
+   * PipeParameters} added by (non-existed) or replaces with (existed) those from another {@link
+   * PipeParameters}.
+   *
+   * @param that provide the key that needs to be updated along with the value
+   * @return this {@link PipeParameters}
+   */
+  public PipeParameters addOrReplaceEquivalentAttributesWithClone(PipeParameters that) {
+    Map<String, String> thisMap =
+        this.attributes.entrySet().stream()
+            .collect(Collectors.toMap(entry -> KeyReducer.reduce(entry.getKey()), Entry::getValue));
+    Map<String, String> thatMap =
+        that.attributes.entrySet().stream()
+            .collect(Collectors.toMap(entry -> KeyReducer.reduce(entry.getKey()), Entry::getValue));
+    thisMap.putAll(thatMap);
+    return new PipeParameters(thisMap);
+  }
+
   private static class KeyReducer {
 
     private static final Set<String> PREFIXES = new HashSet<>();
diff --git a/iotdb-connector/flink-sql-iotdb-connector/src/main/java/org/apache/iotdb/flink/sql/client/IoTDBWebSocketClient.java b/iotdb-connector/flink-sql-iotdb-connector/src/main/java/org/apache/iotdb/flink/sql/client/IoTDBWebSocketClient.java
index 768ba986d73..bec3171119f 100644
--- a/iotdb-connector/flink-sql-iotdb-connector/src/main/java/org/apache/iotdb/flink/sql/client/IoTDBWebSocketClient.java
+++ b/iotdb-connector/flink-sql-iotdb-connector/src/main/java/org/apache/iotdb/flink/sql/client/IoTDBWebSocketClient.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.flink.sql.client;
 
 import org.apache.iotdb.flink.sql.function.IoTDBCDCSourceFunction;
@@ -52,7 +53,7 @@ public class IoTDBWebSocketClient extends WebSocketClient {
 
   @Override
   public void onOpen(ServerHandshake serverHandshake) {
-    String log =
+    final String log =
         String.format("The connection with %s:%d has been created.", uri.getHost(), uri.getPort());
     LOGGER.info(log);
   }
@@ -69,8 +70,8 @@ public class IoTDBWebSocketClient extends WebSocketClient {
   @Override
   public void onMessage(ByteBuffer bytes) {
     super.onMessage(bytes);
-    long commitId = bytes.getLong();
-    Tablet tablet = Tablet.deserialize(bytes);
+    final long commitId = bytes.getLong();
+    final Tablet tablet = Tablet.deserialize(bytes);
     function.addTabletWrapper(new TabletWrapper(commitId, this, tablet));
   }
 
@@ -82,7 +83,7 @@ public class IoTDBWebSocketClient extends WebSocketClient {
 
   @Override
   public void onError(Exception e) {
-    String log =
+    final String log =
         String.format(
             "An error occurred when connecting to %s:%s: %s.",
             uri.getHost(), uri.getPort(), e.getMessage());
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java
index e5a6ce7ec35..ad3ae6b5b92 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java
@@ -509,7 +509,7 @@ public class ProcedureManager {
    * AddConfigNodeProcedure}s.
    */
   public void addConfigNode(TConfigNodeRegisterReq req) {
-    AddConfigNodeProcedure addConfigNodeProcedure =
+    final AddConfigNodeProcedure addConfigNodeProcedure =
         new AddConfigNodeProcedure(req.getConfigNodeLocation(), req.getVersionInfo());
     this.executor.submitProcedure(addConfigNodeProcedure);
   }
@@ -519,7 +519,7 @@ public class ProcedureManager {
    * RemoveConfigNodeProcedure}s.
    */
   public void removeConfigNode(RemoveConfigNodePlan removeConfigNodePlan) {
-    RemoveConfigNodeProcedure removeConfigNodeProcedure =
+    final RemoveConfigNodeProcedure removeConfigNodeProcedure =
         new RemoveConfigNodeProcedure(removeConfigNodePlan.getConfigNodeLocation());
     this.executor.submitProcedure(removeConfigNodeProcedure);
     LOGGER.info("Submit RemoveConfigNodeProcedure successfully: {}", removeConfigNodePlan);
@@ -541,7 +541,8 @@ public class ProcedureManager {
   }
 
   public TSStatus migrateRegion(TMigrateRegionReq migrateRegionReq) {
-    TConsensusGroupId regionGroupId;
+    final TConsensusGroupId regionGroupId;
+    final TSStatus status;
     if (configManager
         .getPartitionManager()
         .isRegionGroupExists(
@@ -560,7 +561,7 @@ public class ProcedureManager {
       LOGGER.warn(
           "Submit RegionMigrateProcedure failed, because RegionGroup: {} doesn't exist",
           migrateRegionReq.getRegionId());
-      TSStatus status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
+      status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
       status.setMessage(
           String.format(
               "Submit RegionMigrateProcedure failed, because RegionGroup: %s doesn't exist",
@@ -568,12 +569,12 @@ public class ProcedureManager {
       return status;
     }
 
-    TDataNodeLocation originalDataNode =
+    final TDataNodeLocation originalDataNode =
         configManager
             .getNodeManager()
             .getRegisteredDataNode(migrateRegionReq.getFromId())
             .getLocation();
-    TDataNodeLocation destDataNode =
+    final TDataNodeLocation destDataNode =
         configManager
             .getNodeManager()
             .getRegisteredDataNode(migrateRegionReq.getToId())
@@ -583,7 +584,7 @@ public class ProcedureManager {
       LOGGER.warn(
           "Submit RegionMigrateProcedure failed, because no original DataNode {}",
           migrateRegionReq.getFromId());
-      TSStatus status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
+      status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
       status.setMessage(
           "Submit RegionMigrateProcedure failed, because no original DataNode "
               + migrateRegionReq.getFromId());
@@ -592,7 +593,7 @@ public class ProcedureManager {
       LOGGER.warn(
           "Submit RegionMigrateProcedure failed, because no target DataNode {}",
           migrateRegionReq.getToId());
-      TSStatus status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
+      status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
       status.setMessage(
           "Submit RegionMigrateProcedure failed, because no target DataNode "
               + migrateRegionReq.getToId());
@@ -604,7 +605,7 @@ public class ProcedureManager {
           "Submit RegionMigrateProcedure failed, because the original DataNode {} doesn't contain Region {}",
           migrateRegionReq.getFromId(),
           migrateRegionReq.getRegionId());
-      TSStatus status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
+      status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
       status.setMessage(
           "Submit RegionMigrateProcedure failed, because the original DataNode "
               + migrateRegionReq.getFromId()
@@ -618,7 +619,7 @@ public class ProcedureManager {
           "Submit RegionMigrateProcedure failed, because the target DataNode {} already contains Region {}",
           migrateRegionReq.getToId(),
           migrateRegionReq.getRegionId());
-      TSStatus status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
+      status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
       status.setMessage(
           "Submit RegionMigrateProcedure failed, because the target DataNode "
               + migrateRegionReq.getToId()
@@ -628,7 +629,7 @@ public class ProcedureManager {
     }
     // Here we only check Running DataNode to implement migration, because removing nodes may not
     // exist when add peer is performing
-    Set<Integer> aliveDataNodes =
+    final Set<Integer> aliveDataNodes =
         configManager.getNodeManager().filterDataNodeThroughStatus(NodeStatus.Running).stream()
             .map(TDataNodeConfiguration::getLocation)
             .map(TDataNodeLocation::getDataNodeId)
@@ -638,7 +639,7 @@ public class ProcedureManager {
       LOGGER.warn(
           "Submit RegionMigrateProcedure failed, because the sourceDataNode {} is Unknown.",
           migrateRegionReq.getFromId());
-      TSStatus status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
+      status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
       status.setMessage(
           "Submit RegionMigrateProcedure failed, because the sourceDataNode "
               + migrateRegionReq.getFromId()
@@ -650,7 +651,7 @@ public class ProcedureManager {
       LOGGER.warn(
           "Submit RegionMigrateProcedure failed, because the destDataNode {} is ReadOnly or Unknown.",
           migrateRegionReq.getToId());
-      TSStatus status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
+      status = new TSStatus(TSStatusCode.MIGRATE_REGION_ERROR.getStatusCode());
       status.setMessage(
           "Submit RegionMigrateProcedure failed, because the destDataNode "
               + migrateRegionReq.getToId()
@@ -675,11 +676,11 @@ public class ProcedureManager {
    */
   public TSStatus createRegionGroups(
       TConsensusGroupType consensusGroupType, CreateRegionGroupsPlan createRegionGroupsPlan) {
-    long procedureId =
+    final long procedureId =
         executor.submitProcedure(
             new CreateRegionGroupsProcedure(consensusGroupType, createRegionGroupsPlan));
-    List<TSStatus> statusList = new ArrayList<>();
-    boolean isSucceed =
+    final List<TSStatus> statusList = new ArrayList<>();
+    final boolean isSucceed =
         waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
     if (isSucceed) {
       return RpcUtils.SUCCESS_STATUS;
@@ -713,9 +714,9 @@ public class ProcedureManager {
           .setMessage(e.getMessage());
     }
 
-    long procedureId = executor.submitProcedure(createTriggerProcedure);
-    List<TSStatus> statusList = new ArrayList<>();
-    boolean isSucceed =
+    final long procedureId = executor.submitProcedure(createTriggerProcedure);
+    final List<TSStatus> statusList = new ArrayList<>();
+    final boolean isSucceed =
         waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
     if (isSucceed) {
       return RpcUtils.SUCCESS_STATUS;
@@ -734,8 +735,8 @@ public class ProcedureManager {
   public TSStatus dropTrigger(String triggerName, boolean isGeneratedByPipe) {
     long procedureId =
         executor.submitProcedure(new DropTriggerProcedure(triggerName, isGeneratedByPipe));
-    List<TSStatus> statusList = new ArrayList<>();
-    boolean isSucceed =
+    final List<TSStatus> statusList = new ArrayList<>();
+    final boolean isSucceed =
         waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
     if (isSucceed) {
       return RpcUtils.SUCCESS_STATUS;
@@ -746,8 +747,9 @@ public class ProcedureManager {
   }
 
   public TSStatus createCQ(TCreateCQReq req, ScheduledExecutorService scheduledExecutor) {
-    long procedureId = executor.submitProcedure(new CreateCQProcedure(req, scheduledExecutor));
-    List<TSStatus> statusList = new ArrayList<>();
+    final long procedureId =
+        executor.submitProcedure(new CreateCQProcedure(req, scheduledExecutor));
+    final List<TSStatus> statusList = new ArrayList<>();
     waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
     return statusList.get(0);
   }
@@ -770,9 +772,9 @@ public class ProcedureManager {
           .setMessage(e.getMessage());
     }
 
-    long procedureId = executor.submitProcedure(createPipePluginProcedure);
-    List<TSStatus> statusList = new ArrayList<>();
-    boolean isSucceed =
+    final long procedureId = executor.submitProcedure(createPipePluginProcedure);
+    final List<TSStatus> statusList = new ArrayList<>();
+    final boolean isSucceed =
         waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
     if (isSucceed) {
       return RpcUtils.SUCCESS_STATUS;
@@ -783,9 +785,9 @@ public class ProcedureManager {
   }
 
   public TSStatus dropPipePlugin(String pluginName) {
-    long procedureId = executor.submitProcedure(new DropPipePluginProcedure(pluginName));
-    List<TSStatus> statusList = new ArrayList<>();
-    boolean isSucceed =
+    final long procedureId = executor.submitProcedure(new DropPipePluginProcedure(pluginName));
+    final List<TSStatus> statusList = new ArrayList<>();
+    final boolean isSucceed =
         waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
     if (isSucceed) {
       return RpcUtils.SUCCESS_STATUS;
@@ -797,9 +799,9 @@ public class ProcedureManager {
 
   public TSStatus createPipe(TCreatePipeReq req) {
     try {
-      long procedureId = executor.submitProcedure(new CreatePipeProcedureV2(req));
-      List<TSStatus> statusList = new ArrayList<>();
-      boolean isSucceed =
+      final long procedureId = executor.submitProcedure(new CreatePipeProcedureV2(req));
+      final List<TSStatus> statusList = new ArrayList<>();
+      final boolean isSucceed =
           waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
       if (isSucceed) {
         return statusList.get(0);
@@ -814,9 +816,9 @@ public class ProcedureManager {
 
   public TSStatus alterPipe(TAlterPipeReq req) {
     try {
-      long procedureId = executor.submitProcedure(new AlterPipeProcedureV2(req));
-      List<TSStatus> statusList = new ArrayList<>();
-      boolean isSucceed =
+      final long procedureId = executor.submitProcedure(new AlterPipeProcedureV2(req));
+      final List<TSStatus> statusList = new ArrayList<>();
+      final boolean isSucceed =
           waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
       if (isSucceed) {
         return statusList.get(0);
@@ -831,9 +833,9 @@ public class ProcedureManager {
 
   public TSStatus startPipe(String pipeName) {
     try {
-      long procedureId = executor.submitProcedure(new StartPipeProcedureV2(pipeName));
-      List<TSStatus> statusList = new ArrayList<>();
-      boolean isSucceed =
+      final long procedureId = executor.submitProcedure(new StartPipeProcedureV2(pipeName));
+      final List<TSStatus> statusList = new ArrayList<>();
+      final boolean isSucceed =
           waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
       if (isSucceed) {
         return statusList.get(0);
@@ -848,9 +850,9 @@ public class ProcedureManager {
 
   public TSStatus stopPipe(String pipeName) {
     try {
-      long procedureId = executor.submitProcedure(new StopPipeProcedureV2(pipeName));
-      List<TSStatus> statusList = new ArrayList<>();
-      boolean isSucceed =
+      final long procedureId = executor.submitProcedure(new StopPipeProcedureV2(pipeName));
+      final List<TSStatus> statusList = new ArrayList<>();
+      final boolean isSucceed =
           waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
       if (isSucceed) {
         return statusList.get(0);
@@ -865,9 +867,9 @@ public class ProcedureManager {
 
   public TSStatus dropPipe(String pipeName) {
     try {
-      long procedureId = executor.submitProcedure(new DropPipeProcedureV2(pipeName));
-      List<TSStatus> statusList = new ArrayList<>();
-      boolean isSucceed =
+      final long procedureId = executor.submitProcedure(new DropPipeProcedureV2(pipeName));
+      final List<TSStatus> statusList = new ArrayList<>();
+      final boolean isSucceed =
           waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
       if (isSucceed) {
         return statusList.get(0);
@@ -945,9 +947,9 @@ public class ProcedureManager {
 
   public TSStatus createTopic(TCreateTopicReq req) {
     try {
-      long procedureId = executor.submitProcedure(new CreateTopicProcedure(req));
-      List<TSStatus> statusList = new ArrayList<>();
-      boolean isSucceed =
+      final long procedureId = executor.submitProcedure(new CreateTopicProcedure(req));
+      final List<TSStatus> statusList = new ArrayList<>();
+      final boolean isSucceed =
           waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
       if (isSucceed) {
         return statusList.get(0);
@@ -963,9 +965,9 @@ public class ProcedureManager {
 
   public TSStatus dropTopic(String topicName) {
     try {
-      long procedureId = executor.submitProcedure(new DropTopicProcedure(topicName));
-      List<TSStatus> statusList = new ArrayList<>();
-      boolean isSucceed =
+      final long procedureId = executor.submitProcedure(new DropTopicProcedure(topicName));
+      final List<TSStatus> statusList = new ArrayList<>();
+      final boolean isSucceed =
           waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
       if (isSucceed) {
         return statusList.get(0);
@@ -980,9 +982,9 @@ public class ProcedureManager {
 
   public TSStatus createConsumer(TCreateConsumerReq req) {
     try {
-      long procedureId = executor.submitProcedure(new CreateConsumerProcedure(req));
-      List<TSStatus> statusList = new ArrayList<>();
-      boolean isSucceed =
+      final long procedureId = executor.submitProcedure(new CreateConsumerProcedure(req));
+      final List<TSStatus> statusList = new ArrayList<>();
+      final boolean isSucceed =
           waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
       if (isSucceed) {
         return statusList.get(0);
@@ -998,9 +1000,9 @@ public class ProcedureManager {
 
   public TSStatus dropConsumer(TCloseConsumerReq req) {
     try {
-      long procedureId = executor.submitProcedure(new DropConsumerProcedure(req));
-      List<TSStatus> statusList = new ArrayList<>();
-      boolean isSucceed =
+      final long procedureId = executor.submitProcedure(new DropConsumerProcedure(req));
+      final List<TSStatus> statusList = new ArrayList<>();
+      final boolean isSucceed =
           waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
       if (isSucceed) {
         return statusList.get(0);
@@ -1016,9 +1018,9 @@ public class ProcedureManager {
 
   public TSStatus createSubscription(TSubscribeReq req) {
     try {
-      long procedureId = executor.submitProcedure(new CreateSubscriptionProcedure(req));
-      List<TSStatus> statusList = new ArrayList<>();
-      boolean isSucceed =
+      final long procedureId = executor.submitProcedure(new CreateSubscriptionProcedure(req));
+      final List<TSStatus> statusList = new ArrayList<>();
+      final boolean isSucceed =
           waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
       if (isSucceed) {
         return statusList.get(0);
@@ -1034,9 +1036,9 @@ public class ProcedureManager {
 
   public TSStatus dropSubscription(TUnsubscribeReq req) {
     try {
-      long procedureId = executor.submitProcedure(new DropSubscriptionProcedure(req));
-      List<TSStatus> statusList = new ArrayList<>();
-      boolean isSucceed =
+      final long procedureId = executor.submitProcedure(new DropSubscriptionProcedure(req));
+      final List<TSStatus> statusList = new ArrayList<>();
+      final boolean isSucceed =
           waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
       if (isSucceed) {
         return statusList.get(0);
@@ -1055,8 +1057,8 @@ public class ProcedureManager {
     try {
       final long procedureId =
           executor.submitProcedure(new AuthOperationProcedure(authorPlan, dns, isGeneratedByPipe));
-      List<TSStatus> statusList = new ArrayList<>();
-      boolean isSucceed =
+      final List<TSStatus> statusList = new ArrayList<>();
+      final boolean isSucceed =
           waitingProcedureFinished(Collections.singletonList(procedureId), statusList);
       if (isSucceed) {
         return RpcUtils.SUCCESS_STATUS;
@@ -1079,13 +1081,13 @@ public class ProcedureManager {
   private boolean waitingProcedureFinished(List<Long> procedureIds, List<TSStatus> statusList) {
     boolean isSucceed = true;
     for (long procedureId : procedureIds) {
-      long startTimeForCurrentProcedure = System.currentTimeMillis();
+      final long startTimeForCurrentProcedure = System.currentTimeMillis();
       while (executor.isRunning()
           && !executor.isFinished(procedureId)
           && System.currentTimeMillis() - startTimeForCurrentProcedure < PROCEDURE_WAIT_TIME_OUT) {
         sleepWithoutInterrupt(PROCEDURE_WAIT_RETRY_TIMEOUT);
       }
-      Procedure<ConfigNodeProcedureEnv> finishedProcedure =
+      final Procedure<ConfigNodeProcedureEnv> finishedProcedure =
           executor.getResultOrProcedure(procedureId);
       if (!finishedProcedure.isFinished()) {
         // the procedure is still executing
@@ -1105,7 +1107,7 @@ public class ProcedureManager {
         }
       } else {
         if (finishedProcedure.getException().getCause() instanceof IoTDBException) {
-          IoTDBException e = (IoTDBException) finishedProcedure.getException().getCause();
+          final IoTDBException e = (IoTDBException) finishedProcedure.getException().getCause();
           if (e instanceof BatchProcessException) {
             statusList.add(
                 RpcUtils.getStatus(
@@ -1127,7 +1129,7 @@ public class ProcedureManager {
 
   public static void sleepWithoutInterrupt(final long timeToSleep) {
     long currentTime = System.currentTimeMillis();
-    long endTime = timeToSleep + currentTime;
+    final long endTime = timeToSleep + currentTime;
     boolean interrupted = false;
     while (currentTime < endTime) {
       try {
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigNodePluginAgent.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigNodePluginAgent.java
index 6b936e45b62..d70a6825288 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigNodePluginAgent.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigNodePluginAgent.java
@@ -19,8 +19,10 @@
 
 package org.apache.iotdb.confignode.manager.pipe.transfer.agent.plugin;
 
+import org.apache.iotdb.commons.pipe.agent.plugin.PipeConnectorConstructor;
+import org.apache.iotdb.commons.pipe.agent.plugin.PipeExtractorConstructor;
 import org.apache.iotdb.commons.pipe.agent.plugin.PipePluginAgent;
-import org.apache.iotdb.commons.pipe.agent.plugin.PipePluginConstructor;
+import org.apache.iotdb.commons.pipe.agent.plugin.PipeProcessorConstructor;
 import org.apache.iotdb.commons.pipe.plugin.meta.PipePluginMetaKeeper;
 
 public class PipeConfigNodePluginAgent extends PipePluginAgent {
@@ -30,19 +32,19 @@ public class PipeConfigNodePluginAgent extends PipePluginAgent {
   }
 
   @Override
-  protected PipePluginConstructor createPipeExtractorConstructor(
+  protected PipeExtractorConstructor createPipeExtractorConstructor(
       PipePluginMetaKeeper pipePluginMetaKeeper) {
     return new PipeConfigRegionExtractorConstructor();
   }
 
   @Override
-  protected PipePluginConstructor createPipeProcessorConstructor(
+  protected PipeProcessorConstructor createPipeProcessorConstructor(
       PipePluginMetaKeeper pipePluginMetaKeeper) {
     return new PipeConfigRegionProcessorConstructor();
   }
 
   @Override
-  protected PipePluginConstructor createPipeConnectorConstructor(
+  protected PipeConnectorConstructor createPipeConnectorConstructor(
       PipePluginMetaKeeper pipePluginMetaKeeper) {
     return new PipeConfigRegionConnectorConstructor();
   }
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigRegionConnectorConstructor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigRegionConnectorConstructor.java
index c1c027eaa92..128f30bcbcc 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigRegionConnectorConstructor.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigRegionConnectorConstructor.java
@@ -67,7 +67,7 @@ class PipeConfigRegionConnectorConstructor extends PipeConnectorConstructor {
   }
 
   @Override
-  protected PipeConnector reflectPluginByKey(String pluginKey) {
+  public PipeConnector reflectPluginByKey(String pluginKey) {
     // TODO: support constructing plugin by reflection
     return (PipeConnector)
         pluginConstructors.getOrDefault(pluginKey, DoNothingConnector::new).get();
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigRegionExtractorConstructor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigRegionExtractorConstructor.java
index 0017651094d..41a93323016 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigRegionExtractorConstructor.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigRegionExtractorConstructor.java
@@ -41,7 +41,7 @@ class PipeConfigRegionExtractorConstructor extends PipeExtractorConstructor {
   }
 
   @Override
-  protected PipeExtractor reflectPluginByKey(String pluginKey) {
+  public PipeExtractor reflectPluginByKey(String pluginKey) {
     // TODO: support constructing plugin by reflection
     return (PipeExtractor)
         pluginConstructors.getOrDefault(pluginKey, DoNothingExtractor::new).get();
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigRegionProcessorConstructor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigRegionProcessorConstructor.java
index c1331176d53..31bd1b9a669 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigRegionProcessorConstructor.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/agent/plugin/PipeConfigRegionProcessorConstructor.java
@@ -33,7 +33,7 @@ class PipeConfigRegionProcessorConstructor extends PipeProcessorConstructor {
   }
 
   @Override
-  protected PipeProcessor reflectPluginByKey(String pluginKey) {
+  public PipeProcessor reflectPluginByKey(String pluginKey) {
     // TODO: support constructing plugin by reflection
     return (PipeProcessor)
         pluginConstructors.getOrDefault(pluginKey, DoNothingProcessor::new).get();
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/execution/PipeConfigNodeSubtask.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/execution/PipeConfigNodeSubtask.java
index 603b26ea9a6..66533122bf7 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/execution/PipeConfigNodeSubtask.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/transfer/execution/PipeConfigNodeSubtask.java
@@ -20,10 +20,12 @@
 package org.apache.iotdb.confignode.manager.pipe.transfer.execution;
 
 import org.apache.iotdb.commons.exception.pipe.PipeRuntimeException;
+import org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant;
 import org.apache.iotdb.commons.pipe.config.plugin.configuraion.PipeTaskRuntimeConfiguration;
 import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskExtractorRuntimeEnvironment;
 import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskRuntimeEnvironment;
 import org.apache.iotdb.commons.pipe.event.EnrichedEvent;
+import org.apache.iotdb.commons.pipe.plugin.builtin.BuiltinPipePlugin;
 import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta;
 import org.apache.iotdb.commons.pipe.task.subtask.PipeAbstractConnectorSubtask;
 import org.apache.iotdb.confignode.manager.pipe.transfer.agent.PipeConfigNodeAgent;
@@ -88,20 +90,21 @@ public class PipeConfigNodeSubtask extends PipeAbstractConnectorSubtask {
     extractor.customize(extractorParameters, runtimeConfiguration);
   }
 
-  private void initProcessor(Map<String, String> processorAttributes) throws Exception {
+  private void initProcessor(Map<String, String> processorAttributes) {
     final PipeParameters processorParameters = new PipeParameters(processorAttributes);
 
-    // 1. Construct processor
-    processor = PipeConfigNodeAgent.plugin().reflectProcessor(processorParameters);
-
-    // 2. Validate processor parameters
-    processor.validate(new PipeParameterValidator(processorParameters));
-
-    // 3. Customize processor
     final PipeTaskRuntimeConfiguration runtimeConfiguration =
         new PipeTaskRuntimeConfiguration(
             new PipeTaskRuntimeEnvironment(taskID, creationTime, CONFIG_REGION_ID.getId()));
-    processor.customize(processorParameters, runtimeConfiguration);
+
+    processor =
+        PipeConfigNodeAgent.plugin()
+            .getConfiguredProcessor(
+                processorParameters.getStringOrDefault(
+                    PipeProcessorConstant.PROCESSOR_KEY,
+                    BuiltinPipePlugin.DO_NOTHING_PROCESSOR.getPipePluginName()),
+                processorParameters,
+                runtimeConfiguration);
   }
 
   private void initConnector(Map<String, String> connectorAttributes) throws Exception {
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipePluginInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipePluginInfo.java
index 6c63739107e..382ebd51620 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipePluginInfo.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipePluginInfo.java
@@ -219,7 +219,7 @@ public class PipePluginInfo implements SnapshotProcessor {
 
   public JarResp getPipePluginJar(GetPipePluginJarPlan getPipePluginJarPlan) {
     try {
-      List<ByteBuffer> jarList = new ArrayList<>();
+      final List<ByteBuffer> jarList = new ArrayList<>();
       for (String jarName : getPipePluginJarPlan.getJarNames()) {
         jarList.add(
             ExecutableManager.transferToBytebuffer(
@@ -298,7 +298,7 @@ public class PipePluginInfo implements SnapshotProcessor {
     if (getClass() != obj.getClass()) {
       return false;
     }
-    PipePluginInfo other = (PipePluginInfo) obj;
+    final PipePluginInfo other = (PipePluginInfo) obj;
     return Objects.equals(pipePluginExecutableManager, other.pipePluginExecutableManager)
         && Objects.equals(pipePluginMetaKeeper, other.pipePluginMetaKeeper);
   }
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java
index b8818076051..9bf66eb84e7 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java
@@ -183,10 +183,10 @@ public class PipeTaskInfo implements SnapshotProcessor {
       throw new PipeException(exceptionMessage);
     }
 
-    PipeStaticMeta pipeStaticMetaFromCoordinator =
+    final PipeStaticMeta pipeStaticMetaFromCoordinator =
         getPipeMetaByPipeName(alterPipeRequest.getPipeName()).getStaticMeta();
     // deep copy current pipe static meta
-    PipeStaticMeta copiedPipeStaticMetaFromCoordinator =
+    final PipeStaticMeta copiedPipeStaticMetaFromCoordinator =
         new PipeStaticMeta(
             pipeStaticMetaFromCoordinator.getPipeName(),
             pipeStaticMetaFromCoordinator.getCreationTime(),
@@ -768,7 +768,7 @@ public class PipeTaskInfo implements SnapshotProcessor {
     if (obj == null || getClass() != obj.getClass()) {
       return false;
     }
-    PipeTaskInfo other = (PipeTaskInfo) obj;
+    final PipeTaskInfo other = (PipeTaskInfo) obj;
     return pipeMetaKeeper.equals(other.pipeMetaKeeper);
   }
 
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java
index be5ec0e9218..bbe342adad9 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java
@@ -167,7 +167,7 @@ public class CreatePipeProcedureV2 extends AbstractOperatePipeProcedureV2 {
     final String pipeName = createPipeRequest.getPipeName();
     LOGGER.info("CreatePipeProcedureV2: executeFromOperateOnDataNodes({})", pipeName);
 
-    String exceptionMessage =
+    final String exceptionMessage =
         parsePushPipeMetaExceptionForPipe(pipeName, pushSinglePipeMetaToDataNodes(pipeName, env));
     if (!exceptionMessage.isEmpty()) {
       LOGGER.warn(
@@ -220,7 +220,7 @@ public class CreatePipeProcedureV2 extends AbstractOperatePipeProcedureV2 {
         createPipeRequest.getPipeName());
 
     // Push all pipe metas to datanode, may be time-consuming
-    String exceptionMessage =
+    final String exceptionMessage =
         parsePushPipeMetaExceptionForPipe(
             createPipeRequest.getPipeName(), pushPipeMetaToDataNodes(env));
     if (!exceptionMessage.isEmpty()) {
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/DropPipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/DropPipeProcedureV2.java
index 5dd47bc9bbf..dc6f7afb91d 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/DropPipeProcedureV2.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/DropPipeProcedureV2.java
@@ -94,7 +94,7 @@ public class DropPipeProcedureV2 extends AbstractOperatePipeProcedureV2 {
   public void executeFromOperateOnDataNodes(ConfigNodeProcedureEnv env) {
     LOGGER.info("DropPipeProcedureV2: executeFromOperateOnDataNodes({})", pipeName);
 
-    String exceptionMessage =
+    final String exceptionMessage =
         parsePushPipeMetaExceptionForPipe(pipeName, dropSinglePipeOnDataNodes(pipeName, env));
     if (!exceptionMessage.isEmpty()) {
       LOGGER.warn(
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StartPipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StartPipeProcedureV2.java
index 606a0463927..6992b7be5d5 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StartPipeProcedureV2.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StartPipeProcedureV2.java
@@ -99,7 +99,7 @@ public class StartPipeProcedureV2 extends AbstractOperatePipeProcedureV2 {
   public void executeFromOperateOnDataNodes(ConfigNodeProcedureEnv env) throws IOException {
     LOGGER.info("StartPipeProcedureV2: executeFromOperateOnDataNodes({})", pipeName);
 
-    String exceptionMessage =
+    final String exceptionMessage =
         parsePushPipeMetaExceptionForPipe(pipeName, pushSinglePipeMetaToDataNodes(pipeName, env));
     if (!exceptionMessage.isEmpty()) {
       LOGGER.warn(
@@ -151,7 +151,7 @@ public class StartPipeProcedureV2 extends AbstractOperatePipeProcedureV2 {
     LOGGER.info("StartPipeProcedureV2: rollbackFromOperateOnDataNodes({})", pipeName);
 
     // Push all pipe metas to datanode, may be time-consuming
-    String exceptionMessage =
+    final String exceptionMessage =
         parsePushPipeMetaExceptionForPipe(pipeName, pushPipeMetaToDataNodes(env));
     if (!exceptionMessage.isEmpty()) {
       LOGGER.warn(
diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2.java
index 45cf49a5204..38b5094b262 100644
--- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2.java
+++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/StopPipeProcedureV2.java
@@ -98,7 +98,7 @@ public class StopPipeProcedureV2 extends AbstractOperatePipeProcedureV2 {
   public void executeFromOperateOnDataNodes(ConfigNodeProcedureEnv env) throws IOException {
     LOGGER.info("StopPipeProcedureV2: executeFromOperateOnDataNodes({})", pipeName);
 
-    String exceptionMessage =
+    final String exceptionMessage =
         parsePushPipeMetaExceptionForPipe(pipeName, pushSinglePipeMetaToDataNodes(pipeName, env));
     if (!exceptionMessage.isEmpty()) {
       LOGGER.warn(
@@ -144,7 +144,7 @@ public class StopPipeProcedureV2 extends AbstractOperatePipeProcedureV2 {
     LOGGER.info("StopPipeProcedureV2: rollbackFromOperateOnDataNodes({})", pipeName);
 
     // Push all pipe metas to datanode, may be time-consuming
-    String exceptionMessage =
+    final String exceptionMessage =
         parsePushPipeMetaExceptionForPipe(pipeName, pushPipeMetaToDataNodes(env));
     if (!exceptionMessage.isEmpty()) {
       LOGGER.warn(
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionPluginAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionPluginAgent.java
index 68a260bceea..92d130a12fe 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionPluginAgent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionPluginAgent.java
@@ -19,8 +19,10 @@
 
 package org.apache.iotdb.db.pipe.agent.plugin.dataregion;
 
+import org.apache.iotdb.commons.pipe.agent.plugin.PipeConnectorConstructor;
+import org.apache.iotdb.commons.pipe.agent.plugin.PipeExtractorConstructor;
 import org.apache.iotdb.commons.pipe.agent.plugin.PipePluginAgent;
-import org.apache.iotdb.commons.pipe.agent.plugin.PipePluginConstructor;
+import org.apache.iotdb.commons.pipe.agent.plugin.PipeProcessorConstructor;
 import org.apache.iotdb.commons.pipe.plugin.meta.DataNodePipePluginMetaKeeper;
 import org.apache.iotdb.commons.pipe.plugin.meta.PipePluginMetaKeeper;
 
@@ -31,21 +33,21 @@ public class PipeDataRegionPluginAgent extends PipePluginAgent {
   }
 
   @Override
-  protected PipePluginConstructor createPipeExtractorConstructor(
+  protected PipeExtractorConstructor createPipeExtractorConstructor(
       PipePluginMetaKeeper pipePluginMetaKeeper) {
     return new PipeDataRegionExtractorConstructor(
         (DataNodePipePluginMetaKeeper) pipePluginMetaKeeper);
   }
 
   @Override
-  protected PipePluginConstructor createPipeProcessorConstructor(
+  protected PipeProcessorConstructor createPipeProcessorConstructor(
       PipePluginMetaKeeper pipePluginMetaKeeper) {
     return new PipeDataRegionProcessorConstructor(
         (DataNodePipePluginMetaKeeper) pipePluginMetaKeeper);
   }
 
   @Override
-  protected PipePluginConstructor createPipeConnectorConstructor(
+  protected PipeConnectorConstructor createPipeConnectorConstructor(
       PipePluginMetaKeeper pipePluginMetaKeeper) {
     return new PipeDataRegionConnectorConstructor(
         (DataNodePipePluginMetaKeeper) pipePluginMetaKeeper);
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionProcessorConstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionProcessorConstructor.java
index 8c926dd06e3..bc33c6d97ea 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionProcessorConstructor.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/dataregion/PipeDataRegionProcessorConstructor.java
@@ -24,6 +24,9 @@ import org.apache.iotdb.commons.pipe.plugin.builtin.BuiltinPipePlugin;
 import org.apache.iotdb.commons.pipe.plugin.builtin.processor.donothing.DoNothingProcessor;
 import org.apache.iotdb.commons.pipe.plugin.builtin.processor.throwing.ThrowingExceptionProcessor;
 import org.apache.iotdb.commons.pipe.plugin.meta.DataNodePipePluginMetaKeeper;
+import org.apache.iotdb.db.pipe.processor.aggregate.AggregateProcessor;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.processor.StandardStatisticsOperatorProcessor;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.processor.TumblingWindowingProcessor;
 import org.apache.iotdb.db.pipe.processor.downsampling.sdt.SwingingDoorTrendingSamplingProcessor;
 import org.apache.iotdb.db.pipe.processor.downsampling.tumbling.TumblingTimeSamplingProcessor;
 
@@ -46,5 +49,13 @@ class PipeDataRegionProcessorConstructor extends PipeProcessorConstructor {
     pluginConstructors.put(
         BuiltinPipePlugin.THROWING_EXCEPTION_PROCESSOR.getPipePluginName(),
         ThrowingExceptionProcessor::new);
+    pluginConstructors.put(
+        BuiltinPipePlugin.AGGREGATE_PROCESSOR.getPipePluginName(), AggregateProcessor::new);
+    pluginConstructors.put(
+        BuiltinPipePlugin.STANDARD_STATISTICS_PROCESSOR.getPipePluginName(),
+        StandardStatisticsOperatorProcessor::new);
+    pluginConstructors.put(
+        BuiltinPipePlugin.TUMBLING_WINDOWING_PROCESSOR.getPipePluginName(),
+        TumblingWindowingProcessor::new);
   }
 }
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionConnectorConstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionConnectorConstructor.java
index 8d3399086d6..20132a9756a 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionConnectorConstructor.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionConnectorConstructor.java
@@ -67,7 +67,7 @@ class PipeSchemaRegionConnectorConstructor extends PipeConnectorConstructor {
   }
 
   @Override
-  protected PipeConnector reflectPluginByKey(String pluginKey) {
+  public PipeConnector reflectPluginByKey(String pluginKey) {
     // TODO: support constructing plugin by reflection
     return (PipeConnector)
         pluginConstructors.getOrDefault(pluginKey, DoNothingConnector::new).get();
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionExtractorConstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionExtractorConstructor.java
index f69f2f56c67..876ab9f8517 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionExtractorConstructor.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionExtractorConstructor.java
@@ -41,7 +41,7 @@ class PipeSchemaRegionExtractorConstructor extends PipeExtractorConstructor {
   }
 
   @Override
-  protected PipeExtractor reflectPluginByKey(String pluginKey) {
+  public PipeExtractor reflectPluginByKey(String pluginKey) {
     // TODO: support constructing plugin by reflection
     return (PipeExtractor)
         pluginConstructors.getOrDefault(pluginKey, DoNothingExtractor::new).get();
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionPluginAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionPluginAgent.java
index 76280d0ff7c..348b7ce0585 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionPluginAgent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionPluginAgent.java
@@ -19,8 +19,10 @@
 
 package org.apache.iotdb.db.pipe.agent.plugin.schemaregion;
 
+import org.apache.iotdb.commons.pipe.agent.plugin.PipeConnectorConstructor;
+import org.apache.iotdb.commons.pipe.agent.plugin.PipeExtractorConstructor;
 import org.apache.iotdb.commons.pipe.agent.plugin.PipePluginAgent;
-import org.apache.iotdb.commons.pipe.agent.plugin.PipePluginConstructor;
+import org.apache.iotdb.commons.pipe.agent.plugin.PipeProcessorConstructor;
 import org.apache.iotdb.commons.pipe.plugin.meta.PipePluginMetaKeeper;
 
 public class PipeSchemaRegionPluginAgent extends PipePluginAgent {
@@ -30,19 +32,19 @@ public class PipeSchemaRegionPluginAgent extends PipePluginAgent {
   }
 
   @Override
-  protected PipePluginConstructor createPipeExtractorConstructor(
+  protected PipeExtractorConstructor createPipeExtractorConstructor(
       PipePluginMetaKeeper pipePluginMetaKeeper) {
     return new PipeSchemaRegionExtractorConstructor();
   }
 
   @Override
-  protected PipePluginConstructor createPipeProcessorConstructor(
+  protected PipeProcessorConstructor createPipeProcessorConstructor(
       PipePluginMetaKeeper pipePluginMetaKeeper) {
     return new PipeSchemaRegionProcessorConstructor();
   }
 
   @Override
-  protected PipePluginConstructor createPipeConnectorConstructor(
+  protected PipeConnectorConstructor createPipeConnectorConstructor(
       PipePluginMetaKeeper pipePluginMetaKeeper) {
     return new PipeSchemaRegionConnectorConstructor();
   }
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionProcessorConstructor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionProcessorConstructor.java
index bd99e69862e..099283dbf57 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionProcessorConstructor.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/plugin/schemaregion/PipeSchemaRegionProcessorConstructor.java
@@ -33,7 +33,7 @@ class PipeSchemaRegionProcessorConstructor extends PipeProcessorConstructor {
   }
 
   @Override
-  protected PipeProcessor reflectPluginByKey(String pluginKey) {
+  public PipeProcessor reflectPluginByKey(String pluginKey) {
     // TODO: Support constructing plugin by reflection
     return (PipeProcessor)
         pluginConstructors.getOrDefault(pluginKey, DoNothingProcessor::new).get();
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java
index 8975459d79a..e269d16441b 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java
@@ -112,7 +112,7 @@ public class IoTDBLegacyPipeConnector implements PipeConnector {
   public void validate(PipeParameterValidator validator) throws Exception {
     final PipeParameters parameters = validator.getParameters();
     final IoTDBConfig ioTDBConfig = IoTDBDescriptor.getInstance().getConfig();
-    Set<TEndPoint> givenNodeUrls = parseNodeUrls(validator.getParameters());
+    final Set<TEndPoint> givenNodeUrls = parseNodeUrls(validator.getParameters());
 
     validator
         .validate(
@@ -230,7 +230,7 @@ public class IoTDBLegacyPipeConnector implements PipeConnector {
               pipeName, System.currentTimeMillis(), syncConnectorVersion, IoTDBConstant.PATH_ROOT);
       final TSStatus status = client.handshake(identityInfo);
       if (status.code != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
-        String errorMsg =
+        final String errorMsg =
             String.format(
                 "The receiver %s:%s rejected the pipe task because %s",
                 ipAddress, port, status.message);
@@ -358,7 +358,7 @@ public class IoTDBLegacyPipeConnector implements PipeConnector {
           randomAccessFile.seek(position);
           LOGGER.info("Redirect to position {} in transferring tsFile {}.", position, file);
         } else if (status.code == TSStatusCode.SYNC_FILE_ERROR.getStatusCode()) {
-          String errorMsg =
+          final String errorMsg =
               String.format("Network failed to receive tsFile %s, status: %s", file, status);
           LOGGER.warn(errorMsg);
           throw new PipeConnectionException(errorMsg);
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaConnector.java
index 383b40a0bb3..a762103fa9b 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaConnector.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaConnector.java
@@ -90,24 +90,24 @@ public class OpcUaConnector implements PipeConnector {
   @Override
   public void customize(PipeParameters parameters, PipeConnectorRuntimeConfiguration configuration)
       throws Exception {
-    int tcpBindPort =
+    final int tcpBindPort =
         parameters.getIntOrDefault(
             Arrays.asList(CONNECTOR_OPC_UA_TCP_BIND_PORT_KEY, SINK_OPC_UA_TCP_BIND_PORT_KEY),
             CONNECTOR_OPC_UA_TCP_BIND_PORT_DEFAULT_VALUE);
-    int httpsBindPort =
+    final int httpsBindPort =
         parameters.getIntOrDefault(
             Arrays.asList(CONNECTOR_OPC_UA_HTTPS_BIND_PORT_KEY, SINK_OPC_UA_HTTPS_BIND_PORT_KEY),
             CONNECTOR_OPC_UA_HTTPS_BIND_PORT_DEFAULT_VALUE);
 
-    String user =
+    final String user =
         parameters.getStringOrDefault(
             Arrays.asList(CONNECTOR_IOTDB_USER_KEY, SINK_IOTDB_USER_KEY),
             CONNECTOR_IOTDB_USER_DEFAULT_VALUE);
-    String password =
+    final String password =
         parameters.getStringOrDefault(
             Arrays.asList(CONNECTOR_IOTDB_PASSWORD_KEY, SINK_IOTDB_PASSWORD_KEY),
             CONNECTOR_IOTDB_PASSWORD_DEFAULT_VALUE);
-    String securityDir =
+    final String securityDir =
         parameters.getStringOrDefault(
             Arrays.asList(CONNECTOR_OPC_UA_SECURITY_DIR_KEY, SINK_OPC_UA_SECURITY_DIR_KEY),
             CONNECTOR_OPC_UA_SECURITY_DIR_DEFAULT_VALUE);
@@ -187,8 +187,8 @@ public class OpcUaConnector implements PipeConnector {
    */
   private void transferTablet(OpcUaServer server, Tablet tablet) throws UaException {
     // There is no nameSpace, so that nameSpaceIndex is always 0
-    int pseudoNameSpaceIndex = 0;
-    BaseEventTypeNode eventNode =
+    final int pseudoNameSpaceIndex = 0;
+    final BaseEventTypeNode eventNode =
         server
             .getEventFactory()
             .createEvent(
@@ -196,7 +196,7 @@ public class OpcUaConnector implements PipeConnector {
     // Use eventNode here because other nodes doesn't support values and times simultaneously
     for (int columnIndex = 0; columnIndex < tablet.getSchemas().size(); ++columnIndex) {
 
-      TSDataType dataType = tablet.getSchemas().get(columnIndex).getType();
+      final TSDataType dataType = tablet.getSchemas().get(columnIndex).getType();
 
       // Source name --> Sensor path, like root.test.d_0.s_0
       eventNode.setSourceName(
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaKeyStoreLoader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaKeyStoreLoader.java
index 23006ff77fd..45632485e3f 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaKeyStoreLoader.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaKeyStoreLoader.java
@@ -52,20 +52,20 @@ class OpcUaKeyStoreLoader {
   private KeyPair serverKeyPair;
 
   OpcUaKeyStoreLoader load(Path baseDir, char[] password) throws Exception {
-    KeyStore keyStore = KeyStore.getInstance("PKCS12");
+    final KeyStore keyStore = KeyStore.getInstance("PKCS12");
 
-    File serverKeyStore = baseDir.resolve("iotdb-server.pfx").toFile();
+    final File serverKeyStore = baseDir.resolve("iotdb-server.pfx").toFile();
 
     LOGGER.info("Loading KeyStore at {}", serverKeyStore);
 
     if (!serverKeyStore.exists()) {
       keyStore.load(null, password);
 
-      KeyPair keyPair = SelfSignedCertificateGenerator.generateRsaKeyPair(2048);
+      final KeyPair keyPair = SelfSignedCertificateGenerator.generateRsaKeyPair(2048);
 
-      String applicationUri = "urn:apache:iotdb:opc-ua-server:" + UUID.randomUUID();
+      final String applicationUri = "urn:apache:iotdb:opc-ua-server:" + UUID.randomUUID();
 
-      SelfSignedCertificateBuilder builder =
+      final SelfSignedCertificateBuilder builder =
           new SelfSignedCertificateBuilder(keyPair)
               .setCommonName("Apache IoTDB OPC UA server")
               .setOrganization("Apache")
@@ -75,8 +75,8 @@ class OpcUaKeyStoreLoader {
               .setCountryCode("CN")
               .setApplicationUri(applicationUri);
 
-      // Get as many hostnames and IP addresses as we can listed in the certificate.
-      Set<String> hostnames =
+      // Get as many hostnames and IP addresses as we can list in the certificate.
+      final Set<String> hostnames =
           Sets.union(
               Sets.newHashSet(HostnameUtil.getHostname()),
               HostnameUtil.getHostnames("0.0.0.0", false));
@@ -89,7 +89,7 @@ class OpcUaKeyStoreLoader {
         }
       }
 
-      X509Certificate certificate = builder.build();
+      final X509Certificate certificate = builder.build();
 
       keyStore.setKeyEntry(
           SERVER_ALIAS, keyPair.getPrivate(), password, new X509Certificate[] {certificate});
@@ -98,11 +98,11 @@ class OpcUaKeyStoreLoader {
       keyStore.load(new FileInputStream(serverKeyStore), password);
     }
 
-    Key serverPrivateKey = keyStore.getKey(SERVER_ALIAS, password);
+    final Key serverPrivateKey = keyStore.getKey(SERVER_ALIAS, password);
     if (serverPrivateKey instanceof PrivateKey) {
       serverCertificate = (X509Certificate) keyStore.getCertificate(SERVER_ALIAS);
 
-      PublicKey serverPublicKey = serverCertificate.getPublicKey();
+      final PublicKey serverPublicKey = serverCertificate.getPublicKey();
       serverKeyPair = new KeyPair(serverPublicKey, (PrivateKey) serverPrivateKey);
     }
 
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaServerBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaServerBuilder.java
index 2fd90d33a05..ceb39d3eac7 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaServerBuilder.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/opcua/OpcUaServerBuilder.java
@@ -119,36 +119,36 @@ public class OpcUaServerBuilder {
       throw new PipeException("Unable to create security dir: " + securityDir);
     }
 
-    File pkiDir = securityDir.resolve("pki").toFile();
+    final File pkiDir = securityDir.resolve("pki").toFile();
 
     LoggerFactory.getLogger(OpcUaServerBuilder.class)
         .info("Security dir: {}", securityDir.toAbsolutePath());
     LoggerFactory.getLogger(OpcUaServerBuilder.class)
         .info("Security pki dir: {}", pkiDir.getAbsolutePath());
 
-    OpcUaKeyStoreLoader loader =
+    final OpcUaKeyStoreLoader loader =
         new OpcUaKeyStoreLoader().load(securityDir, password.toCharArray());
 
-    DefaultCertificateManager certificateManager =
+    final DefaultCertificateManager certificateManager =
         new DefaultCertificateManager(loader.getServerKeyPair(), loader.getServerCertificate());
 
-    DefaultTrustListManager trustListManager = new DefaultTrustListManager(pkiDir);
+    final DefaultTrustListManager trustListManager = new DefaultTrustListManager(pkiDir);
     LOGGER.info(
         "Certificate directory is: {}, Please move certificates from the reject dir to the trusted directory to allow encrypted access",
         pkiDir.getAbsolutePath());
 
-    KeyPair httpsKeyPair = SelfSignedCertificateGenerator.generateRsaKeyPair(2048);
+    final KeyPair httpsKeyPair = SelfSignedCertificateGenerator.generateRsaKeyPair(2048);
 
-    SelfSignedHttpsCertificateBuilder httpsCertificateBuilder =
+    final SelfSignedHttpsCertificateBuilder httpsCertificateBuilder =
         new SelfSignedHttpsCertificateBuilder(httpsKeyPair);
     httpsCertificateBuilder.setCommonName(HostnameUtil.getHostname());
     HostnameUtil.getHostnames(WILD_CARD_ADDRESS).forEach(httpsCertificateBuilder::addDnsName);
-    X509Certificate httpsCertificate = httpsCertificateBuilder.build();
+    final X509Certificate httpsCertificate = httpsCertificateBuilder.build();
 
-    DefaultServerCertificateValidator certificateValidator =
+    final DefaultServerCertificateValidator certificateValidator =
         new DefaultServerCertificateValidator(trustListManager);
 
-    UsernameIdentityValidator identityValidator =
+    final UsernameIdentityValidator identityValidator =
         new UsernameIdentityValidator(
             true,
             authChallenge -> {
@@ -158,9 +158,9 @@ public class OpcUaServerBuilder {
               return inputUsername.equals(user) && inputPassword.equals(password);
             });
 
-    X509IdentityValidator x509IdentityValidator = new X509IdentityValidator(c -> true);
+    final X509IdentityValidator x509IdentityValidator = new X509IdentityValidator(c -> true);
 
-    X509Certificate certificate =
+    final X509Certificate certificate =
         certificateManager.getCertificates().stream()
             .findFirst()
             .orElseThrow(
@@ -168,7 +168,7 @@ public class OpcUaServerBuilder {
                     new UaRuntimeException(
                         StatusCodes.Bad_ConfigurationError, "No certificate found"));
 
-    String applicationUri =
+    final String applicationUri =
         CertificateUtil.getSanUri(certificate)
             .orElseThrow(
                 () ->
@@ -176,10 +176,10 @@ public class OpcUaServerBuilder {
                         StatusCodes.Bad_ConfigurationError,
                         "Certificate is missing the application URI"));
 
-    Set<EndpointConfiguration> endpointConfigurations =
+    final Set<EndpointConfiguration> endpointConfigurations =
         createEndpointConfigurations(certificate, tcpBindPort, httpsBindPort);
 
-    OpcUaServerConfig serverConfig =
+    final OpcUaServerConfig serverConfig =
         OpcUaServerConfig.builder()
             .setApplicationUri(applicationUri)
             .setApplicationName(LocalizedText.english("Apache IoTDB OPC UA server"))
@@ -202,8 +202,8 @@ public class OpcUaServerBuilder {
             .build();
 
     // Setup server to enable event posting
-    OpcUaServer server = new OpcUaServer(serverConfig);
-    UaNode serverNode =
+    final OpcUaServer server = new OpcUaServer(serverConfig);
+    final UaNode serverNode =
         server.getAddressSpaceManager().getManagedNode(Identifiers.Server).orElse(null);
     if (serverNode instanceof ServerTypeNode) {
       ((ServerTypeNode) serverNode).setEventNotifier(ubyte(1));
@@ -213,18 +213,18 @@ public class OpcUaServerBuilder {
 
   private Set<EndpointConfiguration> createEndpointConfigurations(
       X509Certificate certificate, int tcpBindPort, int httpsBindPort) {
-    Set<EndpointConfiguration> endpointConfigurations = new LinkedHashSet<>();
+    final Set<EndpointConfiguration> endpointConfigurations = new LinkedHashSet<>();
 
-    List<String> bindAddresses = newArrayList();
+    final List<String> bindAddresses = newArrayList();
     bindAddresses.add(WILD_CARD_ADDRESS);
 
-    Set<String> hostnames = new LinkedHashSet<>();
+    final Set<String> hostnames = new LinkedHashSet<>();
     hostnames.add(HostnameUtil.getHostname());
     hostnames.addAll(HostnameUtil.getHostnames(WILD_CARD_ADDRESS));
 
     for (String bindAddress : bindAddresses) {
       for (String hostname : hostnames) {
-        EndpointConfiguration.Builder builder =
+        final EndpointConfiguration.Builder builder =
             EndpointConfiguration.newBuilder()
                 .setBindAddress(bindAddress)
                 .setHostname(hostname)
@@ -235,7 +235,7 @@ public class OpcUaServerBuilder {
                     USER_TOKEN_POLICY_USERNAME,
                     USER_TOKEN_POLICY_X509);
 
-        EndpointConfiguration.Builder noSecurityBuilder =
+        final EndpointConfiguration.Builder noSecurityBuilder =
             builder
                 .copy()
                 .setSecurityPolicy(SecurityPolicy.None)
@@ -260,7 +260,7 @@ public class OpcUaServerBuilder {
                     .setSecurityMode(MessageSecurityMode.Sign),
                 httpsBindPort));
 
-        EndpointConfiguration.Builder discoveryBuilder =
+        final EndpointConfiguration.Builder discoveryBuilder =
             builder
                 .copy()
                 .setPath("/iotdb/discovery")
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java
index b48248d6d7d..bbfdf8e8dc5 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java
@@ -113,7 +113,8 @@ public class IoTDBDataRegionAsyncConnector extends IoTDBConnector {
     super.customize(parameters, configuration);
 
     // Disable batch mode for retry connector, in case retry events are never sent again
-    PipeParameters retryParameters = new PipeParameters(new HashMap<>(parameters.getAttribute()));
+    final PipeParameters retryParameters =
+        new PipeParameters(new HashMap<>(parameters.getAttribute()));
     retryParameters.getAttribute().put(SINK_IOTDB_BATCH_MODE_ENABLE_KEY, "false");
     retryParameters.getAttribute().put(CONNECTOR_IOTDB_BATCH_MODE_ENABLE_KEY, "false");
     retryConnector.customize(retryParameters, configuration);
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java
index f9e1328c250..3e41c98ad78 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java
@@ -116,7 +116,7 @@ public class IoTDBDataRegionSyncConnector extends IoTDBDataNodeSyncConnector {
     }
 
     try {
-      // in order to commit in order
+      // In order to commit in order
       if (isTabletBatchModeEnabled && !tabletBatchBuilder.isEmpty()) {
         doTransfer();
       }
@@ -150,7 +150,7 @@ public class IoTDBDataRegionSyncConnector extends IoTDBDataNodeSyncConnector {
   }
 
   private void doTransfer() {
-    Pair<IoTDBSyncClient, Boolean> clientAndStatus = clientManager.getClient();
+    final Pair<IoTDBSyncClient, Boolean> clientAndStatus = clientManager.getClient();
     final TPipeTransferResp resp;
     try {
       resp = clientAndStatus.getLeft().pipeTransfer(tabletBatchBuilder.toTPipeTransferReq());
@@ -171,7 +171,7 @@ public class IoTDBDataRegionSyncConnector extends IoTDBDataNodeSyncConnector {
 
   private void doTransfer(PipeInsertNodeTabletInsertionEvent pipeInsertNodeTabletInsertionEvent)
       throws PipeException {
-    InsertNode insertNode;
+    final InsertNode insertNode;
     Pair<IoTDBSyncClient, Boolean> clientAndStatus = null;
     final TPipeTransferResp resp;
 
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/writeback/WriteBackConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/writeback/WriteBackConnector.java
index 4b9437372ca..629746c3dd7 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/writeback/WriteBackConnector.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/writeback/WriteBackConnector.java
@@ -136,7 +136,7 @@ public class WriteBackConnector implements PipeConnector {
 
   private void doTransfer(PipeRawTabletInsertionEvent pipeRawTabletInsertionEvent)
       throws PipeException {
-    InsertBaseStatement statement =
+    final InsertBaseStatement statement =
         PipeTransferTabletRawReq.toTPipeTransferRawReq(
                 pipeRawTabletInsertionEvent.convertToTablet(),
                 pipeRawTabletInsertionEvent.isAligned())
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeResetTabletRow.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeResetTabletRow.java
new file mode 100644
index 00000000000..ee0a5cbc36d
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeResetTabletRow.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.pipe.event.common.row;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BitMap;
+import org.apache.iotdb.tsfile.write.record.Tablet;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+/**
+ * The pipe framework will reset a new {@link Tablet} when this kind of {@link PipeRow} is
+ * encountered.
+ */
+public class PipeResetTabletRow extends PipeRow {
+
+  public PipeResetTabletRow(
+      int rowIndex,
+      String deviceId,
+      boolean isAligned,
+      MeasurementSchema[] measurementSchemaList,
+      long[] timestampColumn,
+      TSDataType[] valueColumnTypes,
+      Object[] valueColumns,
+      BitMap[] bitMaps,
+      String[] columnNameStringList) {
+    super(
+        rowIndex,
+        deviceId,
+        isAligned,
+        measurementSchemaList,
+        timestampColumn,
+        valueColumnTypes,
+        valueColumns,
+        bitMaps,
+        columnNameStringList);
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java
index 3ef46d933fb..6080928fb2e 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/row/PipeRowCollector.java
@@ -39,8 +39,8 @@ public class PipeRowCollector implements RowCollector {
   private final List<TabletInsertionEvent> tabletInsertionEventList = new ArrayList<>();
   private Tablet tablet = null;
   private boolean isAligned = false;
-  private final PipeTaskMeta pipeTaskMeta; // used to report progress
-  private final EnrichedEvent sourceEvent; // used to report progress
+  private final PipeTaskMeta pipeTaskMeta; // Used to report progress
+  private final EnrichedEvent sourceEvent; // Used to report progress
 
   public PipeRowCollector(PipeTaskMeta pipeTaskMeta, EnrichedEvent sourceEvent) {
     this.pipeTaskMeta = pipeTaskMeta;
@@ -56,6 +56,11 @@ public class PipeRowCollector implements RowCollector {
     final PipeRow pipeRow = (PipeRow) row;
     final MeasurementSchema[] measurementSchemaArray = pipeRow.getMeasurementSchemaList();
 
+    // Trigger collection when a PipeResetTabletRow is encountered
+    if (row instanceof PipeResetTabletRow) {
+      collectTabletInsertionEvent();
+    }
+
     if (tablet == null) {
       final String deviceId = pipeRow.getDeviceId();
       final List<MeasurementSchema> measurementSchemaList =
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java
index 19368be2412..23bd0b4307c 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java
@@ -51,12 +51,13 @@ public class PipeInsertNodeTabletInsertionEvent extends EnrichedEvent
       LoggerFactory.getLogger(PipeInsertNodeTabletInsertionEvent.class);
 
   private final WALEntryHandler walEntryHandler;
-  private final ProgressIndex progressIndex;
   private final boolean isAligned;
   private final boolean isGeneratedByPipe;
 
   private TabletInsertionDataContainer dataContainer;
 
+  private ProgressIndex progressIndex;
+
   public PipeInsertNodeTabletInsertionEvent(
       WALEntryHandler walEntryHandler,
       ProgressIndex progressIndex,
@@ -141,6 +142,11 @@ public class PipeInsertNodeTabletInsertionEvent extends EnrichedEvent
     }
   }
 
+  @Override
+  public void bindProgressIndex(ProgressIndex progressIndex) {
+    this.progressIndex = progressIndex;
+  }
+
   @Override
   public ProgressIndex getProgressIndex() {
     return progressIndex == null ? MinimumProgressIndex.INSTANCE : progressIndex;
@@ -173,12 +179,12 @@ public class PipeInsertNodeTabletInsertionEvent extends EnrichedEvent
   @Override
   public boolean mayEventTimeOverlappedWithTimeRange() {
     try {
-      InsertNode insertNode = getInsertNode();
+      final InsertNode insertNode = getInsertNode();
       if (insertNode instanceof InsertRowNode) {
-        long timestamp = ((InsertRowNode) insertNode).getTime();
+        final long timestamp = ((InsertRowNode) insertNode).getTime();
         return startTime <= timestamp && timestamp <= endTime;
       } else if (insertNode instanceof InsertTabletNode) {
-        long[] timestamps = ((InsertTabletNode) insertNode).getTimes();
+        final long[] timestamps = ((InsertTabletNode) insertNode).getTimes();
         if (Objects.isNull(timestamps) || timestamps.length == 0) {
           return false;
         }
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java
index beb6c3ea9e9..c1102188682 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java
@@ -40,13 +40,15 @@ public class PipeRawTabletInsertionEvent extends EnrichedEvent implements Tablet
   private Tablet tablet;
   private final boolean isAligned;
 
-  private EnrichedEvent sourceEvent;
+  private final EnrichedEvent sourceEvent;
   private boolean needToReport;
 
   private PipeTabletMemoryBlock allocatedMemoryBlock;
 
   private TabletInsertionDataContainer dataContainer;
 
+  private ProgressIndex overridingProgressIndex;
+
   private PipeRawTabletInsertionEvent(
       Tablet tablet,
       boolean isAligned,
@@ -109,7 +111,6 @@ public class PipeRawTabletInsertionEvent extends EnrichedEvent implements Tablet
     allocatedMemoryBlock.close();
     // Actually release the occupied memory.
     tablet = null;
-    sourceEvent = null;
     dataContainer = null;
     return true;
   }
@@ -121,8 +122,24 @@ public class PipeRawTabletInsertionEvent extends EnrichedEvent implements Tablet
     }
   }
 
+  @Override
+  public void bindProgressIndex(ProgressIndex overridingProgressIndex) {
+    // Normally not all events need to report progress, but if the overriddenProgressIndex
+    // is given, indicating that the progress needs to be reported.
+    if (Objects.nonNull(overridingProgressIndex)) {
+      markAsNeedToReport();
+    }
+
+    this.overridingProgressIndex = overridingProgressIndex;
+  }
+
   @Override
   public ProgressIndex getProgressIndex() {
+    // If the overriddenProgressIndex is given, ignore the sourceEvent's progressIndex.
+    if (Objects.nonNull(overridingProgressIndex)) {
+      return overridingProgressIndex;
+    }
+
     return sourceEvent != null ? sourceEvent.getProgressIndex() : MinimumProgressIndex.INSTANCE;
   }
 
@@ -152,7 +169,7 @@ public class PipeRawTabletInsertionEvent extends EnrichedEvent implements Tablet
 
   @Override
   public boolean mayEventTimeOverlappedWithTimeRange() {
-    long[] timestamps = tablet.timestamps;
+    final long[] timestamps = tablet.timestamps;
     if (Objects.isNull(timestamps) || timestamps.length == 0) {
       return false;
     }
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/TabletInsertionDataContainer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/TabletInsertionDataContainer.java
index 158b8b35401..0e134083d2a 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/TabletInsertionDataContainer.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/TabletInsertionDataContainer.java
@@ -324,7 +324,7 @@ public class TabletInsertionDataContainer {
                 .boxed()
                 .map(o -> new BitMap(tablet.getMaxRowNumber()))
                 .toArray(BitMap[]::new)
-            : tablet.bitMaps; // we do not reduce bitmaps here by origin row size
+            : tablet.bitMaps; // We do not reduce bitmaps here by origin row size
     for (int i = 0; i < originBitMapList.length; i++) {
       if (originBitMapList[i] == null) {
         originBitMapList[i] = new BitMap(tablet.getMaxRowNumber());
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
index 5e42d7960b1..ae016fe6d1b 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
@@ -96,8 +96,8 @@ public class PipeTsFileInsertionEvent extends EnrichedEvent implements TsFileIns
   }
 
   /**
-   * @return {@code false} if this file can't be sent by pipe due to format violations. {@code true}
-   *     otherwise.
+   * @return {@code false} if this file can't be sent by pipe due to format violations or is empty.
+   *     {@code true} otherwise.
    */
   public boolean waitForTsFileClose() throws InterruptedException {
     if (!isClosed.get()) {
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java
index f5efe198ecf..5f0d6e6561a 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.historical;
 
 import org.apache.iotdb.commons.consensus.DataRegionId;
 import org.apache.iotdb.commons.consensus.index.ProgressIndex;
+import org.apache.iotdb.commons.consensus.index.impl.TimeWindowStateProgressIndex;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.pipe.config.constant.SystemConstant;
 import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskExtractorRuntimeEnvironment;
@@ -100,7 +101,7 @@ public class PipeHistoricalDataRegionTsFileExtractor implements PipeHistoricalDa
 
   @Override
   public void validate(PipeParameterValidator validator) {
-    PipeParameters parameters = validator.getParameters();
+    final PipeParameters parameters = validator.getParameters();
 
     if (parameters.hasAnyAttributes(SOURCE_START_TIME_KEY, SOURCE_END_TIME_KEY)) {
       isHistoricalExtractorEnabled = true;
@@ -168,7 +169,7 @@ public class PipeHistoricalDataRegionTsFileExtractor implements PipeHistoricalDa
                 SOURCE_HISTORY_END_TIME_KEY));
       }
     } catch (Exception e) {
-      // compatible with the current validation framework
+      // Compatible with the current validation framework
       throw new PipeParameterNotValidException(e.getMessage());
     }
   }
@@ -345,10 +346,7 @@ public class PipeHistoricalDataRegionTsFileExtractor implements PipeHistoricalDa
                         // Some resource may not be closed due to the control of
                         // PIPE_MIN_FLUSH_INTERVAL_IN_MS. We simply ignore them.
                         resource.isClosed()
-                            // Some different tsFiles may share the same max progressIndex, thus
-                            // tsFiles with an "equals" max progressIndex must be transmitted to
-                            // avoid data loss
-                            && !startIndex.isAfter(resource.getMaxProgressIndexAfterClose())
+                            && mayTsFileContainUnprocessedData(resource)
                             && isTsFileResourceOverlappedWithTimeRange(resource)
                             && isTsFileGeneratedAfterExtractionTimeLowerBound(resource))
                 .collect(Collectors.toList());
@@ -361,10 +359,7 @@ public class PipeHistoricalDataRegionTsFileExtractor implements PipeHistoricalDa
                         // Some resource may not be closed due to the control of
                         // PIPE_MIN_FLUSH_INTERVAL_IN_MS. We simply ignore them.
                         resource.isClosed()
-                            // Some different tsFiles may share the same max progressIndex, thus
-                            // tsFiles with an "equals" max progressIndex must be transmitted to
-                            // avoid data loss
-                            && !startIndex.isAfter(resource.getMaxProgressIndexAfterClose())
+                            && mayTsFileContainUnprocessedData(resource)
                             && isTsFileResourceOverlappedWithTimeRange(resource)
                             && isTsFileGeneratedAfterExtractionTimeLowerBound(resource))
                 .collect(Collectors.toList());
@@ -382,7 +377,10 @@ public class PipeHistoricalDataRegionTsFileExtractor implements PipeHistoricalDa
             });
 
         resourceList.sort(
-            (o1, o2) -> o1.getMaxProgressIndex().topologicalCompareTo(o2.getMaxProgressIndex()));
+            (o1, o2) ->
+                startIndex instanceof TimeWindowStateProgressIndex
+                    ? Long.compare(o1.getFileStartTime(), o2.getFileStartTime())
+                    : o1.getMaxProgressIndex().topologicalCompareTo(o2.getMaxProgressIndex()));
         pendingQueue = new ArrayDeque<>(resourceList);
 
         LOGGER.info(
@@ -405,6 +403,15 @@ public class PipeHistoricalDataRegionTsFileExtractor implements PipeHistoricalDa
     }
   }
 
+  private boolean mayTsFileContainUnprocessedData(TsFileResource resource) {
+    return startIndex instanceof TimeWindowStateProgressIndex
+        // The resource is closed thus the TsFileResource#getFileEndTime() is safe to use
+        ? ((TimeWindowStateProgressIndex) startIndex).getMinTime() <= resource.getFileEndTime()
+        // Some different tsFiles may share the same max progressIndex, thus tsFiles with an
+        // "equals" max progressIndex must be transmitted to avoid data loss
+        : !startIndex.isAfter(resource.getMaxProgressIndexAfterClose());
+  }
+
   private boolean isTsFileResourceOverlappedWithTimeRange(TsFileResource resource) {
     return !(resource.getFileEndTime() < historicalDataExtractionStartTime
         || historicalDataExtractionEndTime < resource.getFileStartTime());
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java
index 85ac5dd2f54..e4bd174fc78 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java
@@ -105,7 +105,7 @@ public abstract class PipeRealtimeDataRegionExtractor implements PipeExtractor {
 
   @Override
   public void validate(PipeParameterValidator validator) throws Exception {
-    PipeParameters parameters = validator.getParameters();
+    final PipeParameters parameters = validator.getParameters();
 
     try {
       realtimeDataExtractionStartTime =
@@ -149,7 +149,7 @@ public abstract class PipeRealtimeDataRegionExtractor implements PipeExtractor {
     // indexed by the taskID of IoTDBDataRegionExtractor. To avoid PipeRealtimeDataRegionExtractor
     // holding a reference to IoTDBDataRegionExtractor, the taskID should be constructed to
     // match that of IoTDBDataRegionExtractor.
-    long creationTime = environment.getCreationTime();
+    final long creationTime = environment.getCreationTime();
     taskID = pipeName + "_" + dataRegionId + "_" + creationTime;
 
     pipePattern = PipePattern.parsePipePatternFromSourceParameters(parameters);
@@ -386,7 +386,7 @@ public abstract class PipeRealtimeDataRegionExtractor implements PipeExtractor {
   }
 
   private boolean isDataRegionTimePartitionCoveredByTimeRange() {
-    Pair<Long, Long> timePartitionIdBound = dataRegionTimePartitionIdBound.get();
+    final Pair<Long, Long> timePartitionIdBound = dataRegionTimePartitionIdBound.get();
     return startTimePartitionIdLowerBound <= timePartitionIdBound.left
         && timePartitionIdBound.right <= endTimePartitionIdUpperBound;
   }
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/AbstractFormalProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/AbstractFormalProcessor.java
new file mode 100644
index 00000000000..357d7a0515e
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/AbstractFormalProcessor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.pipe.processor.aggregate;
+
+import org.apache.iotdb.pipe.api.PipePlugin;
+import org.apache.iotdb.pipe.api.PipeProcessor;
+import org.apache.iotdb.pipe.api.collector.EventCollector;
+import org.apache.iotdb.pipe.api.event.Event;
+import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent;
+import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent;
+
+/**
+ * {@link AbstractFormalProcessor} is a formal {@link PipeProcessor} that does not support data
+ * processing. It is mainly used as a configurable plugin for other {@link PipePlugin}s that can be
+ * dynamically loaded into IoTDB.
+ */
+public abstract class AbstractFormalProcessor implements PipeProcessor {
+  @Override
+  public final void process(
+      TabletInsertionEvent tabletInsertionEvent, EventCollector eventCollector) throws Exception {
+    throw new UnsupportedOperationException(
+        "The abstract formal processor does not support process events");
+  }
+
+  @Override
+  public final void process(
+      TsFileInsertionEvent tsFileInsertionEvent, EventCollector eventCollector) throws Exception {
+    throw new UnsupportedOperationException(
+        "The abstract formal processor does not support process events");
+  }
+
+  @Override
+  public final void process(Event event, EventCollector eventCollector) throws Exception {
+    throw new UnsupportedOperationException(
+        "The abstract formal processor does not support process events");
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/AggregateProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/AggregateProcessor.java
new file mode 100644
index 00000000000..01cb854ae53
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/AggregateProcessor.java
@@ -0,0 +1,719 @@
+/*
+ * 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.pipe.processor.aggregate;
+
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.consensus.DataRegionId;
+import org.apache.iotdb.commons.consensus.index.ProgressIndex;
+import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex;
+import org.apache.iotdb.commons.consensus.index.impl.TimeWindowStateProgressIndex;
+import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskProcessorRuntimeEnvironment;
+import org.apache.iotdb.commons.pipe.event.EnrichedEvent;
+import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta;
+import org.apache.iotdb.db.pipe.agent.PipeAgent;
+import org.apache.iotdb.db.pipe.agent.plugin.dataregion.PipeDataRegionPluginAgent;
+import org.apache.iotdb.db.pipe.event.common.row.PipeResetTabletRow;
+import org.apache.iotdb.db.pipe.event.common.row.PipeRow;
+import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent;
+import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent;
+import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.IntermediateResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.processor.AbstractOperatorProcessor;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.datastructure.WindowOutput;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.processor.AbstractWindowingProcessor;
+import org.apache.iotdb.db.queryengine.transformation.dag.udf.UDFParametersFactory;
+import org.apache.iotdb.db.storageengine.StorageEngine;
+import org.apache.iotdb.pipe.api.PipeProcessor;
+import org.apache.iotdb.pipe.api.access.Row;
+import org.apache.iotdb.pipe.api.collector.EventCollector;
+import org.apache.iotdb.pipe.api.collector.RowCollector;
+import org.apache.iotdb.pipe.api.customizer.configuration.PipeProcessorRuntimeConfiguration;
+import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator;
+import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters;
+import org.apache.iotdb.pipe.api.event.Event;
+import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent;
+import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent;
+import org.apache.iotdb.pipe.api.exception.PipeException;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.BitMap;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_OPERATORS_DEFAULT_VALUE;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_OPERATORS_KEY;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_OUTPUT_DATABASE_DEFAULT_VALUE;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_OUTPUT_DATABASE_KEY;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_OUTPUT_MAX_DELAY_SECONDS_DEFAULT_VALUE;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_OUTPUT_MAX_DELAY_SECONDS_KEY;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_OUTPUT_MEASUREMENTS_DEFAULT_VALUE;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_OUTPUT_MEASUREMENTS_KEY;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_OUTPUT_MIN_REPORT_INTERVAL_SECONDS_DEFAULT_VALUE;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_OUTPUT_MIN_REPORT_INTERVAL_SECONDS_KEY;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_WINDOWING_STRATEGY_DEFAULT_VALUE;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_WINDOWING_STRATEGY_KEY;
+
+/**
+ * {@link AggregateProcessor} is a {@link PipeProcessor} that can adopt different implementations of
+ * {@link AbstractWindowingProcessor} as windowing strategy and use calculation methods from all the
+ * {@link AbstractOperatorProcessor}s to calculate the given operators. Both the {@link
+ * AbstractWindowingProcessor} and {@link AbstractOperatorProcessor} can be implemented by user and
+ * loaded as a normal {@link PipeProcessor}
+ */
+public class AggregateProcessor implements PipeProcessor {
+  private static final String WINDOWING_PROCESSOR_SUFFIX = "-windowing-processor";
+
+  private String pipeName;
+  private String database;
+  private PipeTaskMeta pipeTaskMeta;
+  private long outputMaxDelayMilliseconds;
+  private long outputMinReportIntervalMilliseconds;
+  private String outputDatabase;
+
+  private final Map<String, AggregatedResultOperator> outputName2OperatorMap = new HashMap<>();
+  private final Map<String, Supplier<IntermediateResultOperator>>
+      intermediateResultName2OperatorSupplierMap = new HashMap<>();
+  private final Map<String, String> systemParameters = new HashMap<>();
+
+  private static final Map<String, Integer> pipeName2referenceCountMap = new ConcurrentHashMap<>();
+  private static final ConcurrentMap<
+          String, ConcurrentMap<String, AtomicReference<TimeSeriesRuntimeState>>>
+      pipeName2timeSeries2TimeSeriesRuntimeStateMap = new ConcurrentHashMap<>();
+
+  private AbstractWindowingProcessor windowingProcessor;
+  private final List<AbstractOperatorProcessor> operatorProcessors = new ArrayList<>();
+
+  private static final AtomicLong lastValueReceiveTime = new AtomicLong(0);
+
+  // Static values, calculated on initialization
+  private String[] columnNameStringList;
+
+  @Override
+  public void validate(PipeParameterValidator validator) throws Exception {
+    final PipeParameters parameters = validator.getParameters();
+    validator
+        .validate(
+            args -> !((String) args).isEmpty(),
+            String.format("The parameter %s must not be empty.", PROCESSOR_OPERATORS_KEY),
+            parameters.getStringOrDefault(
+                PROCESSOR_OPERATORS_KEY, PROCESSOR_OPERATORS_DEFAULT_VALUE))
+        .validate(
+            args -> !((String) args).isEmpty(),
+            String.format("The parameter %s must not be empty.", PROCESSOR_WINDOWING_STRATEGY_KEY),
+            parameters.getStringOrDefault(
+                PROCESSOR_WINDOWING_STRATEGY_KEY, PROCESSOR_WINDOWING_STRATEGY_DEFAULT_VALUE));
+  }
+
+  @Override
+  public void customize(PipeParameters parameters, PipeProcessorRuntimeConfiguration configuration)
+      throws Exception {
+    pipeName = configuration.getRuntimeEnvironment().getPipeName();
+    pipeName2referenceCountMap.compute(
+        pipeName, (name, count) -> Objects.nonNull(count) ? count + 1 : 1);
+    pipeName2timeSeries2TimeSeriesRuntimeStateMap.put(pipeName, new ConcurrentHashMap<>());
+
+    database =
+        StorageEngine.getInstance()
+            .getDataRegion(
+                new DataRegionId(
+                    ((PipeTaskProcessorRuntimeEnvironment) configuration.getRuntimeEnvironment())
+                        .getRegionId()))
+            .getDatabaseName();
+
+    pipeTaskMeta =
+        ((PipeTaskProcessorRuntimeEnvironment) configuration.getRuntimeEnvironment())
+            .getPipeTaskMeta();
+    // Load parameters
+    long outputMaxDelaySeconds =
+        parameters.getLongOrDefault(
+            PROCESSOR_OUTPUT_MAX_DELAY_SECONDS_KEY,
+            PROCESSOR_OUTPUT_MAX_DELAY_SECONDS_DEFAULT_VALUE);
+    outputMaxDelayMilliseconds =
+        outputMaxDelaySeconds < 0 ? Long.MAX_VALUE : outputMaxDelaySeconds * 1000;
+    outputMinReportIntervalMilliseconds =
+        parameters.getLongOrDefault(
+                PROCESSOR_OUTPUT_MIN_REPORT_INTERVAL_SECONDS_KEY,
+                PROCESSOR_OUTPUT_MIN_REPORT_INTERVAL_SECONDS_DEFAULT_VALUE)
+            * 1000;
+    outputDatabase =
+        parameters.getStringOrDefault(
+            PROCESSOR_OUTPUT_DATABASE_KEY, PROCESSOR_OUTPUT_DATABASE_DEFAULT_VALUE);
+
+    // Set output name
+    final List<String> operatorNameList =
+        Arrays.stream(
+                parameters
+                    .getStringOrDefault(PROCESSOR_OPERATORS_KEY, PROCESSOR_OPERATORS_DEFAULT_VALUE)
+                    .replace(" ", "")
+                    .split(","))
+            .collect(Collectors.toList());
+
+    final String outputMeasurementString =
+        parameters.getStringOrDefault(
+            PROCESSOR_OUTPUT_MEASUREMENTS_KEY, PROCESSOR_OUTPUT_MEASUREMENTS_DEFAULT_VALUE);
+    final List<String> outputMeasurementNameList =
+        outputMeasurementString.isEmpty()
+            ? Collections.emptyList()
+            : Arrays.stream(outputMeasurementString.replace(" ", "").split(","))
+                .collect(Collectors.toList());
+
+    final Map<String, String> aggregatorName2OutputNameMap = new HashMap<>();
+    for (int i = 0; i < operatorNameList.size(); ++i) {
+      if (i < outputMeasurementNameList.size()) {
+        aggregatorName2OutputNameMap.put(
+            operatorNameList.get(i).toLowerCase(), outputMeasurementNameList.get(i));
+      } else {
+        aggregatorName2OutputNameMap.put(
+            operatorNameList.get(i).toLowerCase(), operatorNameList.get(i));
+      }
+    }
+
+    // Load the useful aggregators' and their corresponding intermediate results' computational
+    // logic.
+    final Set<String> declaredIntermediateResultSet = new HashSet<>();
+    final PipeDataRegionPluginAgent agent = PipeAgent.plugin().dataRegion();
+    for (String pipePluginName :
+        agent.getSubProcessorNamesWithSpecifiedParent(AbstractOperatorProcessor.class)) {
+      // Children are allowed to validate and configure the computational logic
+      // from the same parameters other than processor name
+      final AbstractOperatorProcessor operatorProcessor =
+          (AbstractOperatorProcessor)
+              agent.getConfiguredProcessor(pipePluginName, parameters, configuration);
+      operatorProcessor.getAggregatorOperatorSet().stream()
+          .filter(
+              operator ->
+                  aggregatorName2OutputNameMap.containsKey(operator.getName().toLowerCase()))
+          .forEach(
+              operator -> {
+                outputName2OperatorMap.put(
+                    aggregatorName2OutputNameMap.get(operator.getName().toLowerCase()), operator);
+                declaredIntermediateResultSet.addAll(operator.getDeclaredIntermediateValueNames());
+              });
+
+      operatorProcessor
+          .getIntermediateResultOperatorSupplierSet()
+          .forEach(
+              supplier ->
+                  intermediateResultName2OperatorSupplierMap.put(
+                      supplier.get().getName(), supplier));
+      operatorProcessors.add(operatorProcessor);
+    }
+
+    aggregatorName2OutputNameMap
+        .entrySet()
+        .removeIf(entry -> outputName2OperatorMap.containsKey(entry.getValue()));
+    if (!aggregatorName2OutputNameMap.isEmpty()) {
+      throw new PipeException(
+          String.format(
+              "The aggregator and output name %s is invalid.", aggregatorName2OutputNameMap));
+    }
+
+    intermediateResultName2OperatorSupplierMap.keySet().retainAll(declaredIntermediateResultSet);
+    declaredIntermediateResultSet.removeAll(intermediateResultName2OperatorSupplierMap.keySet());
+    if (!declaredIntermediateResultSet.isEmpty()) {
+      throw new PipeException(
+          String.format(
+              "The needed intermediate values %s are not defined.", declaredIntermediateResultSet));
+    }
+
+    // Set up column name strings
+    columnNameStringList = new String[outputName2OperatorMap.size()];
+    final List<String> operatorNames = new ArrayList<>(outputName2OperatorMap.keySet());
+    for (int i = 0; i < outputName2OperatorMap.size(); ++i) {
+      columnNameStringList[i] = operatorNames.get(i);
+    }
+
+    // Get windowing processor
+    final String processorName =
+        parameters.getStringOrDefault(
+                PROCESSOR_WINDOWING_STRATEGY_KEY, PROCESSOR_WINDOWING_STRATEGY_DEFAULT_VALUE)
+            + WINDOWING_PROCESSOR_SUFFIX;
+    final PipeProcessor windowProcessor =
+        agent.getConfiguredProcessor(processorName, parameters, configuration);
+    if (!(windowProcessor instanceof AbstractWindowingProcessor)) {
+      throw new PipeException(
+          String.format("The processor %s is not a windowing processor.", processorName));
+    }
+    windowingProcessor = (AbstractWindowingProcessor) windowProcessor;
+
+    // Configure system parameters
+    systemParameters.put(
+        UDFParametersFactory.TIMESTAMP_PRECISION,
+        CommonDescriptor.getInstance().getConfig().getTimestampPrecision());
+
+    // Restore window state
+    final ProgressIndex index = pipeTaskMeta.getProgressIndex();
+    if (index == MinimumProgressIndex.INSTANCE) {
+      return;
+    }
+    if (!(index instanceof TimeWindowStateProgressIndex)) {
+      throw new PipeException(
+          String.format(
+              "The aggregate processor does not support progressIndexType %s", index.getType()));
+    }
+
+    final TimeWindowStateProgressIndex timeWindowStateProgressIndex =
+        (TimeWindowStateProgressIndex) index;
+    for (Map.Entry<String, Pair<Long, ByteBuffer>> entry :
+        timeWindowStateProgressIndex.getTimeSeries2TimestampWindowBufferPairMap().entrySet()) {
+      final AtomicReference<TimeSeriesRuntimeState> stateReference =
+          pipeName2timeSeries2TimeSeriesRuntimeStateMap
+              .get(pipeName)
+              .computeIfAbsent(
+                  entry.getKey(),
+                  key ->
+                      new AtomicReference<>(
+                          new TimeSeriesRuntimeState(
+                              outputName2OperatorMap,
+                              intermediateResultName2OperatorSupplierMap,
+                              systemParameters,
+                              windowingProcessor)));
+      synchronized (stateReference) {
+        try {
+          stateReference.get().restoreTimestampAndWindows(entry.getValue());
+        } catch (IOException e) {
+          throw new PipeException("Encountered exception when deserializing from PipeTaskMeta", e);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void process(TabletInsertionEvent tabletInsertionEvent, EventCollector eventCollector)
+      throws Exception {
+    if (!(tabletInsertionEvent instanceof PipeInsertNodeTabletInsertionEvent)
+        && !(tabletInsertionEvent instanceof PipeRawTabletInsertionEvent)) {
+      eventCollector.collect(tabletInsertionEvent);
+      return;
+    }
+
+    lastValueReceiveTime.set(System.currentTimeMillis());
+    final AtomicReference<Exception> exception = new AtomicReference<>();
+    final TimeWindowStateProgressIndex progressIndex =
+        new TimeWindowStateProgressIndex(new ConcurrentHashMap<>());
+
+    final Iterable<TabletInsertionEvent> outputEvents =
+        tabletInsertionEvent.processRowByRow(
+            (row, rowCollector) ->
+                progressIndex.updateToMinimumEqualOrIsAfterProgressIndex(
+                    new TimeWindowStateProgressIndex(processRow(row, rowCollector, exception))));
+
+    // Must reset progressIndex before collection
+    ((EnrichedEvent) tabletInsertionEvent).bindProgressIndex(progressIndex);
+
+    outputEvents.forEach(
+        event -> {
+          try {
+            eventCollector.collect(event);
+          } catch (Exception e) {
+            exception.set(e);
+          }
+        });
+
+    if (Objects.nonNull(exception.get())) {
+      throw exception.get();
+    }
+  }
+
+  private Map<String, Pair<Long, ByteBuffer>> processRow(
+      Row row, RowCollector rowCollector, AtomicReference<Exception> exception) {
+    final Map<String, Pair<Long, ByteBuffer>> resultMap = new HashMap<>();
+
+    final long timestamp = row.getTime();
+    for (int index = 0, size = row.size(); index < size; ++index) {
+      // Do not calculate null values
+      if (row.isNull(index)) {
+        continue;
+      }
+
+      final String timeSeries =
+          row.getDeviceId() + TsFileConstant.PATH_SEPARATOR + row.getColumnName(index);
+
+      final AtomicReference<TimeSeriesRuntimeState> stateReference =
+          pipeName2timeSeries2TimeSeriesRuntimeStateMap
+              .get(pipeName)
+              .computeIfAbsent(
+                  timeSeries,
+                  key ->
+                      new AtomicReference<>(
+                          new TimeSeriesRuntimeState(
+                              outputName2OperatorMap,
+                              intermediateResultName2OperatorSupplierMap,
+                              systemParameters,
+                              windowingProcessor)));
+
+      final Pair<List<WindowOutput>, Pair<Long, ByteBuffer>> result;
+      synchronized (stateReference) {
+        final TimeSeriesRuntimeState state = stateReference.get();
+        try {
+          switch (row.getDataType(index)) {
+            case BOOLEAN:
+              result =
+                  state.updateWindows(
+                      timestamp, row.getBoolean(index), outputMinReportIntervalMilliseconds);
+              break;
+            case INT32:
+              result =
+                  state.updateWindows(
+                      timestamp, row.getInt(index), outputMinReportIntervalMilliseconds);
+              break;
+            case INT64:
+              result =
+                  state.updateWindows(
+                      timestamp, row.getLong(index), outputMinReportIntervalMilliseconds);
+              break;
+            case FLOAT:
+              result =
+                  state.updateWindows(
+                      timestamp, row.getFloat(index), outputMinReportIntervalMilliseconds);
+              break;
+            case DOUBLE:
+              result =
+                  state.updateWindows(
+                      timestamp, row.getDouble(index), outputMinReportIntervalMilliseconds);
+              break;
+            case TEXT:
+              result =
+                  state.updateWindows(
+                      timestamp, row.getString(index), outputMinReportIntervalMilliseconds);
+              break;
+            default:
+              throw new UnsupportedOperationException(
+                  String.format("The type %s is not supported", row.getDataType(index)));
+          }
+          if (!Objects.isNull(result)) {
+            collectWindowOutputs(result.getLeft(), timeSeries, rowCollector);
+            if (Objects.nonNull(result.getRight())) {
+              resultMap.put(timeSeries, result.getRight());
+            }
+          }
+        } catch (IOException | UnsupportedOperationException e) {
+          exception.set(e);
+        }
+      }
+    }
+    return resultMap;
+  }
+
+  @Override
+  public void process(TsFileInsertionEvent tsFileInsertionEvent, EventCollector eventCollector)
+      throws Exception {
+    try {
+      for (final TabletInsertionEvent tabletInsertionEvent :
+          tsFileInsertionEvent.toTabletInsertionEvents()) {
+        process(tabletInsertionEvent, eventCollector);
+      }
+    } finally {
+      tsFileInsertionEvent.close();
+    }
+    // The timeProgressIndex shall only be reported by the output events
+    // whose progressIndex is bounded with tablet events
+    if (tsFileInsertionEvent instanceof PipeTsFileInsertionEvent) {
+      ((PipeTsFileInsertionEvent) tsFileInsertionEvent).skipReport();
+    }
+  }
+
+  @Override
+  public void process(Event event, EventCollector eventCollector) throws Exception {
+    if (System.currentTimeMillis() - lastValueReceiveTime.get() > outputMaxDelayMilliseconds) {
+      final AtomicReference<Exception> exception = new AtomicReference<>();
+
+      pipeName2timeSeries2TimeSeriesRuntimeStateMap
+          .get(pipeName)
+          .keySet()
+          .forEach(
+              timeSeries -> {
+                final AtomicReference<TimeSeriesRuntimeState> stateReference =
+                    pipeName2timeSeries2TimeSeriesRuntimeStateMap.get(pipeName).get(timeSeries);
+                synchronized (stateReference) {
+                  // This is only a formal tablet insertion event to collect all the results
+                  final PipeRawTabletInsertionEvent tabletInsertionEvent =
+                      new PipeRawTabletInsertionEvent(
+                          null, false, pipeName, pipeTaskMeta, null, false);
+                  tabletInsertionEvent
+                      .processRowByRow(
+                          (row, rowCollector) -> {
+                            try {
+                              collectWindowOutputs(
+                                  stateReference.get().forceOutput(), timeSeries, rowCollector);
+                            } catch (Exception e) {
+                              exception.set(e);
+                            }
+                          })
+                      .forEach(
+                          tabletEvent -> {
+                            try {
+                              eventCollector.collect(tabletEvent);
+                            } catch (Exception e) {
+                              exception.set(e);
+                            }
+                          });
+                }
+              });
+      if (exception.get() != null) {
+        throw exception.get();
+      }
+      // Forbidding emitting results until next data comes
+      lastValueReceiveTime.set(Long.MAX_VALUE);
+    }
+
+    eventCollector.collect(event);
+  }
+
+  /**
+   * Collect {@link WindowOutput}s of a single timeSeries in one turn. The {@link TSDataType}s shall
+   * be the same because the {@link AggregatedResultOperator}s shall return the same value for the
+   * same timeSeries.
+   *
+   * @param outputs the {@link WindowOutput} output
+   * @param timeSeries the timeSeries‘ name
+   * @param collector {@link RowCollector}
+   */
+  public void collectWindowOutputs(
+      List<WindowOutput> outputs, String timeSeries, RowCollector collector) throws IOException {
+    if (Objects.isNull(outputs) || outputs.isEmpty()) {
+      return;
+    }
+    // Sort and same timestamps removal
+    outputs.sort(Comparator.comparingLong(WindowOutput::getTimestamp));
+
+    final AtomicLong lastValue = new AtomicLong(Long.MIN_VALUE);
+    final List<WindowOutput> distinctOutputs = new ArrayList<>();
+    outputs.forEach(
+        output -> {
+          long timeStamp = output.getTimestamp();
+          if (timeStamp != lastValue.get()) {
+            lastValue.set(timeStamp);
+            distinctOutputs.add(output);
+          }
+        });
+
+    final MeasurementSchema[] measurementSchemaList =
+        new MeasurementSchema[columnNameStringList.length];
+    final TSDataType[] valueColumnTypes = new TSDataType[columnNameStringList.length];
+    final Object[] valueColumns = new Object[columnNameStringList.length];
+    final BitMap[] bitMaps = new BitMap[columnNameStringList.length];
+
+    // Setup timestamps
+    final long[] timestampColumn = new long[distinctOutputs.size()];
+    for (int i = 0; i < distinctOutputs.size(); ++i) {
+      timestampColumn[i] = distinctOutputs.get(i).getTimestamp();
+    }
+
+    for (int columnIndex = 0; columnIndex < columnNameStringList.length; ++columnIndex) {
+      bitMaps[columnIndex] = new BitMap(distinctOutputs.size());
+      for (int rowIndex = 0; rowIndex < distinctOutputs.size(); ++rowIndex) {
+        Map<String, Pair<TSDataType, Object>> aggregatedResults =
+            distinctOutputs.get(rowIndex).getAggregatedResults();
+        if (aggregatedResults.containsKey(columnNameStringList[columnIndex])) {
+          if (Objects.isNull(valueColumnTypes[columnIndex])) {
+            // Fill in measurements and init columns when the first non-null value is seen
+            valueColumnTypes[columnIndex] =
+                aggregatedResults.get(columnNameStringList[columnIndex]).getLeft();
+            measurementSchemaList[columnIndex] =
+                new MeasurementSchema(
+                    columnNameStringList[columnIndex], valueColumnTypes[columnIndex]);
+            switch (valueColumnTypes[columnIndex]) {
+              case BOOLEAN:
+                valueColumns[columnIndex] = new boolean[distinctOutputs.size()];
+                break;
+              case INT32:
+                valueColumns[columnIndex] = new int[distinctOutputs.size()];
+                break;
+              case INT64:
+                valueColumns[columnIndex] = new long[distinctOutputs.size()];
+                break;
+              case FLOAT:
+                valueColumns[columnIndex] = new float[distinctOutputs.size()];
+                break;
+              case DOUBLE:
+                valueColumns[columnIndex] = new double[distinctOutputs.size()];
+                break;
+              case TEXT:
+                valueColumns[columnIndex] = new String[distinctOutputs.size()];
+                break;
+              default:
+                throw new UnsupportedOperationException(
+                    String.format(
+                        "The output tablet does not support column type %s",
+                        valueColumnTypes[columnIndex]));
+            }
+          }
+          // Fill in values
+          switch (valueColumnTypes[columnIndex]) {
+            case BOOLEAN:
+              ((boolean[]) valueColumns[columnIndex])[rowIndex] =
+                  (boolean) aggregatedResults.get(columnNameStringList[columnIndex]).getRight();
+              break;
+            case INT32:
+              ((int[]) valueColumns[columnIndex])[rowIndex] =
+                  (int) aggregatedResults.get(columnNameStringList[columnIndex]).getRight();
+              break;
+            case INT64:
+              ((long[]) valueColumns[columnIndex])[rowIndex] =
+                  (long) aggregatedResults.get(columnNameStringList[columnIndex]).getRight();
+              break;
+            case FLOAT:
+              ((float[]) valueColumns[columnIndex])[rowIndex] =
+                  (float) aggregatedResults.get(columnNameStringList[columnIndex]).getRight();
+              break;
+            case DOUBLE:
+              ((double[]) valueColumns[columnIndex])[rowIndex] =
+                  (double) aggregatedResults.get(columnNameStringList[columnIndex]).getRight();
+              break;
+            case TEXT:
+              ((String[]) valueColumns[columnIndex])[rowIndex] =
+                  (String) aggregatedResults.get(columnNameStringList[columnIndex]).getRight();
+              break;
+            default:
+              throw new UnsupportedOperationException(
+                  String.format(
+                      "The output tablet does not support column type %s",
+                      valueColumnTypes[rowIndex]));
+          }
+        } else {
+          bitMaps[columnIndex].mark(rowIndex);
+        }
+      }
+    }
+
+    // Filter null outputs
+    final Integer[] originColumnIndex2FilteredColumnIndexMapperList =
+        new Integer[columnNameStringList.length];
+    int filteredCount = 0;
+    for (int i = 0; i < columnNameStringList.length; ++i) {
+      if (!bitMaps[i].isAllMarked()) {
+        originColumnIndex2FilteredColumnIndexMapperList[i] = ++filteredCount;
+      }
+    }
+
+    final String outputTimeSeries =
+        outputDatabase.isEmpty() ? timeSeries : timeSeries.replaceFirst(database, outputDatabase);
+
+    if (filteredCount == columnNameStringList.length) {
+      // No filter, collect rows
+      for (int rowIndex = 0; rowIndex < distinctOutputs.size(); ++rowIndex) {
+        collector.collectRow(
+            rowIndex == 0
+                ? new PipeResetTabletRow(
+                    rowIndex,
+                    outputTimeSeries,
+                    false,
+                    measurementSchemaList,
+                    timestampColumn,
+                    valueColumnTypes,
+                    valueColumns,
+                    bitMaps,
+                    columnNameStringList)
+                : new PipeRow(
+                    rowIndex,
+                    outputTimeSeries,
+                    false,
+                    measurementSchemaList,
+                    timestampColumn,
+                    valueColumnTypes,
+                    valueColumns,
+                    bitMaps,
+                    columnNameStringList));
+      }
+    } else {
+      // Recompute the column arrays
+      final MeasurementSchema[] filteredMeasurementSchemaList =
+          new MeasurementSchema[filteredCount];
+      final String[] filteredColumnNameStringList = new String[filteredCount];
+      final TSDataType[] filteredValueColumnTypes = new TSDataType[filteredCount];
+      final Object[] filteredValueColumns = new Object[filteredCount];
+      final BitMap[] filteredBitMaps = new BitMap[filteredCount];
+
+      for (int i = 0; i < originColumnIndex2FilteredColumnIndexMapperList.length; i++) {
+        if (originColumnIndex2FilteredColumnIndexMapperList[i] != null) {
+          final int filteredColumnIndex = originColumnIndex2FilteredColumnIndexMapperList[i];
+          filteredMeasurementSchemaList[filteredColumnIndex] = measurementSchemaList[i];
+          filteredColumnNameStringList[filteredColumnIndex] = columnNameStringList[i];
+          filteredValueColumnTypes[filteredColumnIndex] = valueColumnTypes[i];
+          filteredBitMaps[filteredColumnIndex] = bitMaps[i];
+          filteredValueColumns[filteredColumnIndex] = valueColumns[i];
+        }
+      }
+      // Collect rows
+      for (int rowIndex = 0; rowIndex < distinctOutputs.size(); ++rowIndex) {
+        collector.collectRow(
+            rowIndex == 0
+                ? new PipeResetTabletRow(
+                    rowIndex,
+                    outputTimeSeries,
+                    false,
+                    filteredMeasurementSchemaList,
+                    timestampColumn,
+                    filteredValueColumnTypes,
+                    filteredValueColumns,
+                    filteredBitMaps,
+                    filteredColumnNameStringList)
+                : new PipeRow(
+                    rowIndex,
+                    outputTimeSeries,
+                    false,
+                    filteredMeasurementSchemaList,
+                    timestampColumn,
+                    filteredValueColumnTypes,
+                    filteredValueColumns,
+                    filteredBitMaps,
+                    filteredColumnNameStringList));
+      }
+    }
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (Objects.nonNull(pipeName)
+        && pipeName2referenceCountMap.compute(
+                pipeName, (name, count) -> Objects.nonNull(count) ? count - 1 : 0)
+            == 0) {
+      pipeName2timeSeries2TimeSeriesRuntimeStateMap.get(pipeName).clear();
+      pipeName2timeSeries2TimeSeriesRuntimeStateMap.remove(pipeName);
+    }
+    if (Objects.nonNull(windowingProcessor)) {
+      windowingProcessor.close();
+    }
+    for (PipeProcessor operatorProcessor : operatorProcessors) {
+      operatorProcessor.close();
+    }
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/TimeSeriesRuntimeState.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/TimeSeriesRuntimeState.java
new file mode 100644
index 00000000000..1b0677d0d10
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/TimeSeriesRuntimeState.java
@@ -0,0 +1,396 @@
+/*
+ * 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.pipe.processor.aggregate;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.IntermediateResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.datastructure.TimeSeriesWindow;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.datastructure.WindowOutput;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.datastructure.WindowState;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.processor.AbstractWindowingProcessor;
+import org.apache.iotdb.tsfile.utils.Pair;
+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;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+public class TimeSeriesRuntimeState {
+  // References
+  private final Map<String, AggregatedResultOperator> aggregatorOutputName2OperatorMap;
+  private final Map<String, Supplier<IntermediateResultOperator>>
+      intermediateResultName2OperatorSupplierMap;
+  private final Map<String, String> systemParameters;
+  private final AbstractWindowingProcessor windowingProcessor;
+
+  // Inner set values
+  private long lastStateReportPhysicalTime;
+
+  // Outer set values
+  private long lastReportTimeStamp = Long.MIN_VALUE;
+  private final List<TimeSeriesWindow> currentOpeningWindows = new ArrayList<>();
+
+  // Variables to avoid "new" operation
+  private final List<WindowOutput> outputList = new ArrayList<>();
+
+  public TimeSeriesRuntimeState(
+      Map<String, AggregatedResultOperator> aggregatorOutputName2OperatorMap,
+      Map<String, Supplier<IntermediateResultOperator>> intermediateResultName2OperatorSupplierMap,
+      Map<String, String> systemParameters,
+      AbstractWindowingProcessor windowingProcessor) {
+    this.aggregatorOutputName2OperatorMap = aggregatorOutputName2OperatorMap;
+    this.intermediateResultName2OperatorSupplierMap = intermediateResultName2OperatorSupplierMap;
+    this.systemParameters = systemParameters;
+    this.windowingProcessor = windowingProcessor;
+  }
+
+  // The following "updateWindows" are the same except for the input value types
+  public Pair<List<WindowOutput>, Pair<Long, ByteBuffer>> updateWindows(
+      long timestamp, boolean value, long outputMinReportIntervalMilliseconds) throws IOException {
+    Pair<List<WindowOutput>, Pair<Long, ByteBuffer>> output = null;
+
+    if (timestamp <= lastReportTimeStamp) {
+      return null;
+    }
+    final Set<TimeSeriesWindow> addedWindows =
+        windowingProcessor.mayAddWindow(currentOpeningWindows, timestamp, value);
+    if (Objects.nonNull(addedWindows)) {
+      addedWindows.forEach(
+          window ->
+              window.initWindow(
+                  intermediateResultName2OperatorSupplierMap,
+                  aggregatorOutputName2OperatorMap,
+                  systemParameters));
+    }
+    final Iterator<TimeSeriesWindow> windowIterator = currentOpeningWindows.iterator();
+    while (windowIterator.hasNext()) {
+      final TimeSeriesWindow window = windowIterator.next();
+      final Pair<WindowState, WindowOutput> stateWindowOutputPair =
+          window.updateIntermediateResult(timestamp, value);
+      if (Objects.isNull(stateWindowOutputPair)) {
+        continue;
+      }
+      if (stateWindowOutputPair.getLeft().isEmit()
+          && Objects.nonNull(stateWindowOutputPair.getRight())) {
+        outputList.add(stateWindowOutputPair.getRight());
+        lastReportTimeStamp =
+            Math.max(lastReportTimeStamp, stateWindowOutputPair.getRight().getProgressTime());
+      }
+      if (stateWindowOutputPair.getLeft().isPurge()) {
+        windowIterator.remove();
+      }
+    }
+    if (!outputList.isEmpty()) {
+      output =
+          new Pair<>(
+              new ArrayList<>(outputList),
+              getTimestampWindowBufferPair(outputMinReportIntervalMilliseconds));
+      outputList.clear();
+    }
+    return output;
+  }
+
+  public Pair<List<WindowOutput>, Pair<Long, ByteBuffer>> updateWindows(
+      long timestamp, int value, long outputMinReportIntervalMilliseconds) throws IOException {
+    Pair<List<WindowOutput>, Pair<Long, ByteBuffer>> output = null;
+
+    if (timestamp <= lastReportTimeStamp) {
+      return null;
+    }
+    final Set<TimeSeriesWindow> addedWindows =
+        windowingProcessor.mayAddWindow(currentOpeningWindows, timestamp, value);
+    if (Objects.nonNull(addedWindows)) {
+      addedWindows.forEach(
+          window ->
+              window.initWindow(
+                  intermediateResultName2OperatorSupplierMap,
+                  aggregatorOutputName2OperatorMap,
+                  systemParameters));
+    }
+    final Iterator<TimeSeriesWindow> windowIterator = currentOpeningWindows.iterator();
+    while (windowIterator.hasNext()) {
+      final TimeSeriesWindow window = windowIterator.next();
+      final Pair<WindowState, WindowOutput> stateWindowOutputPair =
+          window.updateIntermediateResult(timestamp, value);
+      if (Objects.isNull(stateWindowOutputPair)) {
+        continue;
+      }
+      if (stateWindowOutputPair.getLeft().isEmit()
+          && Objects.nonNull(stateWindowOutputPair.getRight())) {
+        outputList.add(stateWindowOutputPair.getRight());
+        lastReportTimeStamp =
+            Math.max(lastReportTimeStamp, stateWindowOutputPair.getRight().getProgressTime());
+      }
+      if (stateWindowOutputPair.getLeft().isPurge()) {
+        windowIterator.remove();
+      }
+    }
+    if (!outputList.isEmpty()) {
+      output =
+          new Pair<>(
+              new ArrayList<>(outputList),
+              getTimestampWindowBufferPair(outputMinReportIntervalMilliseconds));
+      outputList.clear();
+    }
+    return output;
+  }
+
+  public Pair<List<WindowOutput>, Pair<Long, ByteBuffer>> updateWindows(
+      long timestamp, long value, long outputMinReportIntervalMilliseconds) throws IOException {
+    Pair<List<WindowOutput>, Pair<Long, ByteBuffer>> output = null;
+
+    if (timestamp <= lastReportTimeStamp) {
+      return null;
+    }
+    final Set<TimeSeriesWindow> addedWindows =
+        windowingProcessor.mayAddWindow(currentOpeningWindows, timestamp, value);
+    if (Objects.nonNull(addedWindows)) {
+      addedWindows.forEach(
+          window ->
+              window.initWindow(
+                  intermediateResultName2OperatorSupplierMap,
+                  aggregatorOutputName2OperatorMap,
+                  systemParameters));
+    }
+    final Iterator<TimeSeriesWindow> windowIterator = currentOpeningWindows.iterator();
+    while (windowIterator.hasNext()) {
+      final TimeSeriesWindow window = windowIterator.next();
+      final Pair<WindowState, WindowOutput> stateWindowOutputPair =
+          window.updateIntermediateResult(timestamp, value);
+      if (Objects.isNull(stateWindowOutputPair)) {
+        continue;
+      }
+      if (stateWindowOutputPair.getLeft().isEmit()
+          && Objects.nonNull(stateWindowOutputPair.getRight())) {
+        outputList.add(stateWindowOutputPair.getRight());
+        lastReportTimeStamp =
+            Math.max(lastReportTimeStamp, stateWindowOutputPair.getRight().getProgressTime());
+      }
+      if (stateWindowOutputPair.getLeft().isPurge()) {
+        windowIterator.remove();
+      }
+    }
+    if (!outputList.isEmpty()) {
+      output =
+          new Pair<>(
+              new ArrayList<>(outputList),
+              getTimestampWindowBufferPair(outputMinReportIntervalMilliseconds));
+      outputList.clear();
+    }
+    return output;
+  }
+
+  public Pair<List<WindowOutput>, Pair<Long, ByteBuffer>> updateWindows(
+      long timestamp, float value, long outputMinReportIntervalMilliseconds) throws IOException {
+    Pair<List<WindowOutput>, Pair<Long, ByteBuffer>> output = null;
+
+    if (timestamp <= lastReportTimeStamp) {
+      return null;
+    }
+    final Set<TimeSeriesWindow> addedWindows =
+        windowingProcessor.mayAddWindow(currentOpeningWindows, timestamp, value);
+    if (Objects.nonNull(addedWindows)) {
+      addedWindows.forEach(
+          window ->
+              window.initWindow(
+                  intermediateResultName2OperatorSupplierMap,
+                  aggregatorOutputName2OperatorMap,
+                  systemParameters));
+    }
+    final Iterator<TimeSeriesWindow> windowIterator = currentOpeningWindows.iterator();
+    while (windowIterator.hasNext()) {
+      final TimeSeriesWindow window = windowIterator.next();
+      final Pair<WindowState, WindowOutput> stateWindowOutputPair =
+          window.updateIntermediateResult(timestamp, value);
+      if (Objects.isNull(stateWindowOutputPair)) {
+        continue;
+      }
+      if (stateWindowOutputPair.getLeft().isEmit()
+          && Objects.nonNull(stateWindowOutputPair.getRight())) {
+        outputList.add(stateWindowOutputPair.getRight());
+        lastReportTimeStamp =
+            Math.max(lastReportTimeStamp, stateWindowOutputPair.getRight().getProgressTime());
+      }
+      if (stateWindowOutputPair.getLeft().isPurge()) {
+        windowIterator.remove();
+      }
+    }
+    if (!outputList.isEmpty()) {
+      output =
+          new Pair<>(
+              new ArrayList<>(outputList),
+              getTimestampWindowBufferPair(outputMinReportIntervalMilliseconds));
+      outputList.clear();
+    }
+    return output;
+  }
+
+  public Pair<List<WindowOutput>, Pair<Long, ByteBuffer>> updateWindows(
+      long timestamp, double value, long outputMinReportIntervalMilliseconds) throws IOException {
+    Pair<List<WindowOutput>, Pair<Long, ByteBuffer>> output = null;
+
+    if (timestamp <= lastReportTimeStamp) {
+      return null;
+    }
+    final Set<TimeSeriesWindow> addedWindows =
+        windowingProcessor.mayAddWindow(currentOpeningWindows, timestamp, value);
+    if (Objects.nonNull(addedWindows)) {
+      addedWindows.forEach(
+          window ->
+              window.initWindow(
+                  intermediateResultName2OperatorSupplierMap,
+                  aggregatorOutputName2OperatorMap,
+                  systemParameters));
+    }
+    final Iterator<TimeSeriesWindow> windowIterator = currentOpeningWindows.iterator();
+    while (windowIterator.hasNext()) {
+      final TimeSeriesWindow window = windowIterator.next();
+      final Pair<WindowState, WindowOutput> stateWindowOutputPair =
+          window.updateIntermediateResult(timestamp, value);
+      if (Objects.isNull(stateWindowOutputPair)) {
+        continue;
+      }
+      if (stateWindowOutputPair.getLeft().isEmit()
+          && Objects.nonNull(stateWindowOutputPair.getRight())) {
+        outputList.add(stateWindowOutputPair.getRight());
+        lastReportTimeStamp =
+            Math.max(lastReportTimeStamp, stateWindowOutputPair.getRight().getProgressTime());
+      }
+      if (stateWindowOutputPair.getLeft().isPurge()) {
+        windowIterator.remove();
+      }
+    }
+    if (!outputList.isEmpty()) {
+      output =
+          new Pair<>(
+              new ArrayList<>(outputList),
+              getTimestampWindowBufferPair(outputMinReportIntervalMilliseconds));
+      outputList.clear();
+    }
+    return output;
+  }
+
+  public Pair<List<WindowOutput>, Pair<Long, ByteBuffer>> updateWindows(
+      long timestamp, String value, long outputMinReportIntervalMilliseconds) throws IOException {
+    Pair<List<WindowOutput>, Pair<Long, ByteBuffer>> output = null;
+
+    if (timestamp <= lastReportTimeStamp) {
+      return null;
+    }
+    final Set<TimeSeriesWindow> addedWindows =
+        windowingProcessor.mayAddWindow(currentOpeningWindows, timestamp, value);
+    if (Objects.nonNull(addedWindows)) {
+      addedWindows.forEach(
+          window ->
+              window.initWindow(
+                  intermediateResultName2OperatorSupplierMap,
+                  aggregatorOutputName2OperatorMap,
+                  systemParameters));
+    }
+    final Iterator<TimeSeriesWindow> windowIterator = currentOpeningWindows.iterator();
+    while (windowIterator.hasNext()) {
+      final TimeSeriesWindow window = windowIterator.next();
+      final Pair<WindowState, WindowOutput> stateWindowOutputPair =
+          window.updateIntermediateResult(timestamp, value);
+      if (Objects.isNull(stateWindowOutputPair)) {
+        continue;
+      }
+      if (stateWindowOutputPair.getLeft().isEmit()
+          && Objects.nonNull(stateWindowOutputPair.getRight())) {
+        outputList.add(stateWindowOutputPair.getRight());
+        lastReportTimeStamp =
+            Math.max(lastReportTimeStamp, stateWindowOutputPair.getRight().getProgressTime());
+      }
+      if (stateWindowOutputPair.getLeft().isPurge()) {
+        windowIterator.remove();
+      }
+    }
+    if (!outputList.isEmpty()) {
+      output =
+          new Pair<>(
+              new ArrayList<>(outputList),
+              getTimestampWindowBufferPair(outputMinReportIntervalMilliseconds));
+      outputList.clear();
+    }
+    return output;
+  }
+
+  public List<WindowOutput> forceOutput() {
+    return currentOpeningWindows.stream()
+        .map(TimeSeriesWindow::forceOutput)
+        .collect(Collectors.toList());
+  }
+
+  // A runtime state with window buffer shall be reported after at least output min report
+  // interval time to avoid frequent serialization
+  // Return null if this should not report
+  private Pair<Long, ByteBuffer> getTimestampWindowBufferPair(
+      long outputMinReportIntervalMilliseconds) throws IOException {
+    if (currentOpeningWindows.isEmpty()) {
+      return new Pair<>(lastReportTimeStamp, null);
+    }
+    if (System.currentTimeMillis() - lastStateReportPhysicalTime
+        < outputMinReportIntervalMilliseconds) {
+      return null;
+    }
+    try (final PublicBAOS byteArrayOutputStream = new PublicBAOS();
+        final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) {
+      ReadWriteIOUtils.write(currentOpeningWindows.size(), outputStream);
+      for (TimeSeriesWindow window : currentOpeningWindows) {
+        window.serialize(outputStream);
+      }
+      lastStateReportPhysicalTime = System.currentTimeMillis();
+      return new Pair<>(
+          lastReportTimeStamp,
+          ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()));
+    }
+  }
+
+  public void restoreTimestampAndWindows(Pair<Long, ByteBuffer> timestampWindowBufferPair)
+      throws IOException {
+    if (timestampWindowBufferPair.getLeft() <= lastReportTimeStamp) {
+      // The runtime state may be initialized by different processorSubtasks on one DataNode
+      // Only the subtask with the largest timestamp wins
+      return;
+    }
+    this.lastReportTimeStamp = timestampWindowBufferPair.getLeft();
+    final ByteBuffer buffer = timestampWindowBufferPair.getRight();
+    final int size = ReadWriteIOUtils.readInt(buffer);
+    for (int i = 0; i < size; ++i) {
+      // The runtime value will be deserialized if exists
+      final TimeSeriesWindow currentWindow = new TimeSeriesWindow(windowingProcessor, null);
+      currentWindow.initWindow(
+          intermediateResultName2OperatorSupplierMap,
+          aggregatorOutputName2OperatorMap,
+          systemParameters);
+      currentWindow.deserialize(buffer);
+      currentOpeningWindows.add(currentWindow);
+    }
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/AggregatedResultOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/AggregatedResultOperator.java
new file mode 100644
index 00000000000..1d5de468d0f
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/AggregatedResultOperator.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.CustomizedReadableIntermediateResults;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * The operator to calculate an aggregator. There shall be a one-to-one match between an
+ * aggregator's static attributes and its name.
+ *
+ * <p>Note that there will only be one operator used for calculation and the operator itself is
+ * stateless, and therefore doesn't need any ser/de functions.
+ */
+public interface AggregatedResultOperator {
+
+  /**
+   * Return the name of the operator, the name shall be in lower case
+   *
+   * @return the name of the operator
+   */
+  String getName();
+
+  /**
+   * The system will pass in some parameters (e.g. timestamp precision) to help the inner function
+   * correctly operate. This will be called at the very first of the operator's lifecycle.
+   *
+   * @param systemParams the system parameters
+   */
+  void configureSystemParameters(Map<String, String> systemParams);
+
+  /** Get the needed intermediate value names of this aggregate result. */
+  Set<String> getDeclaredIntermediateValueNames();
+
+  /**
+   * terminateWindow receives a Map to get intermediate results by its names, and produce a
+   * characteristic value of this window.
+   */
+  Pair<TSDataType, Object> terminateWindow(
+      TSDataType measurementDataType, CustomizedReadableIntermediateResults intermediateResults);
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/AverageOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/AverageOperator.java
new file mode 100644
index 00000000000..9d4d42c74ed
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/AverageOperator.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.CustomizedReadableIntermediateResults;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class AverageOperator implements AggregatedResultOperator {
+  @Override
+  public String getName() {
+    return "avg";
+  }
+
+  @Override
+  public void configureSystemParameters(Map<String, String> systemParams) {
+    // Do nothing
+  }
+
+  @Override
+  public Set<String> getDeclaredIntermediateValueNames() {
+    return Collections.unmodifiableSet(new HashSet<>(Arrays.asList("sum_x1", "count")));
+  }
+
+  @Override
+  public Pair<TSDataType, Object> terminateWindow(
+      TSDataType measurementDataType, CustomizedReadableIntermediateResults intermediateResults) {
+    return new Pair<>(
+        TSDataType.DOUBLE,
+        intermediateResults.getDouble("sum_x1") / intermediateResults.getInt("count"));
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/ClearanceFactorOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/ClearanceFactorOperator.java
new file mode 100644
index 00000000000..b32d70cbd15
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/ClearanceFactorOperator.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.CustomizedReadableIntermediateResults;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class ClearanceFactorOperator implements AggregatedResultOperator {
+  @Override
+  public String getName() {
+    return "ce";
+  }
+
+  @Override
+  public void configureSystemParameters(Map<String, String> systemParams) {
+    // Do nothing
+  }
+
+  @Override
+  public Set<String> getDeclaredIntermediateValueNames() {
+    return Collections.unmodifiableSet(
+        new HashSet<>(Arrays.asList("abs_max", "sum_x0.5", "count")));
+  }
+
+  @Override
+  public Pair<TSDataType, Object> terminateWindow(
+      TSDataType measurementDataType, CustomizedReadableIntermediateResults intermediateResults) {
+    return new Pair<>(
+        TSDataType.DOUBLE,
+        intermediateResults.getDouble("abs_max")
+            / Math.pow(
+                intermediateResults.getDouble("sum_x0.5") / intermediateResults.getInt("count"),
+                2));
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/CrestFactorOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/CrestFactorOperator.java
new file mode 100644
index 00000000000..299923b1c47
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/CrestFactorOperator.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.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.CustomizedReadableIntermediateResults;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class CrestFactorOperator implements AggregatedResultOperator {
+  @Override
+  public String getName() {
+    return "cf";
+  }
+
+  @Override
+  public void configureSystemParameters(Map<String, String> systemParams) {
+    // Do nothing
+  }
+
+  @Override
+  public Set<String> getDeclaredIntermediateValueNames() {
+    return Collections.unmodifiableSet(new HashSet<>(Arrays.asList("abs_max", "sum_x2", "count")));
+  }
+
+  @Override
+  public Pair<TSDataType, Object> terminateWindow(
+      TSDataType measurementDataType, CustomizedReadableIntermediateResults intermediateResults) {
+    return new Pair<>(
+        TSDataType.DOUBLE,
+        intermediateResults.getDouble("abs_max")
+            / Math.pow(
+                intermediateResults.getDouble("sum_x2") / intermediateResults.getInt("count"),
+                0.5));
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/FormFactorOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/FormFactorOperator.java
new file mode 100644
index 00000000000..75975984d8e
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/FormFactorOperator.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.CustomizedReadableIntermediateResults;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class FormFactorOperator implements AggregatedResultOperator {
+  @Override
+  public String getName() {
+    return "ff";
+  }
+
+  @Override
+  public void configureSystemParameters(Map<String, String> systemParams) {
+    // Do nothing
+  }
+
+  @Override
+  public Set<String> getDeclaredIntermediateValueNames() {
+    return Collections.unmodifiableSet(new HashSet<>(Arrays.asList("sum_x2", "sum_x1", "count")));
+  }
+
+  @Override
+  public Pair<TSDataType, Object> terminateWindow(
+      TSDataType measurementDataType, CustomizedReadableIntermediateResults intermediateResults) {
+    return new Pair<>(
+        TSDataType.DOUBLE,
+        Math.pow(intermediateResults.getDouble("sum_x2") / intermediateResults.getInt("count"), 0.5)
+            / (intermediateResults.getDouble("sum_x1") / intermediateResults.getInt("count")));
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/KurtosisOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/KurtosisOperator.java
new file mode 100644
index 00000000000..9977c793da6
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/KurtosisOperator.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.CustomizedReadableIntermediateResults;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class KurtosisOperator implements AggregatedResultOperator {
+  @Override
+  public String getName() {
+    return "kurt";
+  }
+
+  @Override
+  public void configureSystemParameters(Map<String, String> systemParams) {
+    // Do nothing
+  }
+
+  @Override
+  public Set<String> getDeclaredIntermediateValueNames() {
+    return Collections.unmodifiableSet(
+        new HashSet<>(Arrays.asList("sum_x4", "sum_x3", "sum_x2", "sum_x1", "count")));
+  }
+
+  @Override
+  public Pair<TSDataType, Object> terminateWindow(
+      TSDataType measurementDataType, CustomizedReadableIntermediateResults intermediateResults) {
+    double sumX4 = intermediateResults.getDouble("sum_x4");
+    double sumX3 = intermediateResults.getDouble("sum_x3");
+    double sumX2 = intermediateResults.getDouble("sum_x2");
+    double sumX1 = intermediateResults.getDouble("sum_x1");
+    int count = intermediateResults.getInt("count");
+
+    return new Pair<>(
+        TSDataType.DOUBLE,
+        (sumX4 / count
+                - 4 * sumX3 / count * sumX1 / count
+                + 6 * sumX2 / count * Math.pow(sumX1 / count, 2)
+                - 3 * Math.pow(sumX1 / count, 4))
+            / Math.pow(sumX2 / count - Math.pow(sumX1 / count, 2), 2));
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/PeakOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/PeakOperator.java
new file mode 100644
index 00000000000..7dadcc8c9da
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/PeakOperator.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.CustomizedReadableIntermediateResults;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+public class PeakOperator implements AggregatedResultOperator {
+  @Override
+  public String getName() {
+    return "peak";
+  }
+
+  @Override
+  public void configureSystemParameters(Map<String, String> systemParams) {
+    // Do nothing
+  }
+
+  @Override
+  public Set<String> getDeclaredIntermediateValueNames() {
+    return Collections.singleton("abs_max");
+  }
+
+  @Override
+  public Pair<TSDataType, Object> terminateWindow(
+      TSDataType measurementDataType, CustomizedReadableIntermediateResults intermediateResults) {
+    return new Pair<>(measurementDataType, intermediateResults.getObject("abs_max"));
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/PulseFactorOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/PulseFactorOperator.java
new file mode 100644
index 00000000000..6ebbf5cc858
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/PulseFactorOperator.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.CustomizedReadableIntermediateResults;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class PulseFactorOperator implements AggregatedResultOperator {
+  @Override
+  public String getName() {
+    return "pf";
+  }
+
+  @Override
+  public void configureSystemParameters(Map<String, String> systemParams) {
+    // Do nothing
+  }
+
+  @Override
+  public Set<String> getDeclaredIntermediateValueNames() {
+    return Collections.unmodifiableSet(new HashSet<>(Arrays.asList("abs_max", "sum_x1", "count")));
+  }
+
+  @Override
+  public Pair<TSDataType, Object> terminateWindow(
+      TSDataType measurementDataType, CustomizedReadableIntermediateResults intermediateResults) {
+    return new Pair<>(
+        TSDataType.DOUBLE,
+        intermediateResults.getDouble("abs_max")
+            / (intermediateResults.getDouble("sum_x1") / intermediateResults.getInt("count")));
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/RootMeanSquareOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/RootMeanSquareOperator.java
new file mode 100644
index 00000000000..da689b5703d
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/RootMeanSquareOperator.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.CustomizedReadableIntermediateResults;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class RootMeanSquareOperator implements AggregatedResultOperator {
+  @Override
+  public String getName() {
+    return "rms";
+  }
+
+  @Override
+  public void configureSystemParameters(Map<String, String> systemParams) {
+    // Do nothing
+  }
+
+  @Override
+  public Set<String> getDeclaredIntermediateValueNames() {
+    return Collections.unmodifiableSet(new HashSet<>(Arrays.asList("sum_x2", "count")));
+  }
+
+  @Override
+  public Pair<TSDataType, Object> terminateWindow(
+      TSDataType measurementDataType, CustomizedReadableIntermediateResults intermediateResults) {
+    return new Pair<>(
+        TSDataType.DOUBLE,
+        Math.pow(
+            intermediateResults.getDouble("sum_x2") / intermediateResults.getInt("count"), 0.5));
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/SkewnessOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/SkewnessOperator.java
new file mode 100644
index 00000000000..4ef5000ee95
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/SkewnessOperator.java
@@ -0,0 +1,63 @@
+/*
+ * 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.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.CustomizedReadableIntermediateResults;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class SkewnessOperator implements AggregatedResultOperator {
+  @Override
+  public String getName() {
+    return "skew";
+  }
+
+  @Override
+  public void configureSystemParameters(Map<String, String> systemParams) {
+    // Do nothing
+  }
+
+  @Override
+  public Set<String> getDeclaredIntermediateValueNames() {
+    return Collections.unmodifiableSet(
+        new HashSet<>(Arrays.asList("sum_x3", "sum_x2", "sum_x1", "count")));
+  }
+
+  @Override
+  public Pair<TSDataType, Object> terminateWindow(
+      TSDataType measurementDataType, CustomizedReadableIntermediateResults intermediateResults) {
+    double sumX3 = intermediateResults.getDouble("sum_x3");
+    double sumX2 = intermediateResults.getDouble("sum_x2");
+    double sumX1 = intermediateResults.getDouble("sum_x1");
+    int count = intermediateResults.getInt("count");
+
+    return new Pair<>(
+        TSDataType.DOUBLE,
+        (sumX3 / count - 3 * sumX1 / count * sumX2 / count + 2 * Math.pow(sumX1 / count, 3))
+            / Math.pow(sumX2 / count - Math.pow(sumX1 / count, 2), 1.5));
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/VarianceOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/VarianceOperator.java
new file mode 100644
index 00000000000..ad0b46eb0db
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/aggregatedresult/standardstatistics/VarianceOperator.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.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.CustomizedReadableIntermediateResults;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class VarianceOperator implements AggregatedResultOperator {
+
+  @Override
+  public String getName() {
+    return "var";
+  }
+
+  @Override
+  public void configureSystemParameters(Map<String, String> systemParams) {
+    // Do nothing
+  }
+
+  @Override
+  public Set<String> getDeclaredIntermediateValueNames() {
+    return Collections.unmodifiableSet(new HashSet<>(Arrays.asList("sum_x2", "sum_x1", "count")));
+  }
+
+  @Override
+  public Pair<TSDataType, Object> terminateWindow(
+      TSDataType measurementDataType, CustomizedReadableIntermediateResults intermediateResults) {
+    return new Pair<>(
+        TSDataType.DOUBLE,
+        intermediateResults.getDouble("sum_x2") / intermediateResults.getInt("count")
+            - Math.pow(
+                intermediateResults.getDouble("sum_x1") / intermediateResults.getInt("count"), 2));
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/CustomizedReadableIntermediateResults.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/CustomizedReadableIntermediateResults.java
new file mode 100644
index 00000000000..1911fc1ff77
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/CustomizedReadableIntermediateResults.java
@@ -0,0 +1,158 @@
+/*
+ * 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.pipe.processor.aggregate.operator.intermediateresult;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Map;
+
+public class CustomizedReadableIntermediateResults {
+  private final Map<String, Pair<TSDataType, Object>> intermediateResults;
+
+  public CustomizedReadableIntermediateResults(
+      Map<String, Pair<TSDataType, Object>> intermediateResults) {
+    this.intermediateResults = intermediateResults;
+  }
+
+  public boolean getBoolean(String key) {
+    final Pair<TSDataType, Object> typeResultPair = intermediateResults.get(key);
+    if (typeResultPair.getLeft() == TSDataType.BOOLEAN) {
+      return (boolean) typeResultPair.getRight();
+    }
+    throw new UnsupportedOperationException(
+        String.format("The type %s cannot be casted to boolean.", typeResultPair.getLeft()));
+  }
+
+  public int getInt(String key) {
+    final Pair<TSDataType, Object> typeResultPair = intermediateResults.get(key);
+
+    final TSDataType type = typeResultPair.getLeft();
+    final Object value = typeResultPair.getRight();
+    switch (type) {
+      case INT32:
+        return (int) value;
+      case INT64:
+        return (int) (long) value;
+      case FLOAT:
+        return (int) (float) value;
+      case DOUBLE:
+        return (int) (double) value;
+      default:
+        throw new UnsupportedOperationException(
+            String.format("The type %s cannot be casted to int.", typeResultPair.getLeft()));
+    }
+  }
+
+  public long getLong(String key) {
+    final Pair<TSDataType, Object> typeResultPair = intermediateResults.get(key);
+
+    final TSDataType type = typeResultPair.getLeft();
+    final Object value = typeResultPair.getRight();
+    switch (type) {
+      case INT32:
+        return (int) value;
+      case INT64:
+        return (long) value;
+      case FLOAT:
+        return (long) (float) value;
+      case DOUBLE:
+        return (long) (double) value;
+      default:
+        throw new UnsupportedOperationException(
+            String.format("The type %s cannot be casted to long.", typeResultPair.getLeft()));
+    }
+  }
+
+  public float getFloat(String key) {
+    final Pair<TSDataType, Object> typeResultPair = intermediateResults.get(key);
+
+    final TSDataType type = typeResultPair.getLeft();
+    final Object value = typeResultPair.getRight();
+    switch (type) {
+      case INT32:
+        return (int) value;
+      case INT64:
+        return (long) value;
+      case FLOAT:
+        return (float) value;
+      case DOUBLE:
+        return (float) (double) value;
+      default:
+        throw new UnsupportedOperationException(
+            String.format("The type %s cannot be casted to float.", typeResultPair.getLeft()));
+    }
+  }
+
+  public double getDouble(String key) {
+    final Pair<TSDataType, Object> typeResultPair = intermediateResults.get(key);
+
+    final TSDataType type = typeResultPair.getLeft();
+    final Object value = typeResultPair.getRight();
+    switch (type) {
+      case INT32:
+        return (int) value;
+      case INT64:
+        return (long) value;
+      case FLOAT:
+        return (float) value;
+      case DOUBLE:
+        return (double) value;
+      default:
+        throw new UnsupportedOperationException(
+            String.format("The type %s cannot be casted to double.", typeResultPair.getLeft()));
+    }
+  }
+
+  // Note: This method will cast any decimal types to string without throwing
+  // any exceptions.
+  public String getString(String key) {
+    final Pair<TSDataType, Object> typeResultPair = intermediateResults.get(key);
+
+    final TSDataType type = typeResultPair.getLeft();
+    final Object value = typeResultPair.getRight();
+    switch (type) {
+      case BOOLEAN:
+        return Boolean.toString((boolean) value);
+      case INT32:
+        return Integer.toString((int) value);
+      case INT64:
+        return Long.toString((long) value);
+      case FLOAT:
+        return Float.toString((float) value);
+      case DOUBLE:
+        return Double.toString((double) value);
+      case TEXT:
+        return (String) value;
+      default:
+        throw new UnsupportedOperationException(
+            String.format("The type %s cannot be casted to string.", typeResultPair.getLeft()));
+    }
+  }
+
+  // The caller may cast the object by itself.
+  public Object getObject(String key) {
+    return intermediateResults.get(key).getRight();
+  }
+
+  public TSDataType getType(String key) {
+    return intermediateResults.get(key).getLeft();
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/IntermediateResultOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/IntermediateResultOperator.java
new file mode 100644
index 00000000000..523d87af7eb
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/IntermediateResultOperator.java
@@ -0,0 +1,113 @@
+/*
+ * 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.pipe.processor.aggregate.operator.intermediateresult;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+/**
+ * The class that define the calculation and ser/de logic of an intermediate result. There shall be
+ * a one-to-one match between an intermediate result's operator and its name.
+ *
+ * <p>The {@link IntermediateResultOperator}'s update function is called once per point, thus it
+ * shall implement update functions with primitive type input and shall handle the intermediate
+ * value itself in order to better optimize the calculation and save resource.
+ *
+ * <p>Besides, the variables can also be passed in by the instantiating function, to help
+ * customizing calculation.
+ */
+public interface IntermediateResultOperator {
+
+  /**
+   * Return the name of the operator, the name shall be in lower case
+   *
+   * @return the name of the operator
+   */
+  String getName();
+
+  /**
+   * The system will pass in some parameters (e.g. timestamp precision) to help the inner function
+   * correctly operate. This will be called at the very first of the operator's lifecycle.
+   *
+   * @param systemParams the system parameters
+   */
+  void configureSystemParameters(Map<String, String> systemParams);
+
+  /**
+   * The operator may init its value and typically set the output intermediate value type by the
+   * input data type. The output value type is arbitrary, such as a List, a Map, or any type that is
+   * needed in aggregation value calculation. You can even hold all the points in the output value
+   * and hand it to the aggregation function.
+   *
+   * <p>If the input type is unsupported, return {@code false}
+   *
+   * @param initialInput the initial data
+   * @return if the input type is supported
+   */
+  boolean initAndGetIsSupport(boolean initialInput, long initialTimestamp);
+
+  boolean initAndGetIsSupport(int initialInput, long initialTimestamp);
+
+  boolean initAndGetIsSupport(long initialInput, long initialTimestamp);
+
+  boolean initAndGetIsSupport(float initialInput, long initialTimestamp);
+
+  boolean initAndGetIsSupport(double initialInput, long initialTimestamp);
+
+  boolean initAndGetIsSupport(String initialInput, long initialTimestamp);
+
+  /**
+   * Use the input to update the intermediate result. The input is all raw types instead of Object
+   * to avoid the boxing and unboxing operations' resource occupation.
+   *
+   * @param input the inputs
+   */
+  void updateValue(boolean input, long timestamp);
+
+  void updateValue(int input, long timestamp);
+
+  void updateValue(long input, long timestamp);
+
+  void updateValue(float input, long timestamp);
+
+  void updateValue(double input, long timestamp);
+
+  void updateValue(String input, long timestamp);
+
+  /**
+   * Get the result and its type to calculate the aggregated value. If the type is List, Map or
+   * other non-included types please use {@link TSDataType#UNKNOWN}.
+   */
+  Pair<TSDataType, Object> getResult();
+
+  /**
+   * Serialize its intermediate result to the outputStream to allow shutdown restart. The operator
+   * may as well serialize its own status.
+   */
+  void serialize(DataOutputStream outputStream) throws IOException;
+
+  /** Deserialize the object from byteBuffer to allow shutdown restart */
+  void deserialize(ByteBuffer byteBuffer) throws IOException;
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/sametype/numeric/AbsoluteMaxOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/sametype/numeric/AbsoluteMaxOperator.java
new file mode 100644
index 00000000000..db9e16f84a3
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/sametype/numeric/AbsoluteMaxOperator.java
@@ -0,0 +1,71 @@
+/*
+ * 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.pipe.processor.aggregate.operator.intermediateresult.sametype.numeric;
+
+public class AbsoluteMaxOperator extends AbstractSameTypeNumericOperator {
+  @Override
+  public String getName() {
+    return "abs_max";
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(int initialInput, long initialTimestamp) {
+    intValue = Math.abs(initialInput);
+    return super.initAndGetIsSupport(initialInput, initialTimestamp);
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(long initialInput, long initialTimestamp) {
+    longValue = Math.abs(initialInput);
+    return super.initAndGetIsSupport(initialInput, initialTimestamp);
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(float initialInput, long initialTimestamp) {
+    floatValue = Math.abs(initialInput);
+    return super.initAndGetIsSupport(initialInput, initialTimestamp);
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(double initialInput, long initialTimestamp) {
+    doubleValue = Math.abs(initialInput);
+    return super.initAndGetIsSupport(initialInput, initialTimestamp);
+  }
+
+  @Override
+  public void updateValue(int input, long timestamp) {
+    intValue = Math.max(intValue, Math.abs(input));
+  }
+
+  @Override
+  public void updateValue(long input, long timestamp) {
+    longValue = Math.max(longValue, Math.abs(input));
+  }
+
+  @Override
+  public void updateValue(float input, long timestamp) {
+    floatValue = Math.max(floatValue, Math.abs(input));
+  }
+
+  @Override
+  public void updateValue(double input, long timestamp) {
+    doubleValue = Math.max(doubleValue, Math.abs(input));
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/sametype/numeric/AbstractSameTypeNumericOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/sametype/numeric/AbstractSameTypeNumericOperator.java
new file mode 100644
index 00000000000..9dfa13f405e
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/sametype/numeric/AbstractSameTypeNumericOperator.java
@@ -0,0 +1,152 @@
+/*
+ * 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.pipe.processor.aggregate.operator.intermediateresult.sametype.numeric;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.IntermediateResultOperator;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+/**
+ * {@link AbstractSameTypeNumericOperator} is the parent class of all the operators where there
+ * value type is the same as the input data type, the input data type is a numeric type, and the
+ * initial value equals to the first input value.
+ */
+public abstract class AbstractSameTypeNumericOperator implements IntermediateResultOperator {
+  protected TSDataType outPutDataType;
+  protected int intValue;
+  protected long longValue;
+  protected float floatValue;
+  protected double doubleValue;
+
+  @Override
+  public void configureSystemParameters(Map<String, String> systemParams) {
+    // Do nothing
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(boolean initialInput, long initialTimestamp) {
+    return false;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(int initialInput, long initialTimestamp) {
+    outPutDataType = TSDataType.INT32;
+    return true;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(long initialInput, long initialTimestamp) {
+    outPutDataType = TSDataType.INT64;
+    return true;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(float initialInput, long initialTimestamp) {
+    outPutDataType = TSDataType.FLOAT;
+    return true;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(double initialInput, long initialTimestamp) {
+    outPutDataType = TSDataType.DOUBLE;
+    return true;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(String initialInput, long initialTimestamp) {
+    return false;
+  }
+
+  @Override
+  public void updateValue(boolean input, long timestamp) {
+    throw new UnsupportedOperationException(
+        "AbstractSameTypeNumericOperator does not support boolean input");
+  }
+
+  @Override
+  public void updateValue(String input, long timestamp) {
+    throw new UnsupportedOperationException(
+        "AbstractSameTypeNumericOperator does not support string input");
+  }
+
+  @Override
+  public Pair<TSDataType, Object> getResult() {
+    switch (outPutDataType) {
+      case INT32:
+        return new Pair<>(TSDataType.INT32, intValue);
+      case INT64:
+        return new Pair<>(TSDataType.INT64, longValue);
+      case FLOAT:
+        return new Pair<>(TSDataType.FLOAT, floatValue);
+      case DOUBLE:
+        return new Pair<>(TSDataType.DOUBLE, doubleValue);
+      default:
+        return null;
+    }
+  }
+
+  @Override
+  public void serialize(DataOutputStream outputStream) throws IOException {
+    outPutDataType.serializeTo(outputStream);
+    switch (outPutDataType) {
+      case INT32:
+        ReadWriteIOUtils.write(intValue, outputStream);
+        break;
+      case INT64:
+        ReadWriteIOUtils.write(longValue, outputStream);
+        break;
+      case FLOAT:
+        ReadWriteIOUtils.write(floatValue, outputStream);
+        break;
+      case DOUBLE:
+        ReadWriteIOUtils.write(doubleValue, outputStream);
+        break;
+      default:
+        throw new IOException(String.format("Unsupported output datatype %s", outPutDataType));
+    }
+  }
+
+  @Override
+  public void deserialize(ByteBuffer byteBuffer) throws IOException {
+    outPutDataType = TSDataType.deserializeFrom(byteBuffer);
+    switch (outPutDataType) {
+      case INT32:
+        intValue = ReadWriteIOUtils.readInt(byteBuffer);
+        break;
+      case INT64:
+        longValue = ReadWriteIOUtils.readLong(byteBuffer);
+        break;
+      case FLOAT:
+        floatValue = ReadWriteIOUtils.readFloat(byteBuffer);
+        break;
+      case DOUBLE:
+        doubleValue = ReadWriteIOUtils.readDouble(byteBuffer);
+        break;
+      default:
+        throw new IOException(String.format("Unsupported output datatype %s", outPutDataType));
+    }
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/sametype/numeric/IntegralPoweredSumOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/sametype/numeric/IntegralPoweredSumOperator.java
new file mode 100644
index 00000000000..f5b532319c1
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/sametype/numeric/IntegralPoweredSumOperator.java
@@ -0,0 +1,77 @@
+/*
+ * 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.pipe.processor.aggregate.operator.intermediateresult.sametype.numeric;
+
+public class IntegralPoweredSumOperator extends AbstractSameTypeNumericOperator {
+  private final transient int power;
+
+  public IntegralPoweredSumOperator(int power) {
+    this.power = power;
+  }
+
+  @Override
+  public String getName() {
+    return "sum_x" + power;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(int initialInput, long initialTimestamp) {
+    intValue = (int) Math.pow(initialInput, power);
+    return super.initAndGetIsSupport(initialInput, initialTimestamp);
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(long initialInput, long initialTimestamp) {
+    longValue = (long) Math.pow(initialInput, power);
+    return super.initAndGetIsSupport(initialInput, initialTimestamp);
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(float initialInput, long initialTimestamp) {
+    floatValue = (float) Math.pow(initialInput, power);
+    return super.initAndGetIsSupport(initialInput, initialTimestamp);
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(double initialInput, long initialTimestamp) {
+    doubleValue = Math.pow(initialInput, power);
+    return super.initAndGetIsSupport(initialInput, initialTimestamp);
+  }
+
+  @Override
+  public void updateValue(int input, long timestamp) {
+    intValue += (int) Math.pow(input, power);
+  }
+
+  @Override
+  public void updateValue(long input, long timestamp) {
+    longValue += (long) Math.pow(input, power);
+  }
+
+  @Override
+  public void updateValue(float input, long timestamp) {
+    floatValue += (float) Math.pow(input, power);
+  }
+
+  @Override
+  public void updateValue(double input, long timestamp) {
+    doubleValue += Math.pow(input, power);
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/specifictype/doubletype/FractionPoweredSumOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/specifictype/doubletype/FractionPoweredSumOperator.java
new file mode 100644
index 00000000000..16cb46cb19e
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/specifictype/doubletype/FractionPoweredSumOperator.java
@@ -0,0 +1,137 @@
+/*
+ * 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.pipe.processor.aggregate.operator.intermediateresult.specifictype.doubletype;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.IntermediateResultOperator;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+public class FractionPoweredSumOperator implements IntermediateResultOperator {
+  private double sum;
+  private final transient double power;
+
+  /**
+   * Warning: Do not use fraction values like "1/3" to initiate the operator, unless you know what
+   * the output is when printing the double power. For instance, use "0.3333" instead, and you can
+   * identify the operator's name as "sum_x0.3333".
+   *
+   * @param power the power to use
+   */
+  public FractionPoweredSumOperator(double power) {
+    this.power = power;
+  }
+
+  @Override
+  public String getName() {
+    return "sum_x" + power;
+  }
+
+  @Override
+  public void configureSystemParameters(Map<String, String> systemParams) {
+    // Do nothing
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(boolean initialInput, long initialTimestamp) {
+    return false;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(int initialInput, long initialTimestamp) {
+    sum = Math.pow(initialInput, power);
+    return true;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(long initialInput, long initialTimestamp) {
+    sum = Math.pow(initialInput, power);
+    return true;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(float initialInput, long initialTimestamp) {
+    sum = Math.pow(initialInput, power);
+    return true;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(double initialInput, long initialTimestamp) {
+    sum = Math.pow(initialInput, power);
+    return true;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(String initialInput, long initialTimestamp) {
+    return false;
+  }
+
+  @Override
+  public void updateValue(boolean input, long timestamp) {
+    throw new UnsupportedOperationException(
+        "FractionPoweredSumOperator does not support boolean input");
+  }
+
+  @Override
+  public void updateValue(int input, long timestamp) {
+    sum += Math.pow(input, power);
+  }
+
+  @Override
+  public void updateValue(long input, long timestamp) {
+    sum += Math.pow(input, power);
+  }
+
+  @Override
+  public void updateValue(float input, long timestamp) {
+    sum += Math.pow(input, power);
+  }
+
+  @Override
+  public void updateValue(double input, long timestamp) {
+    sum += Math.pow(input, power);
+  }
+
+  @Override
+  public void updateValue(String input, long timestamp) {
+    throw new UnsupportedOperationException(
+        "FractionPoweredSumOperator does not support string input");
+  }
+
+  @Override
+  public Pair<TSDataType, Object> getResult() {
+    return new Pair<>(TSDataType.DOUBLE, sum);
+  }
+
+  @Override
+  public void serialize(DataOutputStream outputStream) throws IOException {
+    ReadWriteIOUtils.write(sum, outputStream);
+  }
+
+  @Override
+  public void deserialize(ByteBuffer byteBuffer) throws IOException {
+    sum = ReadWriteIOUtils.readDouble(byteBuffer);
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/specifictype/integertype/CountOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/specifictype/integertype/CountOperator.java
new file mode 100644
index 00000000000..47f42b8ce59
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/intermediateresult/specifictype/integertype/CountOperator.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.pipe.processor.aggregate.operator.intermediateresult.specifictype.integertype;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.IntermediateResultOperator;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+public class CountOperator implements IntermediateResultOperator {
+  private int count;
+
+  @Override
+  public String getName() {
+    return "count";
+  }
+
+  @Override
+  public void configureSystemParameters(Map<String, String> systemParams) {
+    // Do nothing
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(boolean initialInput, long initialTimestamp) {
+    count = 1;
+    return true;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(int initialInput, long initialTimestamp) {
+    count = 1;
+    return true;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(long initialInput, long initialTimestamp) {
+    count = 1;
+    return true;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(float initialInput, long initialTimestamp) {
+    count = 1;
+    return true;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(double initialInput, long initialTimestamp) {
+    count = 1;
+    return true;
+  }
+
+  @Override
+  public boolean initAndGetIsSupport(String initialInput, long initialTimestamp) {
+    count = 1;
+    return true;
+  }
+
+  @Override
+  public void updateValue(boolean input, long timestamp) {
+    ++count;
+  }
+
+  @Override
+  public void updateValue(int input, long timestamp) {
+    ++count;
+  }
+
+  @Override
+  public void updateValue(long input, long timestamp) {
+    ++count;
+  }
+
+  @Override
+  public void updateValue(float input, long timestamp) {
+    ++count;
+  }
+
+  @Override
+  public void updateValue(double input, long timestamp) {
+    ++count;
+  }
+
+  @Override
+  public void updateValue(String input, long timestamp) {
+    ++count;
+  }
+
+  @Override
+  public Pair<TSDataType, Object> getResult() {
+    return new Pair<>(TSDataType.INT32, count);
+  }
+
+  @Override
+  public void serialize(DataOutputStream outputStream) throws IOException {
+    ReadWriteIOUtils.write(count, outputStream);
+  }
+
+  @Override
+  public void deserialize(ByteBuffer byteBuffer) throws IOException {
+    count = ReadWriteIOUtils.readInt(byteBuffer);
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/processor/AbstractOperatorProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/processor/AbstractOperatorProcessor.java
new file mode 100644
index 00000000000..d4d935bf123
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/processor/AbstractOperatorProcessor.java
@@ -0,0 +1,73 @@
+/*
+ * 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.pipe.processor.aggregate.operator.processor;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.AbstractFormalProcessor;
+import org.apache.iotdb.db.pipe.processor.aggregate.AggregateProcessor;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.IntermediateResultOperator;
+import org.apache.iotdb.pipe.api.customizer.configuration.PipeProcessorRuntimeConfiguration;
+import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator;
+import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters;
+
+import java.util.Set;
+import java.util.function.Supplier;
+
+/**
+ * {@link AbstractOperatorProcessor} is the formal processor defining the operators adoptable for
+ * {@link AggregateProcessor}.
+ */
+public abstract class AbstractOperatorProcessor extends AbstractFormalProcessor {
+  @Override
+  public void validate(PipeParameterValidator validator) throws Exception {
+    // Do nothing by default
+  }
+
+  @Override
+  public void customize(PipeParameters parameters, PipeProcessorRuntimeConfiguration configuration)
+      throws Exception {
+    // Do nothing by default
+  }
+
+  @Override
+  public void close() throws Exception {
+    // Do nothing by default
+  }
+
+  /////////////////////////////// Child classes logic ///////////////////////////////
+
+  // Child classes must override these logics to be functional.
+  /**
+   * Get the supported aggregators and its corresponding {@link AggregatedResultOperator}.
+   *
+   * @return Map {@literal <}AggregatorName, {@link AggregatedResultOperator}{@literal >}
+   */
+  public abstract Set<AggregatedResultOperator> getAggregatorOperatorSet();
+
+  /**
+   * Get the supported intermediate results and its corresponding {@link
+   * IntermediateResultOperator}'s suppliers. The supplier is needed because the operator is
+   * stateful and used in abundance.
+   *
+   * @return Map {@literal <}AggregatorName, {@link IntermediateResultOperator}{@literal >}
+   */
+  public abstract Set<Supplier<IntermediateResultOperator>>
+      getIntermediateResultOperatorSupplierSet();
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/processor/StandardStatisticsOperatorProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/processor/StandardStatisticsOperatorProcessor.java
new file mode 100644
index 00000000000..fe010a0a1a1
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/operator/processor/StandardStatisticsOperatorProcessor.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.pipe.processor.aggregate.operator.processor;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics.AverageOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics.ClearanceFactorOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics.CrestFactorOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics.FormFactorOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics.KurtosisOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics.PeakOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics.PulseFactorOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics.RootMeanSquareOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics.SkewnessOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.standardstatistics.VarianceOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.IntermediateResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.sametype.numeric.AbsoluteMaxOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.sametype.numeric.IntegralPoweredSumOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.specifictype.doubletype.FractionPoweredSumOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.specifictype.integertype.CountOperator;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.function.Supplier;
+
+public class StandardStatisticsOperatorProcessor extends AbstractOperatorProcessor {
+  @Override
+  public Set<AggregatedResultOperator> getAggregatorOperatorSet() {
+    return Collections.unmodifiableSet(
+        new HashSet<>(
+            Arrays.asList(
+                new AverageOperator(),
+                new ClearanceFactorOperator(),
+                new CrestFactorOperator(),
+                new FormFactorOperator(),
+                new KurtosisOperator(),
+                new PeakOperator(),
+                new PulseFactorOperator(),
+                new RootMeanSquareOperator(),
+                new SkewnessOperator(),
+                new VarianceOperator())));
+  }
+
+  @Override
+  public Set<Supplier<IntermediateResultOperator>> getIntermediateResultOperatorSupplierSet() {
+    return Collections.unmodifiableSet(
+        new HashSet<>(
+            Arrays.asList(
+                AbsoluteMaxOperator::new,
+                CountOperator::new,
+                () -> new FractionPoweredSumOperator(0.5),
+                () -> new IntegralPoweredSumOperator(1),
+                () -> new IntegralPoweredSumOperator(2),
+                () -> new IntegralPoweredSumOperator(3),
+                () -> new IntegralPoweredSumOperator(4))));
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/datastructure/TimeSeriesWindow.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/datastructure/TimeSeriesWindow.java
new file mode 100644
index 00000000000..510d7323055
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/datastructure/TimeSeriesWindow.java
@@ -0,0 +1,495 @@
+/*
+ * 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.pipe.processor.aggregate.window.datastructure;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.aggregatedresult.AggregatedResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.CustomizedReadableIntermediateResults;
+import org.apache.iotdb.db.pipe.processor.aggregate.operator.intermediateresult.IntermediateResultOperator;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.processor.AbstractWindowingProcessor;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+public class TimeSeriesWindow {
+  private static final Logger LOGGER = LoggerFactory.getLogger(TimeSeriesWindow.class);
+
+  // A window is typically with a timestamp. We define it here to avoid
+  // boxing/unboxing and simplify the logics.
+  private long timestamp = 0;
+  private Map<String, AggregatedResultOperator> aggregatedOutputName2OperatorMap;
+  private final Map<String, Pair<TSDataType, IntermediateResultOperator>>
+      intermediateResultName2tsTypeAndOperatorMap = new HashMap<>();
+
+  // WARNING: Using the customized runtime value may cause performance loss
+  // due to boxing/unboxing issues.
+  private Object customizedRuntimeValue;
+  private final AbstractWindowingProcessor processor;
+
+  public TimeSeriesWindow(AbstractWindowingProcessor processor, Object customizedRuntimeValue) {
+    this.processor = processor;
+    this.customizedRuntimeValue = customizedRuntimeValue;
+  }
+
+  /////////////////////////////// Getter/Setters for WindowProcessor ///////////////////////////////
+
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  public void setTimestamp(long timestamp) {
+    this.timestamp = timestamp;
+  }
+
+  public Object getCustomizedRuntimeValue() {
+    return customizedRuntimeValue;
+  }
+
+  public void setCustomizedRuntimeValue(Object customizedRuntimeValue) {
+    this.customizedRuntimeValue = customizedRuntimeValue;
+  }
+
+  /////////////////////////////// Calculation ///////////////////////////////
+
+  public void initWindow(
+      Map<String, Supplier<IntermediateResultOperator>> intermediateResult2OperatorSupplierMap,
+      Map<String, AggregatedResultOperator> aggregatedResultOperatorMap,
+      Map<String, String> systemParameters) {
+    for (Map.Entry<String, Supplier<IntermediateResultOperator>> entry :
+        intermediateResult2OperatorSupplierMap.entrySet()) {
+      intermediateResultName2tsTypeAndOperatorMap.put(
+          entry.getKey(), new Pair<>(TSDataType.UNKNOWN, entry.getValue().get()));
+    }
+    // Deep copy because some unsupported aggregated results may be removed
+    this.aggregatedOutputName2OperatorMap = new HashMap<>(aggregatedResultOperatorMap);
+    // Configure system parameters
+    this.intermediateResultName2tsTypeAndOperatorMap.values().stream()
+        .map(Pair::getRight)
+        .forEach(operator -> operator.configureSystemParameters(systemParameters));
+    this.aggregatedOutputName2OperatorMap
+        .values()
+        .forEach(operator -> operator.configureSystemParameters(systemParameters));
+  }
+
+  // Return the output and state of the window.
+  // Return null if the state is normal to avoid boxing.
+  public Pair<WindowState, WindowOutput> updateIntermediateResult(long timestamp, boolean value) {
+    final Pair<WindowState, WindowOutput> stateOutputPair =
+        processor.updateAndMaySetWindowState(this, timestamp, value);
+    final WindowState state = stateOutputPair.getLeft();
+
+    if (state.isEmitWithoutCompute()) {
+      stateOutputPair.getRight().setAggregatedResults(getAggregatedResults(TSDataType.BOOLEAN));
+    }
+
+    if (state.isCalculate()) {
+      final Iterator<Map.Entry<String, Pair<TSDataType, IntermediateResultOperator>>> iterator =
+          intermediateResultName2tsTypeAndOperatorMap.entrySet().iterator();
+      Map.Entry<String, Pair<TSDataType, IntermediateResultOperator>> entry;
+
+      while (iterator.hasNext()) {
+        entry = iterator.next();
+        final IntermediateResultOperator operator = entry.getValue().getRight();
+        if (entry.getValue().getLeft() == TSDataType.UNKNOWN) {
+          if (!operator.initAndGetIsSupport(value, timestamp)) {
+            // Remove unsupported aggregated results
+            aggregatedOutputName2OperatorMap
+                .entrySet()
+                .removeIf(
+                    entry1 ->
+                        entry1
+                            .getValue()
+                            .getDeclaredIntermediateValueNames()
+                            .contains(operator.getName()));
+            // If no aggregated values can be calculated, purge the window
+            if (aggregatedOutputName2OperatorMap.isEmpty()) {
+              return new Pair<>(WindowState.PURGE, null);
+            }
+            // Remove unsupported intermediate values
+            iterator.remove();
+            continue;
+          }
+          entry.getValue().setLeft(TSDataType.BOOLEAN);
+        } else if (entry.getValue().getLeft() != TSDataType.BOOLEAN) {
+          LOGGER.warn(
+              "Different data type encountered in one window, will purge. Previous type: {}, now type: {}",
+              entry.getValue().getLeft(),
+              TSDataType.BOOLEAN);
+          return new Pair<>(WindowState.PURGE, null);
+        } else {
+          operator.updateValue(value, timestamp);
+        }
+      }
+    }
+    if (state.isEmitWithCompute()) {
+      stateOutputPair.getRight().setAggregatedResults(getAggregatedResults(TSDataType.BOOLEAN));
+    }
+    return state.isEmit() ? stateOutputPair : null;
+  }
+
+  // The same logic is repeated because java does not support basic type template :-)
+  public Pair<WindowState, WindowOutput> updateIntermediateResult(long timestamp, int value) {
+    final Pair<WindowState, WindowOutput> stateOutputPair =
+        processor.updateAndMaySetWindowState(this, timestamp, value);
+    final WindowState state = stateOutputPair.getLeft();
+
+    if (state.isEmitWithoutCompute()) {
+      stateOutputPair.getRight().setAggregatedResults(getAggregatedResults(TSDataType.INT32));
+    }
+
+    if (state.isCalculate()) {
+      final Iterator<Map.Entry<String, Pair<TSDataType, IntermediateResultOperator>>> iterator =
+          intermediateResultName2tsTypeAndOperatorMap.entrySet().iterator();
+      Map.Entry<String, Pair<TSDataType, IntermediateResultOperator>> entry;
+
+      while (iterator.hasNext()) {
+        entry = iterator.next();
+        final IntermediateResultOperator operator = entry.getValue().getRight();
+        if (entry.getValue().getLeft() == TSDataType.UNKNOWN) {
+          if (!operator.initAndGetIsSupport(value, timestamp)) {
+            // Remove unsupported aggregated results
+            aggregatedOutputName2OperatorMap
+                .entrySet()
+                .removeIf(
+                    entry1 ->
+                        entry1
+                            .getValue()
+                            .getDeclaredIntermediateValueNames()
+                            .contains(operator.getName()));
+            // If no aggregated values can be calculated, purge the window
+            if (aggregatedOutputName2OperatorMap.isEmpty()) {
+              return new Pair<>(WindowState.PURGE, null);
+            }
+            // Remove unsupported intermediate values
+            iterator.remove();
+            continue;
+          }
+          entry.getValue().setLeft(TSDataType.INT32);
+        } else if (entry.getValue().getLeft() != TSDataType.INT32) {
+          LOGGER.warn(
+              "Different data type encountered in one window, will purge. Previous type: {}, now type: {}",
+              entry.getValue().getLeft(),
+              TSDataType.INT32);
+          return new Pair<>(WindowState.PURGE, null);
+        } else {
+          operator.updateValue(value, timestamp);
+        }
+      }
+    }
+    if (state.isEmitWithCompute()) {
+      stateOutputPair.getRight().setAggregatedResults(getAggregatedResults(TSDataType.INT32));
+    }
+    return state.isEmit() ? stateOutputPair : null;
+  }
+
+  public Pair<WindowState, WindowOutput> updateIntermediateResult(long timestamp, long value) {
+    final Pair<WindowState, WindowOutput> stateOutputPair =
+        processor.updateAndMaySetWindowState(this, timestamp, value);
+    final WindowState state = stateOutputPair.getLeft();
+
+    if (state.isEmitWithoutCompute()) {
+      stateOutputPair.getRight().setAggregatedResults(getAggregatedResults(TSDataType.INT64));
+    }
+
+    if (state.isCalculate()) {
+      final Iterator<Map.Entry<String, Pair<TSDataType, IntermediateResultOperator>>> iterator =
+          intermediateResultName2tsTypeAndOperatorMap.entrySet().iterator();
+      Map.Entry<String, Pair<TSDataType, IntermediateResultOperator>> entry;
+
+      while (iterator.hasNext()) {
+        entry = iterator.next();
+        final IntermediateResultOperator operator = entry.getValue().getRight();
+        if (entry.getValue().getLeft() == TSDataType.UNKNOWN) {
+          if (!operator.initAndGetIsSupport(value, timestamp)) {
+            // Remove unsupported aggregated results
+            aggregatedOutputName2OperatorMap
+                .entrySet()
+                .removeIf(
+                    entry1 ->
+                        entry1
+                            .getValue()
+                            .getDeclaredIntermediateValueNames()
+                            .contains(operator.getName()));
+            // If no aggregated values can be calculated, purge the window
+            if (aggregatedOutputName2OperatorMap.isEmpty()) {
+              return new Pair<>(WindowState.PURGE, null);
+            }
+            // Remove unsupported intermediate values
+            iterator.remove();
+            continue;
+          }
+          entry.getValue().setLeft(TSDataType.INT64);
+        } else if (entry.getValue().getLeft() != TSDataType.INT64) {
+          LOGGER.warn(
+              "Different data type encountered in one window, will purge. Previous type: {}, now type: {}",
+              entry.getValue().getLeft(),
+              TSDataType.INT64);
+          return new Pair<>(WindowState.PURGE, null);
+        } else {
+          operator.updateValue(value, timestamp);
+        }
+      }
+    }
+    if (state.isEmitWithCompute()) {
+      stateOutputPair.getRight().setAggregatedResults(getAggregatedResults(TSDataType.INT64));
+    }
+    return state.isEmit() ? stateOutputPair : null;
+  }
+
+  public Pair<WindowState, WindowOutput> updateIntermediateResult(long timestamp, float value) {
+    final Pair<WindowState, WindowOutput> stateOutputPair =
+        processor.updateAndMaySetWindowState(this, timestamp, value);
+    final WindowState state = stateOutputPair.getLeft();
+
+    if (state.isEmitWithoutCompute()) {
+      stateOutputPair.getRight().setAggregatedResults(getAggregatedResults(TSDataType.FLOAT));
+    }
+
+    if (state.isCalculate()) {
+      final Iterator<Map.Entry<String, Pair<TSDataType, IntermediateResultOperator>>> iterator =
+          intermediateResultName2tsTypeAndOperatorMap.entrySet().iterator();
+      Map.Entry<String, Pair<TSDataType, IntermediateResultOperator>> entry;
+
+      while (iterator.hasNext()) {
+        entry = iterator.next();
+        final IntermediateResultOperator operator = entry.getValue().getRight();
+        if (entry.getValue().getLeft() == TSDataType.UNKNOWN) {
+          if (!operator.initAndGetIsSupport(value, timestamp)) {
+            // Remove unsupported aggregated results
+            aggregatedOutputName2OperatorMap
+                .entrySet()
+                .removeIf(
+                    entry1 ->
+                        entry1
+                            .getValue()
+                            .getDeclaredIntermediateValueNames()
+                            .contains(operator.getName()));
+            // If no aggregated values can be calculated, purge the window
+            if (aggregatedOutputName2OperatorMap.isEmpty()) {
+              return new Pair<>(WindowState.PURGE, null);
+            }
+            // Remove unsupported intermediate values
+            iterator.remove();
+            continue;
+          }
+          entry.getValue().setLeft(TSDataType.FLOAT);
+        } else if (entry.getValue().getLeft() != TSDataType.FLOAT) {
+          LOGGER.warn(
+              "Different data type encountered in one window, will purge. Previous type: {}, now type: {}",
+              entry.getValue().getLeft(),
+              TSDataType.FLOAT);
+          return new Pair<>(WindowState.PURGE, null);
+        } else {
+          operator.updateValue(value, timestamp);
+        }
+      }
+    }
+    if (state.isEmitWithCompute()) {
+      stateOutputPair.getRight().setAggregatedResults(getAggregatedResults(TSDataType.FLOAT));
+    }
+    return state.isEmit() ? stateOutputPair : null;
+  }
+
+  public Pair<WindowState, WindowOutput> updateIntermediateResult(long timestamp, double value) {
+    final Pair<WindowState, WindowOutput> stateOutputPair =
+        processor.updateAndMaySetWindowState(this, timestamp, value);
+    final WindowState state = stateOutputPair.getLeft();
+
+    if (state.isEmitWithoutCompute()) {
+      stateOutputPair.getRight().setAggregatedResults(getAggregatedResults(TSDataType.DOUBLE));
+    }
+    if (state.isCalculate()) {
+      final Iterator<Map.Entry<String, Pair<TSDataType, IntermediateResultOperator>>> iterator =
+          intermediateResultName2tsTypeAndOperatorMap.entrySet().iterator();
+      Map.Entry<String, Pair<TSDataType, IntermediateResultOperator>> entry;
+
+      while (iterator.hasNext()) {
+        entry = iterator.next();
+        final IntermediateResultOperator operator = entry.getValue().getRight();
+        if (entry.getValue().getLeft() == TSDataType.UNKNOWN) {
+          if (!operator.initAndGetIsSupport(value, timestamp)) {
+            // Remove unsupported aggregated results
+            aggregatedOutputName2OperatorMap
+                .entrySet()
+                .removeIf(
+                    entry1 ->
+                        entry1
+                            .getValue()
+                            .getDeclaredIntermediateValueNames()
+                            .contains(operator.getName()));
+            // If no aggregated values can be calculated, purge the window
+            if (aggregatedOutputName2OperatorMap.isEmpty()) {
+              return new Pair<>(WindowState.PURGE, null);
+            }
+            // Remove unsupported intermediate values
+            iterator.remove();
+            continue;
+          }
+          entry.getValue().setLeft(TSDataType.DOUBLE);
+        } else if (entry.getValue().getLeft() != TSDataType.DOUBLE) {
+          LOGGER.warn(
+              "Different data type encountered in one window, will purge. Previous type: {}, now type: {}",
+              entry.getValue().getLeft(),
+              TSDataType.DOUBLE);
+          return new Pair<>(WindowState.PURGE, null);
+        } else {
+          operator.updateValue(value, timestamp);
+        }
+      }
+    }
+    if (state.isEmitWithCompute()) {
+      stateOutputPair.getRight().setAggregatedResults(getAggregatedResults(TSDataType.DOUBLE));
+    }
+    return state.isEmit() ? stateOutputPair : null;
+  }
+
+  public Pair<WindowState, WindowOutput> updateIntermediateResult(long timestamp, String value) {
+    final Pair<WindowState, WindowOutput> stateOutputPair =
+        processor.updateAndMaySetWindowState(this, timestamp, value);
+    final WindowState state = stateOutputPair.getLeft();
+
+    if (state.isEmitWithoutCompute()) {
+      stateOutputPair.getRight().setAggregatedResults(getAggregatedResults(TSDataType.TEXT));
+    }
+    if (state.isCalculate()) {
+      final Iterator<Map.Entry<String, Pair<TSDataType, IntermediateResultOperator>>> iterator =
+          intermediateResultName2tsTypeAndOperatorMap.entrySet().iterator();
+      Map.Entry<String, Pair<TSDataType, IntermediateResultOperator>> entry;
+
+      while (iterator.hasNext()) {
+        entry = iterator.next();
+        final IntermediateResultOperator operator = entry.getValue().getRight();
+        if (entry.getValue().getLeft() == TSDataType.UNKNOWN) {
+          if (!operator.initAndGetIsSupport(value, timestamp)) {
+            // Remove unsupported aggregated results
+            aggregatedOutputName2OperatorMap
+                .entrySet()
+                .removeIf(
+                    entry1 ->
+                        entry1
+                            .getValue()
+                            .getDeclaredIntermediateValueNames()
+                            .contains(operator.getName()));
+            // If no aggregated values can be calculated, purge the window
+            if (aggregatedOutputName2OperatorMap.isEmpty()) {
+              return new Pair<>(WindowState.PURGE, null);
+            }
+            // Remove unsupported intermediate values
+            iterator.remove();
+            continue;
+          }
+          entry.getValue().setLeft(TSDataType.TEXT);
+        } else if (entry.getValue().getLeft() != TSDataType.TEXT) {
+          LOGGER.warn(
+              "Different data type encountered in one window, will purge. Previous type: {}, now type: {}",
+              entry.getValue().getLeft(),
+              TSDataType.TEXT);
+          return new Pair<>(WindowState.PURGE, null);
+        } else {
+          operator.updateValue(value, timestamp);
+        }
+      }
+    }
+    if (state.isEmitWithCompute()) {
+      stateOutputPair.getRight().setAggregatedResults(getAggregatedResults(TSDataType.TEXT));
+    }
+    return state.isEmit() ? stateOutputPair : null;
+  }
+
+  public WindowOutput forceOutput() {
+    return processor
+        .forceOutput(this)
+        .setAggregatedResults(
+            getAggregatedResults(
+                intermediateResultName2tsTypeAndOperatorMap.values().stream()
+                    .findFirst()
+                    .map(Pair::getLeft)
+                    .orElse(TSDataType.UNKNOWN)));
+  }
+
+  private Map<String, Pair<TSDataType, Object>> getAggregatedResults(TSDataType dataType) {
+    // The remaining intermediate results' datatype shall all be equal to this
+    // If not, return nothing
+    if (dataType == TSDataType.UNKNOWN
+        || intermediateResultName2tsTypeAndOperatorMap.entrySet().stream()
+            .anyMatch(entry -> entry.getValue().getLeft() != dataType)) {
+      return Collections.emptyMap();
+    }
+    final CustomizedReadableIntermediateResults readableIntermediateResults =
+        new CustomizedReadableIntermediateResults(
+            intermediateResultName2tsTypeAndOperatorMap.entrySet().stream()
+                .collect(
+                    Collectors.toMap(
+                        Map.Entry::getKey, entry -> entry.getValue().getRight().getResult())));
+    return aggregatedOutputName2OperatorMap.entrySet().stream()
+        .collect(
+            Collectors.toMap(
+                Map.Entry::getKey,
+                entry -> entry.getValue().terminateWindow(dataType, readableIntermediateResults)));
+  }
+
+  /////////////////////////////// Ser/De logics ///////////////////////////////
+
+  public void serialize(DataOutputStream outputStream) throws IOException {
+    ReadWriteIOUtils.write(timestamp, outputStream);
+    ReadWriteIOUtils.write(intermediateResultName2tsTypeAndOperatorMap.size(), outputStream);
+    for (Map.Entry<String, Pair<TSDataType, IntermediateResultOperator>> entry :
+        intermediateResultName2tsTypeAndOperatorMap.entrySet()) {
+      ReadWriteIOUtils.write(entry.getKey(), outputStream);
+      entry.getValue().getLeft().serializeTo(outputStream);
+      entry.getValue().getRight().serialize(outputStream);
+    }
+    processor.serializeCustomizedAttributes(this, outputStream);
+  }
+
+  // Unlike normal deserialization method, the pipe may be altered before deserialization,
+  // which means the operators may increase or decrease. Hence, we need to combine the
+  // deserialized value and existing entries.
+  // WARNING: We do not support removing intermediate values (e.g. intermediate values are
+  // less after the aggregators decreased) or altering windowing processor in altering aggregate
+  // processor, only adding intermediate values is permitted.
+  public void deserialize(ByteBuffer byteBuffer) throws IOException {
+    timestamp = ReadWriteIOUtils.readLong(byteBuffer);
+    final int size = ReadWriteIOUtils.readInt(byteBuffer);
+    for (int i = 0; i < size; i++) {
+      final String intermediateResultName = ReadWriteIOUtils.readString(byteBuffer);
+      final Pair<TSDataType, IntermediateResultOperator> initializedAndOperatorPair =
+          intermediateResultName2tsTypeAndOperatorMap.get(intermediateResultName);
+      if (Objects.nonNull(initializedAndOperatorPair)) {
+        initializedAndOperatorPair.setLeft(TSDataType.deserializeFrom(byteBuffer));
+        initializedAndOperatorPair.getRight().deserialize(byteBuffer);
+      }
+    }
+    processor.deserializeCustomizedAttributes(this, byteBuffer);
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/datastructure/WindowOutput.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/datastructure/WindowOutput.java
new file mode 100644
index 00000000000..d957feaa8aa
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/datastructure/WindowOutput.java
@@ -0,0 +1,71 @@
+/*
+ * 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.pipe.processor.aggregate.window.datastructure;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Map;
+
+public class WindowOutput {
+  private long timestamp;
+  private long progressTime;
+  private Map<String, Pair<TSDataType, Object>> aggregatedResults;
+
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  public WindowOutput setTimestamp(long timestamp) {
+    this.timestamp = timestamp;
+    return this;
+  }
+
+  public long getProgressTime() {
+    return progressTime;
+  }
+
+  public WindowOutput setProgressTime(long progressTime) {
+    this.progressTime = progressTime;
+    return this;
+  }
+
+  public Map<String, Pair<TSDataType, Object>> getAggregatedResults() {
+    return aggregatedResults;
+  }
+
+  public WindowOutput setAggregatedResults(
+      Map<String, Pair<TSDataType, Object>> aggregatedResults) {
+    this.aggregatedResults = aggregatedResults;
+    return this;
+  }
+
+  @Override
+  public String toString() {
+    return "WindowOutput{"
+        + "timestamp='"
+        + timestamp
+        + "', progressTime='"
+        + progressTime
+        + "', aggregatedResults='"
+        + aggregatedResults
+        + "'}";
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/datastructure/WindowState.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/datastructure/WindowState.java
new file mode 100644
index 00000000000..7370d9e6b10
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/datastructure/WindowState.java
@@ -0,0 +1,75 @@
+/*
+ * 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.pipe.processor.aggregate.window.datastructure;
+
+public enum WindowState {
+
+  // Normally calculate the value
+  COMPUTE,
+
+  // Do nothing to the value
+  IGNORE_VALUE,
+
+  // This value is not calculated in this round of emit.
+  // For example, current window is [1, 2], 3 comes, emit [1, 2] and compute 3, do not wipe the
+  // window
+  EMIT_WITHOUT_COMPUTE,
+
+  // This value is calculated in this round of emit.
+  // For example, current window is [1, 2], 3 comes, emit [1, 2, 3] and compute 3, do not wipe the
+  // window
+  EMIT_WITH_COMPUTE,
+
+  // Purely wipe the window
+  PURGE,
+
+  // This value is not calculated in this round of emit, and wipe the window.
+  // For example, current window is [1, 2],  3 comes, emit [1, 2] and wipe the window
+  EMIT_AND_PURGE_WITHOUT_COMPUTE,
+
+  // This value is calculated in this round of emit, and wipe the window.
+  // For example, current window is [1, 2],  3 comes, emit [1, 2, 3] and wipe the window
+  EMIT_AND_PURGE_WITH_COMPUTE;
+
+  public boolean isEmit() {
+    return isEmitWithCompute() || isEmitWithoutCompute();
+  }
+
+  public boolean isEmitWithoutCompute() {
+    return this == EMIT_WITHOUT_COMPUTE || this == EMIT_AND_PURGE_WITHOUT_COMPUTE;
+  }
+
+  public boolean isEmitWithCompute() {
+    return this == EMIT_WITH_COMPUTE || this == EMIT_AND_PURGE_WITH_COMPUTE;
+  }
+
+  public boolean isPurge() {
+    return this == PURGE
+        || this == EMIT_AND_PURGE_WITHOUT_COMPUTE
+        || this == EMIT_AND_PURGE_WITH_COMPUTE;
+  }
+
+  public boolean isCalculate() {
+    return this == COMPUTE
+        || this == EMIT_WITHOUT_COMPUTE
+        || this == EMIT_WITH_COMPUTE
+        || this == EMIT_AND_PURGE_WITH_COMPUTE;
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/processor/AbstractSimpleTimeWindowingProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/processor/AbstractSimpleTimeWindowingProcessor.java
new file mode 100644
index 00000000000..aea23c94116
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/processor/AbstractSimpleTimeWindowingProcessor.java
@@ -0,0 +1,102 @@
+/*
+ * 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.pipe.processor.aggregate.window.processor;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.window.datastructure.TimeSeriesWindow;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.datastructure.WindowOutput;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.datastructure.WindowState;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.List;
+import java.util.Set;
+
+public abstract class AbstractSimpleTimeWindowingProcessor extends AbstractWindowingProcessor {
+  public Set<TimeSeriesWindow> mayAddWindow(
+      List<TimeSeriesWindow> windowList, long timeStamp, boolean value) {
+    return mayAddWindow(windowList, timeStamp);
+  }
+
+  public Set<TimeSeriesWindow> mayAddWindow(
+      List<TimeSeriesWindow> windowList, long timeStamp, int value) {
+    return mayAddWindow(windowList, timeStamp);
+  }
+
+  public Set<TimeSeriesWindow> mayAddWindow(
+      List<TimeSeriesWindow> windowList, long timeStamp, long value) {
+    return mayAddWindow(windowList, timeStamp);
+  }
+
+  public Set<TimeSeriesWindow> mayAddWindow(
+      List<TimeSeriesWindow> windowList, long timeStamp, float value) {
+    return mayAddWindow(windowList, timeStamp);
+  }
+
+  public Set<TimeSeriesWindow> mayAddWindow(
+      List<TimeSeriesWindow> windowList, long timeStamp, double value) {
+    return mayAddWindow(windowList, timeStamp);
+  }
+
+  public Set<TimeSeriesWindow> mayAddWindow(
+      List<TimeSeriesWindow> windowList, long timeStamp, String value) {
+    return mayAddWindow(windowList, timeStamp);
+  }
+
+  public abstract Set<TimeSeriesWindow> mayAddWindow(
+      List<TimeSeriesWindow> windowList, long timeStamp);
+
+  @Override
+  public Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp, boolean value) {
+    return updateAndMaySetWindowState(window, timeStamp);
+  }
+
+  @Override
+  public Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp, int value) {
+    return updateAndMaySetWindowState(window, timeStamp);
+  }
+
+  @Override
+  public Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp, long value) {
+    return updateAndMaySetWindowState(window, timeStamp);
+  }
+
+  @Override
+  public Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp, float value) {
+    return updateAndMaySetWindowState(window, timeStamp);
+  }
+
+  @Override
+  public Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp, double value) {
+    return updateAndMaySetWindowState(window, timeStamp);
+  }
+
+  @Override
+  public Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp, String value) {
+    return updateAndMaySetWindowState(window, timeStamp);
+  }
+
+  public abstract Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp);
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/processor/AbstractWindowingProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/processor/AbstractWindowingProcessor.java
new file mode 100644
index 00000000000..bdcca8d6446
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/processor/AbstractWindowingProcessor.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.pipe.processor.aggregate.window.processor;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.AbstractFormalProcessor;
+import org.apache.iotdb.db.pipe.processor.aggregate.AggregateProcessor;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.datastructure.TimeSeriesWindow;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.datastructure.WindowOutput;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.datastructure.WindowState;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.io.DataOutputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * {@link AbstractWindowingProcessor} is the formal processor defining the windows adoptable for
+ * {@link AggregateProcessor}.
+ */
+public abstract class AbstractWindowingProcessor extends AbstractFormalProcessor {
+
+  /**
+   * The {@link AbstractWindowingProcessor} may add {@link TimeSeriesWindow}s to the windowList,
+   * which will soon be configured by the {@link AggregateProcessor}. Typically only the timestamp
+   * is needed, however the {@link AbstractWindowingProcessor} can use the windowList and values to
+   * assist judgement.
+   *
+   * @return The added windows
+   */
+  public abstract Set<TimeSeriesWindow> mayAddWindow(
+      List<TimeSeriesWindow> windowList, long timeStamp, boolean value);
+
+  public abstract Set<TimeSeriesWindow> mayAddWindow(
+      List<TimeSeriesWindow> windowList, long timeStamp, int value);
+
+  public abstract Set<TimeSeriesWindow> mayAddWindow(
+      List<TimeSeriesWindow> windowList, long timeStamp, long value);
+
+  public abstract Set<TimeSeriesWindow> mayAddWindow(
+      List<TimeSeriesWindow> windowList, long timeStamp, float value);
+
+  public abstract Set<TimeSeriesWindow> mayAddWindow(
+      List<TimeSeriesWindow> windowList, long timeStamp, double value);
+
+  public abstract Set<TimeSeriesWindow> mayAddWindow(
+      List<TimeSeriesWindow> windowList, long timeStamp, String value);
+
+  /**
+   * The {@link AbstractWindowingProcessor} may decide whether a {@link TimeSeriesWindow} shall be
+   * terminated when a point is arrived. If yes, the {@link AbstractWindowingProcessor} shall set
+   * the pair of output timestamp, the time of the progressIndex to be reported, and whether the
+   * window is closed in this round. If not, it shall return {@code null}.
+   *
+   * @return The pair of {@link WindowState} and {@link WindowOutput}, the latter one including the
+   *     report time and output timestamp. Note that when a report time is submitted, the {@link
+   *     AbstractWindowingProcessor} may never see the time below when the system has been
+   *     restarted.
+   */
+  public abstract Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp, boolean value);
+
+  public abstract Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp, int value);
+
+  public abstract Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp, long value);
+
+  public abstract Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp, float value);
+
+  public abstract Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp, double value);
+
+  public abstract Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp, String value);
+
+  public abstract WindowOutput forceOutput(TimeSeriesWindow window);
+
+  /**
+   * Serialize the customized attributes in a window to an output stream
+   *
+   * @param window the window to serialize
+   * @param outputStream the outputStream
+   */
+  public void serializeCustomizedAttributes(
+      TimeSeriesWindow window, DataOutputStream outputStream) {
+    // Do nothing by default
+  }
+
+  /**
+   * Deserialize a customized attributes in a window from an output stream
+   *
+   * @param byteBuffer the customized attributes in one window
+   */
+  public void deserializeCustomizedAttributes(TimeSeriesWindow window, ByteBuffer byteBuffer) {
+    // Do nothing by default
+  }
+
+  @Override
+  public void close() throws Exception {}
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/processor/TumblingWindowingProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/processor/TumblingWindowingProcessor.java
new file mode 100644
index 00000000000..514509b9319
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/aggregate/window/processor/TumblingWindowingProcessor.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.pipe.processor.aggregate.window.processor;
+
+import org.apache.iotdb.db.pipe.processor.aggregate.window.datastructure.TimeSeriesWindow;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.datastructure.WindowOutput;
+import org.apache.iotdb.db.pipe.processor.aggregate.window.datastructure.WindowState;
+import org.apache.iotdb.db.utils.DateTimeUtils;
+import org.apache.iotdb.db.utils.TimestampPrecisionUtils;
+import org.apache.iotdb.pipe.api.customizer.configuration.PipeProcessorRuntimeConfiguration;
+import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator;
+import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_SLIDING_BOUNDARY_TIME_DEFAULT_VALUE;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_SLIDING_BOUNDARY_TIME_KEY;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_SLIDING_SECONDS_DEFAULT_VALUE;
+import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_SLIDING_SECONDS_KEY;
+
+public class TumblingWindowingProcessor extends AbstractSimpleTimeWindowingProcessor {
+  private long slidingBoundaryTime;
+  private long slidingInterval;
+
+  @Override
+  public void validate(PipeParameterValidator validator) throws Exception {
+    final PipeParameters parameters = validator.getParameters();
+    validator.validate(
+        args -> (long) args > 0,
+        String.format("The parameter %s must be greater than 0", PROCESSOR_SLIDING_SECONDS_KEY),
+        parameters.getLongOrDefault(
+            PROCESSOR_SLIDING_SECONDS_KEY, PROCESSOR_SLIDING_SECONDS_DEFAULT_VALUE));
+  }
+
+  @Override
+  public void customize(PipeParameters parameters, PipeProcessorRuntimeConfiguration configuration)
+      throws Exception {
+    slidingBoundaryTime =
+        parameters.hasAnyAttributes(PROCESSOR_SLIDING_BOUNDARY_TIME_KEY)
+            ? DateTimeUtils.convertTimestampOrDatetimeStrToLongWithDefaultZone(
+                parameters.getString(PROCESSOR_SLIDING_BOUNDARY_TIME_KEY))
+            : PROCESSOR_SLIDING_BOUNDARY_TIME_DEFAULT_VALUE;
+    slidingInterval =
+        TimestampPrecisionUtils.convertToCurrPrecision(
+            parameters.getLongOrDefault(
+                PROCESSOR_SLIDING_SECONDS_KEY, PROCESSOR_SLIDING_SECONDS_DEFAULT_VALUE),
+            TimeUnit.SECONDS);
+  }
+
+  @Override
+  public Set<TimeSeriesWindow> mayAddWindow(List<TimeSeriesWindow> windowList, long timeStamp) {
+    final long lastTime =
+        windowList.isEmpty()
+            ? slidingBoundaryTime
+            : windowList.get(windowList.size() - 1).getTimestamp();
+
+    if (timeStamp >= (windowList.isEmpty() ? lastTime : lastTime + slidingInterval)) {
+      final TimeSeriesWindow window = new TimeSeriesWindow(this, null);
+      // Align to the last time + k * slidingInterval, k is a natural number
+      window.setTimestamp(((timeStamp - lastTime) / slidingInterval) * slidingInterval + lastTime);
+      windowList.add(window);
+      return Collections.singleton(window);
+    }
+    return Collections.emptySet();
+  }
+
+  @Override
+  public Pair<WindowState, WindowOutput> updateAndMaySetWindowState(
+      TimeSeriesWindow window, long timeStamp) {
+    if (timeStamp < window.getTimestamp()) {
+      return new Pair<>(WindowState.IGNORE_VALUE, null);
+    }
+    if (timeStamp >= window.getTimestamp() + slidingInterval) {
+      return new Pair<>(
+          WindowState.EMIT_AND_PURGE_WITHOUT_COMPUTE,
+          new WindowOutput()
+              .setTimestamp(window.getTimestamp())
+              .setProgressTime(window.getTimestamp() + slidingInterval));
+    }
+    return new Pair<>(WindowState.COMPUTE, null);
+  }
+
+  @Override
+  public WindowOutput forceOutput(TimeSeriesWindow window) {
+    return new WindowOutput()
+        .setTimestamp(window.getTimestamp())
+        .setProgressTime(window.getTimestamp() + slidingInterval);
+  }
+}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/sdt/SwingingDoorTrendingFilter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/sdt/SwingingDoorTrendingFilter.java
index f80b189427a..bb4825b0d18 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/sdt/SwingingDoorTrendingFilter.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/sdt/SwingingDoorTrendingFilter.java
@@ -26,18 +26,18 @@ public class SwingingDoorTrendingFilter<T> {
   private final SwingingDoorTrendingSamplingProcessor processor;
 
   /**
-   * the maximum curUpperSlope between the lastStoredPoint to the current point upperDoor can only
+   * The maximum curUpperSlope between the lastStoredPoint to the current point upperDoor can only
    * open up
    */
   private double upperDoor;
   /**
-   * the minimum curLowerSlope between the lastStoredPoint to the current point lowerDoor can only
+   * The minimum curLowerSlope between the lastStoredPoint to the current point lowerDoor can only
    * open downward
    */
   private double lowerDoor;
 
   /**
-   * the last read time and value if upperDoor >= lowerDoor meaning out of compressionDeviation
+   * The last read time and value if upperDoor >= lowerDoor meaning out of compressionDeviation
    * range, will store lastReadTimestamp and lastReadValue
    */
   private long lastReadTimestamp;
@@ -45,7 +45,7 @@ public class SwingingDoorTrendingFilter<T> {
   private T lastReadValue;
 
   /**
-   * the last stored time and value we compare current point against lastReadTimestamp and
+   * The last stored time and value we compare current point against lastReadTimestamp and
    * lastReadValue
    */
   private long lastStoredTimestamp;
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/sdt/SwingingDoorTrendingSamplingProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/sdt/SwingingDoorTrendingSamplingProcessor.java
index ebfde47671e..67591ef5678 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/sdt/SwingingDoorTrendingSamplingProcessor.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/downsampling/sdt/SwingingDoorTrendingSamplingProcessor.java
@@ -43,21 +43,21 @@ public class SwingingDoorTrendingSamplingProcessor extends DownSamplingProcessor
       LoggerFactory.getLogger(SwingingDoorTrendingSamplingProcessor.class);
 
   /**
-   * the maximum absolute difference the user set if the data's value is within
+   * The maximum absolute difference the user set if the data's value is within
    * compressionDeviation, it will be compressed and discarded after compression, it will only store
    * out of range (time, data) to form the trend
    */
   private double compressionDeviation;
 
   /**
-   * the minimum time distance between two stored data points if current point time to the last
+   * The minimum time distance between two stored data points if current point time to the last
    * stored point time distance <= compressionMinTimeInterval, current point will NOT be stored
    * regardless of compression deviation
    */
   private long compressionMinTimeInterval;
 
   /**
-   * the maximum time distance between two stored data points if current point time to the last
+   * The maximum time distance between two stored data points if current point time to the last
    * stored point time distance >= compressionMaxTimeInterval, current point will be stored
    * regardless of compression deviation
    */
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/thrift/IoTDBDataNodeReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/thrift/IoTDBDataNodeReceiver.java
index fcb41ee3d0f..60e9deaf00c 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/thrift/IoTDBDataNodeReceiver.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/thrift/IoTDBDataNodeReceiver.java
@@ -298,7 +298,7 @@ public class IoTDBDataNodeReceiver extends IoTDBFileReceiver {
 
   private TSStatus executeStatementAndClassifyExceptions(Statement statement) {
     try {
-      TSStatus result = executeStatement(statement);
+      final TSStatus result = executeStatement(statement);
       if (result.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
         return result;
       } else {
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/builder/PipeDataNodeTaskBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/builder/PipeDataNodeTaskBuilder.java
index fe646192b45..e3e0d36429e 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/builder/PipeDataNodeTaskBuilder.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/builder/PipeDataNodeTaskBuilder.java
@@ -106,7 +106,8 @@ public class PipeDataNodeTaskBuilder {
             regionId,
             extractorStage.getEventSupplier(),
             connectorStage.getPipeConnectorPendingQueue(),
-            PROCESSOR_EXECUTOR);
+            PROCESSOR_EXECUTOR,
+            pipeTaskMeta);
 
     return new PipeDataNodeTask(
         pipeStaticMeta.getPipeName(), regionId, extractorStage, processorStage, connectorStage);
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/stage/PipeTaskProcessorStage.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/stage/PipeTaskProcessorStage.java
index 314ae79f42a..9fd7fcfe2fa 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/stage/PipeTaskProcessorStage.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/stage/PipeTaskProcessorStage.java
@@ -20,10 +20,13 @@
 package org.apache.iotdb.db.pipe.task.stage;
 
 import org.apache.iotdb.commons.consensus.DataRegionId;
+import org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant;
 import org.apache.iotdb.commons.pipe.config.plugin.configuraion.PipeTaskRuntimeConfiguration;
 import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskProcessorRuntimeEnvironment;
+import org.apache.iotdb.commons.pipe.plugin.builtin.BuiltinPipePlugin;
 import org.apache.iotdb.commons.pipe.task.EventSupplier;
 import org.apache.iotdb.commons.pipe.task.connection.BoundedBlockingPendingQueue;
+import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta;
 import org.apache.iotdb.commons.pipe.task.stage.PipeTaskStage;
 import org.apache.iotdb.db.pipe.agent.PipeAgent;
 import org.apache.iotdb.db.pipe.execution.executor.PipeProcessorSubtaskExecutor;
@@ -51,8 +54,8 @@ public class PipeTaskProcessorStage extends PipeTaskStage {
    * @param creationTime pipe creation time
    * @param pipeProcessorParameters used to create {@link PipeProcessor}
    * @param regionId {@link DataRegion} id
-   * @param pipeExtractorInputEventSupplier used to input events from {@link PipeExtractor}
-   * @param pipeConnectorOutputPendingQueue used to output events to {@link PipeConnector}
+   * @param pipeExtractorInputEventSupplier used to input {@link Event}s from {@link PipeExtractor}
+   * @param pipeConnectorOutputPendingQueue used to output {@link Event}s to {@link PipeConnector}
    * @throws PipeException if failed to {@link PipeProcessor#validate(PipeParameterValidator)} or
    *     {@link PipeProcessor#customize(PipeParameters, PipeProcessorRuntimeConfiguration)}}
    */
@@ -63,26 +66,30 @@ public class PipeTaskProcessorStage extends PipeTaskStage {
       int regionId,
       EventSupplier pipeExtractorInputEventSupplier,
       BoundedBlockingPendingQueue<Event> pipeConnectorOutputPendingQueue,
-      PipeProcessorSubtaskExecutor executor) {
+      PipeProcessorSubtaskExecutor executor,
+      PipeTaskMeta pipeTaskMeta) {
+    final PipeProcessorRuntimeConfiguration runtimeConfiguration =
+        new PipeTaskRuntimeConfiguration(
+            new PipeTaskProcessorRuntimeEnvironment(
+                pipeName, creationTime, regionId, pipeTaskMeta));
     final PipeProcessor pipeProcessor =
         StorageEngine.getInstance().getAllDataRegionIds().contains(new DataRegionId(regionId))
-            ? PipeAgent.plugin().dataRegion().reflectProcessor(pipeProcessorParameters)
-            : PipeAgent.plugin().schemaRegion().reflectProcessor(pipeProcessorParameters);
-
-    // Validate and customize should be called before createSubtask. this allows extractor exposing
-    // exceptions in advance.
-    try {
-      // 1. validate processor parameters
-      pipeProcessor.validate(new PipeParameterValidator(pipeProcessorParameters));
-
-      // 2. customize processor
-      final PipeProcessorRuntimeConfiguration runtimeConfiguration =
-          new PipeTaskRuntimeConfiguration(
-              new PipeTaskProcessorRuntimeEnvironment(pipeName, creationTime, regionId));
-      pipeProcessor.customize(pipeProcessorParameters, runtimeConfiguration);
-    } catch (Exception e) {
-      throw new PipeException(e.getMessage(), e);
-    }
+            ? PipeAgent.plugin()
+                .dataRegion()
+                .getConfiguredProcessor(
+                    pipeProcessorParameters.getStringOrDefault(
+                        PipeProcessorConstant.PROCESSOR_KEY,
+                        BuiltinPipePlugin.DO_NOTHING_PROCESSOR.getPipePluginName()),
+                    pipeProcessorParameters,
+                    runtimeConfiguration)
+            : PipeAgent.plugin()
+                .schemaRegion()
+                .getConfiguredProcessor(
+                    pipeProcessorParameters.getStringOrDefault(
+                        PipeProcessorConstant.PROCESSOR_KEY,
+                        BuiltinPipePlugin.DO_NOTHING_PROCESSOR.getPipePluginName()),
+                    pipeProcessorParameters,
+                    runtimeConfiguration);
 
     // Should add creation time in taskID, because subtasks are stored in the hashmap
     // PipeProcessorSubtaskWorker.subtasks, and deleted subtasks will be removed by
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtask.java
index 8d6e6c424ba..0f0bdeee689 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtask.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/subtask/connector/PipeConnectorSubtask.java
@@ -144,7 +144,7 @@ public class PipeConnectorSubtask extends PipeAbstractConnectorSubtask {
     return true;
   }
 
-  private void transferHeartbeatEvent(PipeHeartbeatEvent event) throws Exception {
+  private void transferHeartbeatEvent(PipeHeartbeatEvent event) {
     try {
       outputPipeConnector.heartbeat();
       outputPipeConnector.transfer(event);
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java
index 7056cee2caa..fe68896b168 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java
@@ -1720,7 +1720,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
   @Override
   public SettableFuture<ConfigTaskResult> alterPipe(AlterPipeStatement alterPipeStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
 
     // Validate pipe name
     if (alterPipeStatement.getPipeName().startsWith(PipeStaticMeta.SYSTEM_PIPE_PREFIX)) {
@@ -1754,14 +1754,14 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
     try (ConfigNodeClient configNodeClient =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
-      TAlterPipeReq req =
+      final TAlterPipeReq req =
           new TAlterPipeReq(
               pipeName,
               alterPipeStatement.getProcessorAttributes(),
               alterPipeStatement.getConnectorAttributes(),
               alterPipeStatement.isReplaceAllProcessorAttributes(),
               alterPipeStatement.isReplaceAllConnectorAttributes());
-      TSStatus tsStatus = configNodeClient.alterPipe(req);
+      final TSStatus tsStatus = configNodeClient.alterPipe(req);
       if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) {
         LOGGER.warn("Failed to alter pipe {} in config node, status is {}.", pipeName, tsStatus);
         future.setException(new IoTDBException(tsStatus.message, tsStatus.code));
@@ -1776,7 +1776,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
   @Override
   public SettableFuture<ConfigTaskResult> startPipe(StartPipeStatement startPipeStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
 
     // Validate pipe name
     if (startPipeStatement.getPipeName().startsWith(PipeStaticMeta.SYSTEM_PIPE_PREFIX)) {
@@ -1808,7 +1808,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
   @Override
   public SettableFuture<ConfigTaskResult> dropPipe(DropPipeStatement dropPipeStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
 
     // Validate pipe name
     if (dropPipeStatement.getPipeName().startsWith(PipeStaticMeta.SYSTEM_PIPE_PREFIX)) {
@@ -1824,7 +1824,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
     try (ConfigNodeClient configNodeClient =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
-      TSStatus tsStatus = configNodeClient.dropPipe(dropPipeStatement.getPipeName());
+      final TSStatus tsStatus = configNodeClient.dropPipe(dropPipeStatement.getPipeName());
       if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) {
         LOGGER.warn(
             "Failed to drop pipe {}, status is {}.", dropPipeStatement.getPipeName(), tsStatus);
@@ -1840,7 +1840,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
   @Override
   public SettableFuture<ConfigTaskResult> stopPipe(StopPipeStatement stopPipeStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
 
     // Validate pipe name
     if (stopPipeStatement.getPipeName().startsWith(PipeStaticMeta.SYSTEM_PIPE_PREFIX)) {
@@ -1856,7 +1856,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
     try (ConfigNodeClient configNodeClient =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
-      TSStatus tsStatus = configNodeClient.stopPipe(stopPipeStatement.getPipeName());
+      final TSStatus tsStatus = configNodeClient.stopPipe(stopPipeStatement.getPipeName());
       if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) {
         LOGGER.warn(
             "Failed to stop pipe {}, status is {}.", stopPipeStatement.getPipeName(), tsStatus);
@@ -1872,7 +1872,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
   @Override
   public SettableFuture<ConfigTaskResult> showPipes(ShowPipesStatement showPipesStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       TShowPipeReq tShowPipeReq = new TShowPipeReq();
@@ -1882,7 +1882,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
       if (showPipesStatement.getWhereClause()) {
         tShowPipeReq.setWhereClause(true);
       }
-      List<TShowPipeInfo> tShowPipeInfoList =
+      final List<TShowPipeInfo> tShowPipeInfoList =
           configNodeClient.showPipe(tShowPipeReq).getPipeInfoList();
       ShowPipeTask.buildTSBlock(tShowPipeInfoList, future);
     } catch (Exception e) {
@@ -1952,10 +1952,10 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
   @Override
   public SettableFuture<ConfigTaskResult> dropTopic(DropTopicStatement dropTopicStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
-      TSStatus tsStatus = configNodeClient.dropTopic(dropTopicStatement.getTopicName());
+      final TSStatus tsStatus = configNodeClient.dropTopic(dropTopicStatement.getTopicName());
       if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != tsStatus.getCode()) {
         LOGGER.warn(
             "Failed to drop topic {}, status is {}.", dropTopicStatement.getTopicName(), tsStatus);
@@ -2001,8 +2001,8 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   @Override
   public SettableFuture<ConfigTaskResult> deleteTimeSeries(
       String queryId, DeleteTimeSeriesStatement deleteTimeSeriesStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
-    TDeleteTimeSeriesReq req =
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final TDeleteTimeSeriesReq req =
         new TDeleteTimeSeriesReq(
             queryId,
             serializePatternListToByteBuffer(deleteTimeSeriesStatement.getPathPatternList()));
@@ -2047,8 +2047,8 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   @Override
   public SettableFuture<ConfigTaskResult> deleteLogicalView(
       String queryId, DeleteLogicalViewStatement deleteLogicalViewStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
-    TDeleteLogicalViewReq req =
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final TDeleteLogicalViewReq req =
         new TDeleteLogicalViewReq(
             queryId,
             serializePatternListToByteBuffer(deleteLogicalViewStatement.getPathPatternList()));
@@ -2088,10 +2088,10 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   @Override
   public SettableFuture<ConfigTaskResult> renameLogicalView(
       String queryId, RenameLogicalViewStatement renameLogicalViewStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
 
     // check path
-    PartialPath oldName = renameLogicalViewStatement.getOldName();
+    final PartialPath oldName = renameLogicalViewStatement.getOldName();
     if (oldName.hasWildcard()) {
       future.setException(
           new MetadataException("Rename view doesn't support path pattern with wildcard."));
@@ -2099,26 +2099,27 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     }
 
     // fetch viewExpression
-    PathPatternTree patternTree = new PathPatternTree();
+    final PathPatternTree patternTree = new PathPatternTree();
     patternTree.appendFullPath(oldName);
     patternTree.constructTree();
-    ISchemaTree schemaTree =
+    final ISchemaTree schemaTree =
         ClusterSchemaFetcher.getInstance().fetchSchema(patternTree, true, null);
-    List<MeasurementPath> measurementPathList = schemaTree.searchMeasurementPaths(oldName).left;
+    final List<MeasurementPath> measurementPathList =
+        schemaTree.searchMeasurementPaths(oldName).left;
     if (measurementPathList.isEmpty()) {
       future.setException(new PathNotExistException(oldName.getFullPath()));
       return future;
     }
-    LogicalViewSchema logicalViewSchema =
+    final LogicalViewSchema logicalViewSchema =
         (LogicalViewSchema) measurementPathList.get(0).getMeasurementSchema();
-    ViewExpression viewExpression = logicalViewSchema.getExpression();
+    final ViewExpression viewExpression = logicalViewSchema.getExpression();
 
     // create new view
-    CreateLogicalViewStatement createLogicalViewStatement = new CreateLogicalViewStatement();
+    final CreateLogicalViewStatement createLogicalViewStatement = new CreateLogicalViewStatement();
     createLogicalViewStatement.setTargetFullPaths(
         Collections.singletonList(renameLogicalViewStatement.getNewName()));
     createLogicalViewStatement.setViewExpressions(Collections.singletonList(viewExpression));
-    ExecutionResult executionResult =
+    final ExecutionResult executionResult =
         Coordinator.getInstance()
             .executeForTreeModel(
                 createLogicalViewStatement,
@@ -2136,7 +2137,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     }
 
     // delete old view
-    TDeleteLogicalViewReq req =
+    final TDeleteLogicalViewReq req =
         new TDeleteLogicalViewReq(
             queryId, serializePatternListToByteBuffer(Collections.singletonList(oldName)));
     try (ConfigNodeClient client =
@@ -2172,13 +2173,13 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   @Override
   public SettableFuture<ConfigTaskResult> alterLogicalView(
       AlterLogicalViewStatement alterLogicalViewStatement, MPPQueryContext context) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
-    CreateLogicalViewStatement createLogicalViewStatement = new CreateLogicalViewStatement();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final CreateLogicalViewStatement createLogicalViewStatement = new CreateLogicalViewStatement();
     createLogicalViewStatement.setTargetPaths(alterLogicalViewStatement.getTargetPaths());
     createLogicalViewStatement.setSourcePaths(alterLogicalViewStatement.getSourcePaths());
     createLogicalViewStatement.setQueryStatement(alterLogicalViewStatement.getQueryStatement());
 
-    Analysis analysis = Analyzer.analyze(createLogicalViewStatement, context);
+    final Analysis analysis = Analyzer.analyze(createLogicalViewStatement, context);
     if (analysis.isFailed()) {
       future.setException(
           new IoTDBException(
@@ -2187,17 +2188,17 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
     }
 
     // Transform all Expressions into ViewExpressions.
-    TransformToViewExpressionVisitor transformToViewExpressionVisitor =
+    final TransformToViewExpressionVisitor transformToViewExpressionVisitor =
         new TransformToViewExpressionVisitor();
-    List<Expression> expressionList = alterLogicalViewStatement.getSourceExpressionList();
-    List<ViewExpression> viewExpressionList = new ArrayList<>();
+    final List<Expression> expressionList = alterLogicalViewStatement.getSourceExpressionList();
+    final List<ViewExpression> viewExpressionList = new ArrayList<>();
     for (Expression expression : expressionList) {
       viewExpressionList.add(transformToViewExpressionVisitor.process(expression, null));
     }
 
-    List<PartialPath> viewPathList = alterLogicalViewStatement.getTargetPathList();
+    final List<PartialPath> viewPathList = alterLogicalViewStatement.getTargetPathList();
 
-    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    final ByteArrayOutputStream stream = new ByteArrayOutputStream();
     try {
       ReadWriteIOUtils.write(viewPathList.size(), stream);
       for (int i = 0; i < viewPathList.size(); i++) {
@@ -2208,7 +2209,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
       throw new RuntimeException(e);
     }
 
-    TAlterLogicalViewReq req =
+    final TAlterLogicalViewReq req =
         new TAlterLogicalViewReq(
             context.getQueryId().getId(), ByteBuffer.wrap(stream.toByteArray()));
     try (ConfigNodeClient client =
@@ -2252,10 +2253,10 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
   @Override
   public TSStatus alterLogicalViewByPipe(AlterLogicalViewNode alterLogicalViewNode) {
-    Map<PartialPath, ViewExpression> viewPathToSourceMap =
+    final Map<PartialPath, ViewExpression> viewPathToSourceMap =
         alterLogicalViewNode.getViewPathToSourceMap();
 
-    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    final ByteArrayOutputStream stream = new ByteArrayOutputStream();
     try {
       ReadWriteIOUtils.write(viewPathToSourceMap.size(), stream);
       for (Map.Entry<PartialPath, ViewExpression> entry : viewPathToSourceMap.entrySet()) {
@@ -2266,7 +2267,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
       throw new RuntimeException(e);
     }
 
-    TAlterLogicalViewReq req =
+    final TAlterLogicalViewReq req =
         new TAlterLogicalViewReq(
                 Coordinator.getInstance().createQueryId().getId(),
                 ByteBuffer.wrap(stream.toByteArray()))
@@ -2304,11 +2305,11 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
   @Override
   public SettableFuture<ConfigTaskResult> getRegionId(GetRegionIdStatement getRegionIdStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     TGetRegionIdResp resp = new TGetRegionIdResp();
     try (ConfigNodeClient configNodeClient =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
-      TGetRegionIdReq tGetRegionIdReq =
+      final TGetRegionIdReq tGetRegionIdReq =
           new TGetRegionIdReq(getRegionIdStatement.getPartitionType());
       if (getRegionIdStatement.getDevice() != null) {
         tGetRegionIdReq.setDevice(getRegionIdStatement.getDevice());
@@ -2334,7 +2335,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   @Override
   public SettableFuture<ConfigTaskResult> getSeriesSlotList(
       GetSeriesSlotListStatement getSeriesSlotListStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     TGetSeriesSlotListResp resp = new TGetSeriesSlotListResp();
     try (ConfigNodeClient configNodeClient =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
@@ -2357,11 +2358,11 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   @Override
   public SettableFuture<ConfigTaskResult> getTimeSlotList(
       GetTimeSlotListStatement getTimeSlotListStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     TGetTimeSlotListResp resp = new TGetTimeSlotListResp();
     try (ConfigNodeClient configNodeClient =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
-      TGetTimeSlotListReq tGetTimeSlotListReq = new TGetTimeSlotListReq();
+      final TGetTimeSlotListReq tGetTimeSlotListReq = new TGetTimeSlotListReq();
       if (getTimeSlotListStatement.getDatabase() != null) {
         tGetTimeSlotListReq.setDatabase(getTimeSlotListStatement.getDatabase());
       } else if (getTimeSlotListStatement.getDevice() != null) {
@@ -2390,11 +2391,11 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   @Override
   public SettableFuture<ConfigTaskResult> countTimeSlotList(
       CountTimeSlotListStatement countTimeSlotListStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     TCountTimeSlotListResp resp = new TCountTimeSlotListResp();
     try (ConfigNodeClient configNodeClient =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
-      TCountTimeSlotListReq tCountTimeSlotListReq = new TCountTimeSlotListReq();
+      final TCountTimeSlotListReq tCountTimeSlotListReq = new TCountTimeSlotListReq();
       if (countTimeSlotListStatement.getDatabase() != null) {
         tCountTimeSlotListReq.setDatabase(countTimeSlotListStatement.getDatabase());
       } else if (countTimeSlotListStatement.getDevice() != null) {
@@ -2423,15 +2424,15 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   @Override
   public SettableFuture<ConfigTaskResult> migrateRegion(
       MigrateRegionStatement migrateRegionStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient configNodeClient =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
-      TMigrateRegionReq tMigrateRegionReq =
+      final TMigrateRegionReq tMigrateRegionReq =
           new TMigrateRegionReq(
               migrateRegionStatement.getRegionId(),
               migrateRegionStatement.getFromId(),
               migrateRegionStatement.getToId());
-      TSStatus status = configNodeClient.migrateRegion(tMigrateRegionReq);
+      final TSStatus status = configNodeClient.migrateRegion(tMigrateRegionReq);
       if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
         future.setException(new IoTDBException(status.message, status.code));
         return future;
@@ -2449,14 +2450,14 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
       CreateContinuousQueryStatement createContinuousQueryStatement, MPPQueryContext context) {
     createContinuousQueryStatement.semanticCheck();
 
-    String queryBody = createContinuousQueryStatement.getQueryBody();
+    final String queryBody = createContinuousQueryStatement.getQueryBody();
     // TODO Do not modify Statement in Analyzer
     Analyzer.analyze(createContinuousQueryStatement.getQueryBodyStatement(), context);
 
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient client =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
-      TCreateCQReq tCreateCQReq =
+      final TCreateCQReq tCreateCQReq =
           new TCreateCQReq(
               createContinuousQueryStatement.getCqId(),
               createContinuousQueryStatement.getEveryInterval(),
@@ -2487,7 +2488,7 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
   @Override
   public SettableFuture<ConfigTaskResult> dropContinuousQuery(String cqId) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient client =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
       final TSStatus executionStatus = client.dropCQ(new TDropCQReq(cqId));
@@ -2505,10 +2506,10 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
 
   @Override
   public SettableFuture<ConfigTaskResult> showContinuousQueries() {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     try (ConfigNodeClient client =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
-      TShowCQResp showCQResp = client.showCQ();
+      final TShowCQResp showCQResp = client.showCQ();
       if (showCQResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
         future.setException(
             new IoTDBException(showCQResp.getStatus().message, showCQResp.getStatus().code));
@@ -2526,11 +2527,11 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   @Override
   public SettableFuture<ConfigTaskResult> setSpaceQuota(
       SetSpaceQuotaStatement setSpaceQuotaStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
     TSStatus tsStatus = new TSStatus();
-    TSetSpaceQuotaReq req = new TSetSpaceQuotaReq();
+    final TSetSpaceQuotaReq req = new TSetSpaceQuotaReq();
     req.setDatabase(setSpaceQuotaStatement.getPrefixPathList());
-    TSpaceQuota spaceQuota = new TSpaceQuota();
+    final TSpaceQuota spaceQuota = new TSpaceQuota();
     spaceQuota.setDeviceNum(setSpaceQuotaStatement.getDeviceNum());
     spaceQuota.setTimeserieNum(setSpaceQuotaStatement.getTimeSeriesNum());
     spaceQuota.setDiskSize(setSpaceQuotaStatement.getDiskSize());
@@ -2553,18 +2554,18 @@ public class ClusterConfigTaskExecutor implements IConfigTaskExecutor {
   @Override
   public SettableFuture<ConfigTaskResult> showSpaceQuota(
       ShowSpaceQuotaStatement showSpaceQuotaStatement) {
-    SettableFuture<ConfigTaskResult> future = SettableFuture.create();
+    final SettableFuture<ConfigTaskResult> future = SettableFuture.create();
 
     try (ConfigNodeClient configNodeClient =
         CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
-      List<String> databases = new ArrayList<>();
+      final List<String> databases = new ArrayList<>();
       if (showSpaceQuotaStatement.getDatabases() != null) {
         showSpaceQuotaStatement
             .getDatabases()
             .forEach(database -> databases.add(database.toString()));
       }
       // Send request to some API server
-      TSpaceQuotaResp showSpaceQuotaResp = configNodeClient.showSpaceQuota(databases);
+      final TSpaceQuotaResp showSpaceQuotaResp = configNodeClient.showSpaceQuota(databases);
       // build TSBlock
       ShowSpaceQuotaTask.buildTsBlock(showSpaceQuotaResp, future);
     } catch (Exception e) {
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java
index 798a6736e31..29cca4f52c9 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.queryengine.plan.planner.plan.node.write;
 
 import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
@@ -69,7 +70,7 @@ public class InsertTabletNode extends InsertNode implements WALEntryValue {
 
   private int rowCount = 0;
 
-  // when this plan is sub-plan split from another InsertTabletNode, this indicates the original
+  // When this plan is sub-plan split from another InsertTabletNode, this indicates the original
   // positions of values in
   // this plan. For example, if the plan contains 5 timestamps, and range = [1,4,10,12], then it
   // means that the first 3
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java
index bf591c95ec8..71d93647a0d 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java
@@ -1353,7 +1353,7 @@ public class TsFileProcessor {
     writer = null;
   }
 
-  /** end empty file and remove it from file system */
+  /** End empty file and remove it from file system */
   private void endEmptyFile() throws TsFileProcessorException, IOException {
     logger.info("Start to end empty file {}", tsFileResource);
 
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java
index 0c21234c9d6..f9b20bb5ef2 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java
@@ -62,8 +62,8 @@ public class TsFileManager {
   }
 
   /**
-   * @param sequence true for sequence, false for unsequence
-   * @param timePartitions null for all time partitions, empty for zero time partitions
+   * @param sequence {@code true} for sequence, {@code false} for unsequence
+   * @param timePartitions {@code null} for all time partitions, empty for zero time partitions
    */
   public List<TsFileResource> getTsFileList(
       boolean sequence, List<Long> timePartitions, Filter timeFilter) {
@@ -177,7 +177,7 @@ public class TsFileManager {
   }
 
   /**
-   * insert tsFileResource to a target pos(targetPos = insertPos) e.g. if insertPos = 0, then to the
+   * Insert tsFileResource to a target pos(targetPos = insertPos) e.g. if insertPos = 0, then to the
    * first, if insert Pos = 1, then to the second.
    */
   public void insertToPartitionFileList(
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java
index de492c22609..5b012f6876d 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java
@@ -404,7 +404,7 @@ public class TsFileResource {
     return timeIndex.getMinStartTime();
   }
 
-  /** open file's end time is Long.MIN_VALUE */
+  /** Open file's end time is Long.MIN_VALUE */
   public long getFileEndTime() {
     return timeIndex.getMaxEndTime();
   }
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALInsertNodeCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALInsertNodeCache.java
index a7cd9b8ac0d..905deb07eb4 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALInsertNodeCache.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALInsertNodeCache.java
@@ -75,7 +75,7 @@ public class WALInsertNodeCache {
     final long requestedAllocateSize =
         (long)
             Math.min(
-                2 * CONFIG.getWalFileSizeThresholdInByte(),
+                (double) 2 * CONFIG.getWalFileSizeThresholdInByte(),
                 CONFIG.getAllocateMemoryForPipe() * 0.8 / 5);
     allocatedMemoryBlock =
         PipeResourceManager.memory()
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/schema/SchemaRegionSnapshotParser.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/schema/SchemaRegionSnapshotParser.java
index a12b173b654..dba1262599a 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/schema/SchemaRegionSnapshotParser.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/schema/SchemaRegionSnapshotParser.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.tools.schema;
 
 import org.apache.iotdb.commons.file.SystemFileFactory;
@@ -45,7 +46,7 @@ public class SchemaRegionSnapshotParser {
   private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig();
 
   private SchemaRegionSnapshotParser() {
-    // empty constructor
+    // Empty constructor
   }
 
   private static Path getLatestSnapshotPath(List<Path> snapshotPathList) {
@@ -63,19 +64,19 @@ public class SchemaRegionSnapshotParser {
     return pathArray[0];
   }
 
-  // return all schema region's latest snapshot units in this datanode.
+  // Return all schema region's latest snapshot units in this datanode.
   public static List<Pair<Path, Path>> getSnapshotPaths() {
     String snapshotPath = CONFIG.getSchemaRegionConsensusDir();
     File snapshotDir = new File(snapshotPath);
     ArrayList<Pair<Path, Path>> snapshotUnits = new ArrayList<>();
 
-    // get schema regin path
+    // Get schema regin path
     try (DirectoryStream<Path> stream =
         Files.newDirectoryStream(snapshotDir.toPath(), "[0-9]*-[0-9]*-[0-9]*-[0-9]*-[0-9]*")) {
       for (Path path : stream) {
         try (DirectoryStream<Path> filestream =
             Files.newDirectoryStream(Paths.get(path.toString() + File.separator + "sm"))) {
-          // find the latest snapshots
+          // Find the latest snapshots
           ArrayList<Path> snapshotList = new ArrayList<>();
           for (Path snapshotFolder : filestream) {
             if (snapshotFolder.toFile().isDirectory()) {
@@ -105,7 +106,7 @@ public class SchemaRegionSnapshotParser {
     return snapshotUnits;
   }
 
-  // in schema snapshot path: datanode/consensus/schema_region/47474747-4747-4747-4747-000200000000
+  // In schema snapshot path: datanode/consensus/schema_region/47474747-4747-4747-4747-000200000000
   // this func will get schema region id = 47474747-4747-4747-4747-000200000000's latest snapshot.
   // In one schema region, there is only one snapshot unit.
   public static Pair<Path, Path> getSnapshotPaths(String schemaRegionId) {
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/MemUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/MemUtils.java
index a033539cdcb..ee7e0f4cc91 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/MemUtils.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/utils/MemUtils.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.utils;
 
 import org.apache.iotdb.commons.conf.IoTDBConstant;
@@ -116,7 +117,7 @@ public class MemUtils {
       if (insertTabletNode.getMeasurements()[i] == null) {
         continue;
       }
-      // time column memSize
+      // Time column memSize
       memSize += (end - start) * 8L;
       memSize += (long) (end - start) * insertTabletNode.getDataTypes()[i].getDataTypeSize();
     }
@@ -151,13 +152,13 @@ public class MemUtils {
     return memUsed;
   }
 
-  /** function for getting the memory size of the given string. */
+  /** Function for getting the memory size of the given string. */
   public static long getStringMem(String str) {
     // wide char (2 bytes each) and 64B String overhead
     return str.length() * 2L + 64L;
   }
 
-  /** function for getting the memory size of the given data point. */
+  /** Function for getting the memory size of the given data point. */
   public static long getDataPointMem(DataPoint dataPoint) {
     // type reference
     long memUsed = 8;
@@ -189,7 +190,7 @@ public class MemUtils {
     return memUsed;
   }
 
-  /** function for converting the byte count result to readable string. */
+  /** Function for converting the byte count result to readable string. */
   public static String bytesCntToStr(long inputCnt) {
     long cnt = inputCnt;
     long gbs = cnt / IoTDBConstant.GB;
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/FragmentInstanceSerdeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/FragmentInstanceSerdeTest.java
index cc3363a097a..711e67682ca 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/FragmentInstanceSerdeTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/FragmentInstanceSerdeTest.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.db.queryengine.plan.planner;
 
 import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java
index 84ca7baa85a..3439b2d0af9 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java
@@ -25,6 +25,7 @@ import org.apache.iotdb.commons.consensus.index.impl.MetaProgressIndex;
 import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex;
 import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex;
 import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex;
+import org.apache.iotdb.commons.consensus.index.impl.TimeWindowStateProgressIndex;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import java.io.IOException;
@@ -39,6 +40,7 @@ public enum ProgressIndexType {
   RECOVER_PROGRESS_INDEX((short) 4),
   HYBRID_PROGRESS_INDEX((short) 5),
   META_PROGRESS_INDEX((short) 6),
+  TIME_WINDOW_STATE_PROGRESS_INDEX((short) 7),
   ;
 
   private final short type;
@@ -74,6 +76,8 @@ public enum ProgressIndexType {
         return HybridProgressIndex.deserializeFrom(byteBuffer);
       case 6:
         return MetaProgressIndex.deserializeFrom(byteBuffer);
+      case 7:
+        return TimeWindowStateProgressIndex.deserializeFrom(byteBuffer);
       default:
         throw new UnsupportedOperationException(
             String.format("Unsupported progress index type %s.", indexType));
@@ -93,6 +97,10 @@ public enum ProgressIndexType {
         return RecoverProgressIndex.deserializeFrom(stream);
       case 5:
         return HybridProgressIndex.deserializeFrom(stream);
+      case 6:
+        return MetaProgressIndex.deserializeFrom(stream);
+      case 7:
+        return TimeWindowStateProgressIndex.deserializeFrom(stream);
       default:
         throw new UnsupportedOperationException(
             String.format("Unsupported progress index type %s.", indexType));
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/TimeWindowStateProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/TimeWindowStateProgressIndex.java
new file mode 100644
index 00000000000..31db0c64480
--- /dev/null
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/TimeWindowStateProgressIndex.java
@@ -0,0 +1,280 @@
+/*
+ * 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.consensus.index.impl;
+
+import org.apache.iotdb.commons.consensus.index.ProgressIndex;
+import org.apache.iotdb.commons.consensus.index.ProgressIndexType;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+public class TimeWindowStateProgressIndex extends ProgressIndex {
+
+  private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+  // Only the byteBuffer is nullable, the timeSeries, pair and timestamp must not be null
+  private Map<String, Pair<Long, ByteBuffer>> timeSeries2TimestampWindowBufferPairMap;
+
+  public TimeWindowStateProgressIndex(
+      @Nonnull Map<String, Pair<Long, ByteBuffer>> timeSeries2TimestampWindowBufferPairMap) {
+    this.timeSeries2TimestampWindowBufferPairMap = timeSeries2TimestampWindowBufferPairMap;
+  }
+
+  private TimeWindowStateProgressIndex() {
+    // Empty constructor
+  }
+
+  public Map<String, Pair<Long, ByteBuffer>> getTimeSeries2TimestampWindowBufferPairMap() {
+    return timeSeries2TimestampWindowBufferPairMap;
+  }
+
+  public long getMinTime() {
+    return timeSeries2TimestampWindowBufferPairMap.values().stream()
+        .mapToLong(Pair::getLeft)
+        .min()
+        .orElse(Long.MIN_VALUE);
+  }
+
+  @Override
+  public void serialize(ByteBuffer byteBuffer) {
+    lock.readLock().lock();
+    try {
+      ProgressIndexType.TIME_WINDOW_STATE_PROGRESS_INDEX.serialize(byteBuffer);
+
+      ReadWriteIOUtils.write(timeSeries2TimestampWindowBufferPairMap.size(), byteBuffer);
+      for (final Map.Entry<String, Pair<Long, ByteBuffer>> entry :
+          timeSeries2TimestampWindowBufferPairMap.entrySet()) {
+        ReadWriteIOUtils.write(entry.getKey(), byteBuffer);
+        ReadWriteIOUtils.write(entry.getValue().getLeft(), byteBuffer);
+        final ByteBuffer buffer = entry.getValue().getRight();
+        if (Objects.nonNull(buffer)) {
+          ReadWriteIOUtils.write(buffer.limit(), byteBuffer);
+          byteBuffer.put(buffer.array(), 0, buffer.limit());
+        } else {
+          ReadWriteIOUtils.write(-1, byteBuffer);
+        }
+      }
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  @Override
+  public void serialize(OutputStream stream) throws IOException {
+    lock.readLock().lock();
+    try {
+      ProgressIndexType.TIME_WINDOW_STATE_PROGRESS_INDEX.serialize(stream);
+
+      ReadWriteIOUtils.write(timeSeries2TimestampWindowBufferPairMap.size(), stream);
+      for (final Map.Entry<String, Pair<Long, ByteBuffer>> entry :
+          timeSeries2TimestampWindowBufferPairMap.entrySet()) {
+        ReadWriteIOUtils.write(entry.getKey(), stream);
+        ReadWriteIOUtils.write(entry.getValue().getLeft(), stream);
+        final ByteBuffer buffer = entry.getValue().getRight();
+        if (Objects.nonNull(buffer)) {
+          ReadWriteIOUtils.write(buffer.limit(), stream);
+          stream.write(buffer.array(), 0, buffer.limit());
+        } else {
+          ReadWriteIOUtils.write(-1, stream);
+        }
+      }
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  @Override
+  public boolean isAfter(@Nonnull ProgressIndex progressIndex) {
+    lock.readLock().lock();
+    try {
+      if (progressIndex instanceof MinimumProgressIndex) {
+        return true;
+      }
+
+      if (progressIndex instanceof HybridProgressIndex) {
+        return ((HybridProgressIndex) progressIndex).isGivenProgressIndexAfterSelf(this);
+      }
+
+      if (!(progressIndex instanceof TimeWindowStateProgressIndex)) {
+        return false;
+      }
+
+      final TimeWindowStateProgressIndex thisTimeWindowStateProgressIndex = this;
+      final TimeWindowStateProgressIndex thatTimeWindowStateProgressIndex =
+          (TimeWindowStateProgressIndex) progressIndex;
+      return thatTimeWindowStateProgressIndex.timeSeries2TimestampWindowBufferPairMap.entrySet()
+          .stream()
+          .noneMatch(
+              entry ->
+                  !thisTimeWindowStateProgressIndex.timeSeries2TimestampWindowBufferPairMap
+                          .containsKey(entry.getKey())
+                      || thisTimeWindowStateProgressIndex
+                              .timeSeries2TimestampWindowBufferPairMap
+                              .get(entry.getKey())
+                              .getLeft()
+                          <= entry.getValue().getLeft());
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  @Override
+  public boolean equals(ProgressIndex progressIndex) {
+    lock.readLock().lock();
+    try {
+      if (!(progressIndex instanceof TimeWindowStateProgressIndex)) {
+        return false;
+      }
+
+      final TimeWindowStateProgressIndex thisTimeWindowStateProgressIndex = this;
+      final TimeWindowStateProgressIndex thatTimeWindowStateProgressIndex =
+          (TimeWindowStateProgressIndex) progressIndex;
+      return thisTimeWindowStateProgressIndex.timeSeries2TimestampWindowBufferPairMap.equals(
+          thatTimeWindowStateProgressIndex.timeSeries2TimestampWindowBufferPairMap);
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
+    }
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof TimeWindowStateProgressIndex)) {
+      return false;
+    }
+    return this.equals((TimeWindowStateProgressIndex) obj);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(timeSeries2TimestampWindowBufferPairMap);
+  }
+
+  @Override
+  public ProgressIndex updateToMinimumEqualOrIsAfterProgressIndex(ProgressIndex progressIndex) {
+    lock.writeLock().lock();
+    try {
+      if (!(progressIndex instanceof TimeWindowStateProgressIndex)) {
+        return this;
+      }
+
+      this.timeSeries2TimestampWindowBufferPairMap.putAll(
+          ((TimeWindowStateProgressIndex) progressIndex)
+              .timeSeries2TimestampWindowBufferPairMap.entrySet().stream()
+                  .filter(
+                      entry ->
+                          !this.timeSeries2TimestampWindowBufferPairMap.containsKey(entry.getKey())
+                              || this.timeSeries2TimestampWindowBufferPairMap
+                                      .get(entry.getKey())
+                                      .getLeft()
+                                  <= entry.getValue().getLeft())
+                  .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
+      return this;
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  @Override
+  public ProgressIndexType getType() {
+    return ProgressIndexType.TIME_WINDOW_STATE_PROGRESS_INDEX;
+  }
+
+  @Override
+  public TotalOrderSumTuple getTotalOrderSumTuple() {
+    throw new UnsupportedOperationException(
+        "TimeWindowStateProgressIndex does not support topological sorting");
+  }
+
+  public static TimeWindowStateProgressIndex deserializeFrom(ByteBuffer byteBuffer) {
+    final TimeWindowStateProgressIndex timeWindowStateProgressIndex =
+        new TimeWindowStateProgressIndex();
+    timeWindowStateProgressIndex.timeSeries2TimestampWindowBufferPairMap = new HashMap<>();
+
+    final int size = ReadWriteIOUtils.readInt(byteBuffer);
+    for (int i = 0; i < size; ++i) {
+      final String timeSeries = ReadWriteIOUtils.readString(byteBuffer);
+      final long timestamp = ReadWriteIOUtils.readLong(byteBuffer);
+      final int length = ReadWriteIOUtils.readInt(byteBuffer);
+      if (length < 0) {
+        continue;
+      }
+      final byte[] body = new byte[length];
+      byteBuffer.get(body);
+      final ByteBuffer dstBuffer = ByteBuffer.wrap(body);
+      timeWindowStateProgressIndex.timeSeries2TimestampWindowBufferPairMap.put(
+          timeSeries, new Pair<>(timestamp, dstBuffer));
+    }
+    return timeWindowStateProgressIndex;
+  }
+
+  public static TimeWindowStateProgressIndex deserializeFrom(InputStream stream)
+      throws IOException {
+    final TimeWindowStateProgressIndex timeWindowStateProgressIndex =
+        new TimeWindowStateProgressIndex();
+    timeWindowStateProgressIndex.timeSeries2TimestampWindowBufferPairMap = new HashMap<>();
+
+    final int size = ReadWriteIOUtils.readInt(stream);
+    for (int i = 0; i < size; ++i) {
+      final String timeSeries = ReadWriteIOUtils.readString(stream);
+      final long timestamp = ReadWriteIOUtils.readLong(stream);
+      final int length = ReadWriteIOUtils.readInt(stream);
+      if (length < 0) {
+        continue;
+      }
+      final byte[] body = new byte[length];
+      final int readLen = stream.read(body);
+      if (readLen != length) {
+        throw new IOException(
+            String.format(
+                "The intended read length is %s but %s is actually read when deserializing TimeProgressIndex, ProgressIndex: %s",
+                length, readLen, timeWindowStateProgressIndex));
+      }
+      final ByteBuffer dstBuffer = ByteBuffer.wrap(body);
+      timeWindowStateProgressIndex.timeSeries2TimestampWindowBufferPairMap.put(
+          timeSeries, new Pair<>(timestamp, dstBuffer));
+    }
+    return timeWindowStateProgressIndex;
+  }
+
+  @Override
+  public String toString() {
+    return "TimeWindowStateProgressIndex{"
+        + "timeSeries2TimeWindowBufferPairMap='"
+        + timeSeries2TimestampWindowBufferPairMap
+        + "'}";
+  }
+}
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/PipePluginAgent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/PipePluginAgent.java
index 6e51eca43ae..3e1b123ec9b 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/PipePluginAgent.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/PipePluginAgent.java
@@ -19,53 +19,63 @@
 
 package org.apache.iotdb.commons.pipe.agent.plugin;
 
+import org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant;
 import org.apache.iotdb.commons.pipe.config.plugin.configuraion.PipeTaskRuntimeConfiguration;
 import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskTemporaryRuntimeEnvironment;
 import org.apache.iotdb.commons.pipe.plugin.meta.PipePluginMetaKeeper;
 import org.apache.iotdb.pipe.api.PipeConnector;
 import org.apache.iotdb.pipe.api.PipeExtractor;
 import org.apache.iotdb.pipe.api.PipeProcessor;
+import org.apache.iotdb.pipe.api.customizer.configuration.PipeProcessorRuntimeConfiguration;
 import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator;
 import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters;
+import org.apache.iotdb.pipe.api.exception.PipeException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
 
 public abstract class PipePluginAgent {
 
   private static final Logger LOGGER = LoggerFactory.getLogger(PipePluginAgent.class);
 
-  private final PipePluginConstructor pipeExtractorConstructor;
-  private final PipePluginConstructor pipeProcessorConstructor;
-  private final PipePluginConstructor pipeConnectorConstructor;
+  protected final PipePluginMetaKeeper pipePluginMetaKeeper;
+  private final PipeExtractorConstructor pipeExtractorConstructor;
+  private final PipeProcessorConstructor pipeProcessorConstructor;
+  private final PipeConnectorConstructor pipeConnectorConstructor;
 
   protected PipePluginAgent(PipePluginMetaKeeper pipePluginMetaKeeper) {
+    this.pipePluginMetaKeeper = pipePluginMetaKeeper;
     pipeExtractorConstructor = createPipeExtractorConstructor(pipePluginMetaKeeper);
     pipeProcessorConstructor = createPipeProcessorConstructor(pipePluginMetaKeeper);
     pipeConnectorConstructor = createPipeConnectorConstructor(pipePluginMetaKeeper);
   }
 
-  protected abstract PipePluginConstructor createPipeExtractorConstructor(
+  protected abstract PipeExtractorConstructor createPipeExtractorConstructor(
       PipePluginMetaKeeper pipePluginMetaKeeper);
 
-  protected abstract PipePluginConstructor createPipeProcessorConstructor(
+  protected abstract PipeProcessorConstructor createPipeProcessorConstructor(
       PipePluginMetaKeeper pipePluginMetaKeeper);
 
-  protected abstract PipePluginConstructor createPipeConnectorConstructor(
+  protected abstract PipeConnectorConstructor createPipeConnectorConstructor(
       PipePluginMetaKeeper pipePluginMetaKeeper);
 
   public final PipeExtractor reflectExtractor(PipeParameters extractorParameters) {
-    return (PipeExtractor) pipeExtractorConstructor.reflectPlugin(extractorParameters);
+    return pipeExtractorConstructor.reflectPlugin(extractorParameters);
   }
 
   public final PipeProcessor reflectProcessor(PipeParameters processorParameters) {
-    return (PipeProcessor) pipeProcessorConstructor.reflectPlugin(processorParameters);
+    return pipeProcessorConstructor.reflectPlugin(processorParameters);
   }
 
   public final PipeConnector reflectConnector(PipeParameters connectorParameters) {
-    return (PipeConnector) pipeConnectorConstructor.reflectPlugin(connectorParameters);
+    return pipeConnectorConstructor.reflectPlugin(connectorParameters);
   }
 
   public void validate(
@@ -125,4 +135,64 @@ public abstract class PipePluginAgent {
       }
     }
   }
+
+  /**
+   * Get the registered subClasses names of the given parent {@link PipeProcessor}. This method is
+   * usually used to dynamically pick one or more {@link PipeProcessor} as the "plugin" of the
+   * parent class.
+   *
+   * @param parentClass the parent {@link PipeProcessor} to be checked
+   * @return All the pluginNames of the plugin's subClass
+   * @throws PipeException if any exception occurs
+   */
+  public final List<String> getSubProcessorNamesWithSpecifiedParent(
+      Class<? extends PipeProcessor> parentClass) throws PipeException {
+    return Arrays.stream(pipePluginMetaKeeper.getAllPipePluginMeta())
+        .map(pipePluginMeta -> pipePluginMeta.getPluginName().toLowerCase())
+        .filter(
+            pluginName -> {
+              try (PipeProcessor processor =
+                  (PipeProcessor) pipeProcessorConstructor.reflectPluginByKey(pluginName)) {
+                return processor.getClass().getSuperclass() == parentClass;
+              } catch (Exception e) {
+                return false;
+              }
+            })
+        .collect(Collectors.toList());
+  }
+
+  /**
+   * Use the pipeProcessorName, {@link PipeParameters} and {@link PipeProcessorRuntimeConfiguration}
+   * to construct a fully prepared {@link PipeProcessor}. Note that the {@link PipeParameters} with
+   * the processor's name will be used to validate and customize the {@link PipeProcessor}
+   * regardless of its original processor name k-v pair, yet the original {@link PipeParameters} is
+   * left unchanged.
+   *
+   * @param pipeProcessorName the processor's pluginName
+   * @param processorParameters the parameters
+   * @param runtimeConfigurations the runtimeConfigurations
+   * @return the customized {@link PipeProcessor}
+   */
+  public final PipeProcessor getConfiguredProcessor(
+      String pipeProcessorName,
+      PipeParameters processorParameters,
+      PipeProcessorRuntimeConfiguration runtimeConfigurations) {
+    final HashMap<String, String> processorKeyMap = new HashMap<>();
+    if (Objects.nonNull(pipeProcessorName)) {
+      processorKeyMap.put(PipeProcessorConstant.PROCESSOR_KEY, pipeProcessorName);
+    }
+    final PipeParameters replacedParameters =
+        processorParameters.addOrReplaceEquivalentAttributesWithClone(
+            new PipeParameters(processorKeyMap));
+    final PipeProcessor processor = reflectProcessor(replacedParameters);
+    // Validate and customize should be called to expose exceptions in advance
+    // and configure the processor.
+    try {
+      processor.validate(new PipeParameterValidator(replacedParameters));
+      processor.customize(replacedParameters, runtimeConfigurations);
+    } catch (Exception e) {
+      throw new PipeException(e.getMessage(), e);
+    }
+    return processor;
+  }
 }
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/PipePluginConstructor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/PipePluginConstructor.java
index 7ed9a6e3132..80cde27b655 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/PipePluginConstructor.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/plugin/PipePluginConstructor.java
@@ -55,7 +55,7 @@ public abstract class PipePluginConstructor {
 
   public abstract PipePlugin reflectPlugin(PipeParameters pipeParameters);
 
-  protected PipePlugin reflectPluginByKey(String pluginKey) {
+  public PipePlugin reflectPluginByKey(String pluginKey) {
     return pluginConstructors.getOrDefault(pluginKey, () -> reflect(pluginKey)).get();
   }
 
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeProcessorConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeProcessorConstant.java
index 4811f342b0e..719fa3c37e5 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeProcessorConstant.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeProcessorConstant.java
@@ -32,6 +32,26 @@ public class PipeProcessorConstant {
       "processor.down-sampling.memory-limit-in-bytes";
   public static final long PROCESSOR_DOWN_SAMPLING_MEMORY_LIMIT_IN_BYTES_DEFAULT_VALUE = 16 * MB;
 
+  public static final String PROCESSOR_OUTPUT_MAX_DELAY_SECONDS_KEY =
+      "processor.output.max-delay-seconds";
+  public static final long PROCESSOR_OUTPUT_MAX_DELAY_SECONDS_DEFAULT_VALUE = -1;
+  public static final String PROCESSOR_OUTPUT_DATABASE_KEY = "processor.output.database";
+  public static final String PROCESSOR_OUTPUT_DATABASE_DEFAULT_VALUE = "";
+  public static final String PROCESSOR_OUTPUT_MEASUREMENTS_KEY = "processor.output.measurements";
+  public static final String PROCESSOR_OUTPUT_MEASUREMENTS_DEFAULT_VALUE = "";
+  public static final String PROCESSOR_OUTPUT_MIN_REPORT_INTERVAL_SECONDS_KEY =
+      "processor.output.min-report-interval-seconds";
+  public static final long PROCESSOR_OUTPUT_MIN_REPORT_INTERVAL_SECONDS_DEFAULT_VALUE = 30;
+  public static final String PROCESSOR_OPERATORS_KEY = "processor.operators";
+  public static final String PROCESSOR_OPERATORS_DEFAULT_VALUE = "";
+  public static final String PROCESSOR_WINDOWING_STRATEGY_KEY = "processor.windowing-strategy";
+  public static final String PROCESSOR_WINDOWING_STRATEGY_DEFAULT_VALUE = "tumbling";
+  public static final String PROCESSOR_SLIDING_BOUNDARY_TIME_KEY =
+      "processor.sliding.boundary-time";
+  public static final long PROCESSOR_SLIDING_BOUNDARY_TIME_DEFAULT_VALUE = 0;
+  public static final String PROCESSOR_SLIDING_SECONDS_KEY = "processor.sliding.seconds";
+  public static final long PROCESSOR_SLIDING_SECONDS_DEFAULT_VALUE = 120;
+
   public static final String PROCESSOR_TUMBLING_TIME_INTERVAL_SECONDS_KEY =
       "processor.tumbling-time.interval-seconds";
   public static final long PROCESSOR_TUMBLING_TIME_INTERVAL_SECONDS_DEFAULT_VALUE = 60;
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java
index 47314d30a0b..ba2a89db401 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java
@@ -19,9 +19,19 @@
 
 package org.apache.iotdb.commons.pipe.config.plugin.env;
 
+import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta;
+
 public class PipeTaskProcessorRuntimeEnvironment extends PipeTaskRuntimeEnvironment {
 
-  public PipeTaskProcessorRuntimeEnvironment(String pipeName, long creationTime, int regionId) {
+  private final PipeTaskMeta pipeTaskMeta;
+
+  public PipeTaskProcessorRuntimeEnvironment(
+      String pipeName, long creationTime, int regionId, PipeTaskMeta pipeTaskMeta) {
     super(pipeName, creationTime, regionId);
+    this.pipeTaskMeta = pipeTaskMeta;
+  }
+
+  public PipeTaskMeta getPipeTaskMeta() {
+    return pipeTaskMeta;
   }
 }
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java
index 8bbefd80798..e406b314e7a 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java
@@ -52,7 +52,7 @@ public class IoTDBSyncClient extends IClientRPCService.Client
                         trustStorePwd)
                     : DeepCopyRpcTransportFactory.INSTANCE.getTransport(
                         ipAddress, port, property.getConnectionTimeoutMs())));
-    TTransport transport = getInputProtocol().getTransport();
+    final TTransport transport = getInputProtocol().getTransport();
     if (!transport.isOpen()) {
       transport.open();
     }
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java
index fb9c1444b70..5c6cc6c262d 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java
@@ -57,7 +57,7 @@ public abstract class EnrichedEvent implements Event {
   protected boolean isPatternParsed;
   protected boolean isTimeParsed;
 
-  protected boolean shouldReportOnCommit = false;
+  protected boolean shouldReportOnCommit = true;
 
   protected EnrichedEvent(
       String pipeName,
@@ -121,8 +121,8 @@ public abstract class EnrichedEvent implements Event {
     synchronized (this) {
       if (referenceCount.get() == 1) {
         isSuccessful = internallyDecreaseResourceReferenceCount(holderMessage);
-        if (shouldReport) {
-          shouldReportOnCommit = true;
+        if (!shouldReport) {
+          shouldReportOnCommit = false;
         }
         PipeEventCommitManager.getInstance().commit(this, committerKey);
       }
@@ -173,6 +173,14 @@ public abstract class EnrichedEvent implements Event {
     }
   }
 
+  /**
+   * Externally skip the report of the processing {@link ProgressIndex} of this {@link
+   * EnrichedEvent}.
+   */
+  public void skipReport() {
+    shouldReportOnCommit = false;
+  }
+
   public void bindProgressIndex(ProgressIndex progressIndex) {
     throw new UnsupportedOperationException("This event does not support binding progressIndex.");
   }
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/BuiltinPipePlugin.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/BuiltinPipePlugin.java
index d4b57d515e6..a7fc2da5946 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/BuiltinPipePlugin.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/BuiltinPipePlugin.java
@@ -31,6 +31,9 @@ import org.apache.iotdb.commons.pipe.plugin.builtin.connector.websocket.WebSocke
 import org.apache.iotdb.commons.pipe.plugin.builtin.connector.writeback.WriteBackConnector;
 import org.apache.iotdb.commons.pipe.plugin.builtin.extractor.donothing.DoNothingExtractor;
 import org.apache.iotdb.commons.pipe.plugin.builtin.extractor.iotdb.IoTDBExtractor;
+import org.apache.iotdb.commons.pipe.plugin.builtin.processor.aggregate.AggregateProcessor;
+import org.apache.iotdb.commons.pipe.plugin.builtin.processor.aggregate.StandardStatisticsProcessor;
+import org.apache.iotdb.commons.pipe.plugin.builtin.processor.aggregate.TumblingWindowingProcessor;
 import org.apache.iotdb.commons.pipe.plugin.builtin.processor.donothing.DoNothingProcessor;
 import org.apache.iotdb.commons.pipe.plugin.builtin.processor.downsampling.SwingingDoorTrendingSamplingProcessor;
 import org.apache.iotdb.commons.pipe.plugin.builtin.processor.downsampling.TumblingTimeSamplingProcessor;
@@ -56,6 +59,11 @@ public enum BuiltinPipePlugin {
       "tumbling-time-sampling-processor", TumblingTimeSamplingProcessor.class),
   SDT_SAMPLING_PROCESSOR("sdt-sampling-processor", SwingingDoorTrendingSamplingProcessor.class),
   THROWING_EXCEPTION_PROCESSOR("throwing-exception-processor", ThrowingExceptionProcessor.class),
+  AGGREGATE_PROCESSOR("aggregate-processor", AggregateProcessor.class),
+
+  // Hidden-processors, which are plugins of the processors
+  STANDARD_STATISTICS_PROCESSOR("standard-statistics-processor", StandardStatisticsProcessor.class),
+  TUMBLING_WINDOWING_PROCESSOR("tumbling-windowing-processor", TumblingWindowingProcessor.class),
 
   // connectors
   DO_NOTHING_CONNECTOR("do-nothing-connector", DoNothingConnector.class),
@@ -117,6 +125,9 @@ public enum BuiltinPipePlugin {
                   TUMBLING_TIME_SAMPLING_PROCESSOR.getPipePluginName().toUpperCase(),
                   SDT_SAMPLING_PROCESSOR.getPipePluginName().toUpperCase(),
                   THROWING_EXCEPTION_PROCESSOR.getPipePluginName().toUpperCase(),
+                  AGGREGATE_PROCESSOR.getPipePluginName().toUpperCase(),
+                  STANDARD_STATISTICS_PROCESSOR.getPipePluginName().toUpperCase(),
+                  TUMBLING_WINDOWING_PROCESSOR.getPipePluginName().toUpperCase(),
                   // Connectors
                   DO_NOTHING_CONNECTOR.getPipePluginName().toUpperCase(),
                   IOTDB_THRIFT_CONNECTOR.getPipePluginName().toUpperCase(),
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/aggregate/AggregateProcessor.java
similarity index 59%
copy from iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java
copy to iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/aggregate/AggregateProcessor.java
index 47314d30a0b..4b0b41dea9a 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/aggregate/AggregateProcessor.java
@@ -17,11 +17,14 @@
  * under the License.
  */
 
-package org.apache.iotdb.commons.pipe.config.plugin.env;
+package org.apache.iotdb.commons.pipe.plugin.builtin.processor.aggregate;
 
-public class PipeTaskProcessorRuntimeEnvironment extends PipeTaskRuntimeEnvironment {
+import org.apache.iotdb.commons.pipe.plugin.builtin.processor.PlaceHolderProcessor;
 
-  public PipeTaskProcessorRuntimeEnvironment(String pipeName, long creationTime, int regionId) {
-    super(pipeName, creationTime, regionId);
-  }
-}
+/**
+ * This class is a placeholder and should not be initialized. It represents the Aggregate processor.
+ * There is a real implementation in the server module but cannot be imported here. The pipe agent
+ * in the server module will replace this class with the real implementation when initializing the
+ * Down Sampling processor.
+ */
+public class AggregateProcessor extends PlaceHolderProcessor {}
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/aggregate/StandardStatisticsProcessor.java
similarity index 58%
copy from iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java
copy to iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/aggregate/StandardStatisticsProcessor.java
index 47314d30a0b..c1ed6d555b9 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/aggregate/StandardStatisticsProcessor.java
@@ -17,11 +17,14 @@
  * under the License.
  */
 
-package org.apache.iotdb.commons.pipe.config.plugin.env;
+package org.apache.iotdb.commons.pipe.plugin.builtin.processor.aggregate;
 
-public class PipeTaskProcessorRuntimeEnvironment extends PipeTaskRuntimeEnvironment {
+import org.apache.iotdb.commons.pipe.plugin.builtin.processor.PlaceHolderProcessor;
 
-  public PipeTaskProcessorRuntimeEnvironment(String pipeName, long creationTime, int regionId) {
-    super(pipeName, creationTime, regionId);
-  }
-}
+/**
+ * This class is a placeholder and should not be initialized. It represents the Standard Statistics
+ * processor. There is a real implementation in the server module but cannot be imported here. The
+ * pipe agent in the server module will replace this class with the real implementation when
+ * initializing the Standard Statistics processor.
+ */
+public class StandardStatisticsProcessor extends PlaceHolderProcessor {}
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/aggregate/TumblingWindowingProcessor.java
similarity index 58%
copy from iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java
copy to iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/aggregate/TumblingWindowingProcessor.java
index 47314d30a0b..9733ab189d5 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/plugin/env/PipeTaskProcessorRuntimeEnvironment.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/plugin/builtin/processor/aggregate/TumblingWindowingProcessor.java
@@ -17,11 +17,14 @@
  * under the License.
  */
 
-package org.apache.iotdb.commons.pipe.config.plugin.env;
+package org.apache.iotdb.commons.pipe.plugin.builtin.processor.aggregate;
 
-public class PipeTaskProcessorRuntimeEnvironment extends PipeTaskRuntimeEnvironment {
+import org.apache.iotdb.commons.pipe.plugin.builtin.processor.PlaceHolderProcessor;
 
-  public PipeTaskProcessorRuntimeEnvironment(String pipeName, long creationTime, int regionId) {
-    super(pipeName, creationTime, regionId);
-  }
-}
+/**
+ * This class is a placeholder and should not be initialized. It represents the Standard Statistics
+ * processor. There is a real implementation in the server module but cannot be imported here. The
+ * pipe agent in the server module will replace this class with the real implementation when
+ * initializing the Standard Statistics processor.
+ */
+public class TumblingWindowingProcessor extends PlaceHolderProcessor {}
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java
index bafae21d2e4..d21ceebd51c 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java
@@ -109,7 +109,7 @@ public abstract class IoTDBFileReceiver implements IoTDBReceiver {
       LOGGER.info("Current receiver file dir is null. No need to delete.");
     }
 
-    String receiverFileBaseDir;
+    final String receiverFileBaseDir;
     try {
       receiverFileBaseDir = getReceiverFileBaseDir();
       if (Objects.isNull(receiverFileBaseDir)) {
diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/task/meta/PipeMetaDeSerTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/task/meta/PipeMetaDeSerTest.java
index b48c2c4a16b..50a460848de 100644
--- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/task/meta/PipeMetaDeSerTest.java
+++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/task/meta/PipeMetaDeSerTest.java
@@ -25,15 +25,21 @@ import org.apache.iotdb.commons.consensus.index.impl.MetaProgressIndex;
 import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex;
 import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex;
 import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex;
+import org.apache.iotdb.commons.consensus.index.impl.TimeWindowStateProgressIndex;
 import org.apache.iotdb.commons.exception.pipe.PipeRuntimeConnectorCriticalException;
 import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
 public class PipeMetaDeSerTest {
@@ -65,6 +71,15 @@ public class PipeMetaDeSerTest {
     hybridProgressIndex.updateToMinimumEqualOrIsAfterProgressIndex(new SimpleProgressIndex(2, 4));
     hybridProgressIndex.updateToMinimumEqualOrIsAfterProgressIndex(new IoTProgressIndex(3, 6L));
 
+    Map<String, Pair<Long, ByteBuffer>> timeSeries2TimestampWindowBufferPairMap = new HashMap<>();
+    ByteBuffer buffer;
+    try (final PublicBAOS byteArrayOutputStream = new PublicBAOS();
+        final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) {
+      ReadWriteIOUtils.write("123", outputStream);
+      buffer = ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size());
+    }
+    timeSeries2TimestampWindowBufferPairMap.put("root.test.a1", new Pair<>(123L, buffer));
+
     PipeRuntimeMeta pipeRuntimeMeta =
         new PipeRuntimeMeta(
             new ConcurrentHashMap<Integer, PipeTaskMeta>() {
@@ -77,6 +92,11 @@ public class PipeMetaDeSerTest {
                     567,
                     new PipeTaskMeta(
                         new RecoverProgressIndex(1, new SimpleProgressIndex(1, 9)), 123));
+                put(
+                    678,
+                    new PipeTaskMeta(
+                        new TimeWindowStateProgressIndex(timeSeries2TimestampWindowBufferPairMap),
+                        789));
                 put(Integer.MIN_VALUE, new PipeTaskMeta(new MetaProgressIndex(987), 0));
               }
             });
diff --git a/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileReader.java b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileReader.java
index 18a6476af7e..a02740abd70 100644
--- a/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileReader.java
+++ b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileReader.java
@@ -36,7 +36,7 @@ public class TsFileReader implements AutoCloseable {
   private IChunkLoader chunkLoader;
   private TsFileExecutor tsFileExecutor;
 
-  /** constructor, create ReadOnlyTsFile with TsFileSequenceReader. */
+  /** Constructor, create ReadOnlyTsFile with {@link TsFileSequenceReader}. */
   public TsFileReader(TsFileSequenceReader fileReader) throws IOException {
     this.fileReader = fileReader;
     this.metadataQuerier = new MetadataQuerierByFileImpl(fileReader);
diff --git a/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
index 2c8511f3ef7..e942a92754c 100644
--- a/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
+++ b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
@@ -313,7 +313,7 @@ public class TsFileSequenceReader implements AutoCloseable {
   }
 
   /**
-   * this function does not modify the position of the file reader.
+   * This function does not modify the position of the file reader.
    *
    * @throws IOException io error
    */
diff --git a/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
index 03a97e97c96..f40bb3de06b 100644
--- a/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
+++ b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
@@ -106,7 +106,7 @@ public class ReadWriteIOUtils {
     return null;
   }
 
-  /** read a Boolean from byteBuffer. */
+  /** Read a Boolean from byteBuffer. */
   public static Boolean readBoolObject(InputStream inputStream) throws IOException {
     int flag = inputStream.read();
     if (flag == 1) {
@@ -117,13 +117,13 @@ public class ReadWriteIOUtils {
     return null;
   }
 
-  /** read a byte from byteBuffer. */
+  /** Read a byte from byteBuffer. */
   public static byte readByte(ByteBuffer buffer) {
     return buffer.get();
   }
 
   /**
-   * read bytes array in given size
+   * Read bytes array in given size
    *
    * @param buffer buffer
    * @param size size
@@ -316,7 +316,7 @@ public class ReadWriteIOUtils {
   }
 
   /**
-   * write a float n to outputStream.
+   * Write a float n to outputStream.
    *
    * @return The number of bytes used to represent n.
    */
@@ -327,7 +327,7 @@ public class ReadWriteIOUtils {
   }
 
   /**
-   * write a double n to outputStream.
+   * Write a double n to outputStream.
    *
    * @return The number of bytes used to represent n.
    */
@@ -367,7 +367,7 @@ public class ReadWriteIOUtils {
   }
 
   /**
-   * write string to outputStream.
+   * Write string to outputStream.
    *
    * @return the length of string represented by byte[].
    */
@@ -386,7 +386,7 @@ public class ReadWriteIOUtils {
   }
 
   /**
-   * write string to outputStream.
+   * Write string to outputStream.
    *
    * @return the length of string represented by byte[].
    */
@@ -520,12 +520,12 @@ public class ReadWriteIOUtils {
     return BytesUtils.bytesToShort(bytes);
   }
 
-  /** read a short var from byteBuffer. */
+  /** Read a short var from byteBuffer. */
   public static short readShort(ByteBuffer buffer) {
     return buffer.getShort();
   }
 
-  /** read a float var from inputStream. */
+  /** Read a float var from inputStream. */
   public static float readFloat(InputStream inputStream) throws IOException {
     byte[] bytes = new byte[FLOAT_LEN];
     int readLen = inputStream.read(bytes);
@@ -535,14 +535,14 @@ public class ReadWriteIOUtils {
     return BytesUtils.bytesToFloat(bytes);
   }
 
-  /** read a float var from byteBuffer. */
+  /** Read a float var from byteBuffer. */
   public static float readFloat(ByteBuffer byteBuffer) {
     byte[] bytes = new byte[FLOAT_LEN];
     byteBuffer.get(bytes);
     return BytesUtils.bytesToFloat(bytes);
   }
 
-  /** read a double var from inputStream. */
+  /** Read a double var from inputStream. */
   public static double readDouble(InputStream inputStream) throws IOException {
     byte[] bytes = new byte[DOUBLE_LEN];
     int readLen = inputStream.read(bytes);
@@ -552,14 +552,14 @@ public class ReadWriteIOUtils {
     return BytesUtils.bytesToDouble(bytes);
   }
 
-  /** read a double var from byteBuffer. */
+  /** Read a double var from byteBuffer. */
   public static double readDouble(ByteBuffer byteBuffer) {
     byte[] bytes = new byte[DOUBLE_LEN];
     byteBuffer.get(bytes);
     return BytesUtils.bytesToDouble(bytes);
   }
 
-  /** read a int var from inputStream. */
+  /** Read an int var from inputStream. */
   public static int readInt(InputStream inputStream) throws IOException {
     byte[] bytes = new byte[INT_LEN];
     int readLen = inputStream.read(bytes);
@@ -569,13 +569,13 @@ public class ReadWriteIOUtils {
     return BytesUtils.bytesToInt(bytes);
   }
 
-  /** read a int var from byteBuffer. */
+  /** Read a int var from byteBuffer. */
   public static int readInt(ByteBuffer buffer) {
     return buffer.getInt();
   }
 
   /**
-   * read an unsigned byte(0 ~ 255) as InputStream does.
+   * Read an unsigned byte(0 ~ 255) as InputStream does.
    *
    * @return the byte or -1(means there is no byte to read)
    */
@@ -586,7 +586,7 @@ public class ReadWriteIOUtils {
     return buffer.get() & 0xFF;
   }
 
-  /** read a long var from inputStream. */
+  /** Read a long var from inputStream. */
   public static long readLong(InputStream inputStream) throws IOException {
     byte[] bytes = new byte[LONG_LEN];
     int readLen = inputStream.read(bytes);
@@ -596,12 +596,12 @@ public class ReadWriteIOUtils {
     return BytesUtils.bytesToLong(bytes);
   }
 
-  /** read a long var from byteBuffer. */
+  /** Read a long var from byteBuffer. */
   public static long readLong(ByteBuffer buffer) {
     return buffer.getLong();
   }
 
-  /** read string from inputStream. */
+  /** Read string from inputStream. */
   public static String readString(InputStream inputStream) throws IOException {
     int strLength = readInt(inputStream);
     if (strLength <= 0) {
@@ -615,7 +615,7 @@ public class ReadWriteIOUtils {
     return new String(bytes, 0, strLength);
   }
 
-  /** string length's type is varInt */
+  /** String length's type is varInt */
   public static String readVarIntString(InputStream inputStream) throws IOException {
     int strLength = ReadWriteForEncodingUtils.readVarInt(inputStream);
     if (strLength < 0) {
@@ -631,7 +631,7 @@ public class ReadWriteIOUtils {
     return new String(bytes, 0, strLength);
   }
 
-  /** read string from byteBuffer. */
+  /** Read string from byteBuffer. */
   public static String readString(ByteBuffer buffer) {
     int strLength = readInt(buffer);
     if (strLength < 0) {
@@ -644,7 +644,7 @@ public class ReadWriteIOUtils {
     return new String(bytes, 0, strLength);
   }
 
-  /** string length's type is varInt */
+  /** String length's type is varInt */
   public static String readVarIntString(ByteBuffer buffer) {
     int strLength = ReadWriteForEncodingUtils.readVarInt(buffer);
     if (strLength < 0) {
@@ -657,7 +657,7 @@ public class ReadWriteIOUtils {
     return new String(bytes, 0, strLength);
   }
 
-  /** read string from byteBuffer with user define length. */
+  /** Read string from byteBuffer with user define length. */
   public static String readStringWithLength(ByteBuffer buffer, int length) {
     if (length < 0) {
       return null;
@@ -929,7 +929,7 @@ public class ReadWriteIOUtils {
     }
   }
 
-  // read long set with self define length
+  // Read long set with self define length
   public static Set<Long> readLongSet(ByteBuffer buffer) {
     int size = readInt(buffer);
     if (size <= 0) {
@@ -1009,7 +1009,7 @@ public class ReadWriteIOUtils {
   }
 
   /**
-   * to check whether the byte buffer is reach the magic string this method doesn't change the
+   * To check whether the byte buffer is reach the magic string this method doesn't change the
    * position of the byte buffer
    *
    * @param byteBuffer byte buffer
diff --git a/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/Tablet.java b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/Tablet.java
index 1deb10f2543..220059d1354 100644
--- a/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/Tablet.java
+++ b/iotdb-core/tsfile/src/main/java/org/apache/iotdb/tsfile/write/record/Tablet.java
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.iotdb.tsfile.write.record;
 
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
@@ -52,45 +53,46 @@ public class Tablet {
   private static final int DEFAULT_SIZE = 1024;
   private static final String NOT_SUPPORT_DATATYPE = "Data type %s is not supported.";
 
-  /** deviceId of this tablet */
+  /** DeviceId of this {@link Tablet} */
   public String deviceId;
 
-  /** the list of measurement schemas for creating the tablet */
+  /** The list of {@link MeasurementSchema}s for creating the {@link Tablet} */
   private List<MeasurementSchema> schemas;
 
-  /** measurementId->indexOf(measurementSchema) */
+  /** MeasurementId->indexOf({@link MeasurementSchema}) */
   private final Map<String, Integer> measurementIndex;
 
-  /** timestamps in this tablet */
+  /** Timestamps in this {@link Tablet} */
   public long[] timestamps;
-  /** each object is a primitive type array, which represents values of one measurement */
+  /** Each object is a primitive type array, which represents values of one measurement */
   public Object[] values;
-  /** each bitmap represents the existence of each value in the current column. */
+  /** Each {@link BitMap} represents the existence of each value in the current column. */
   public BitMap[] bitMaps;
-  /** the number of rows to include in this tablet */
+  /** The number of rows to include in this {@link Tablet} */
   public int rowSize;
-  /** the maximum number of rows for this tablet */
+  /** The maximum number of rows for this {@link Tablet} */
   private final int maxRowNumber;
 
   /**
-   * Return a tablet with default specified row number. This is the standard constructor (all Tablet
-   * should be the same size).
+   * Return a {@link Tablet} with default specified row number. This is the standard constructor
+   * (all Tablet should be the same size).
    *
    * @param deviceId the name of the device specified to be written in
-   * @param schemas the list of measurement schemas for creating the tablet, only measurementId and
-   *     type take effects
+   * @param schemas the list of {@link MeasurementSchema}s for creating the tablet, only
+   *     measurementId and type take effects
    */
   public Tablet(String deviceId, List<MeasurementSchema> schemas) {
     this(deviceId, schemas, DEFAULT_SIZE);
   }
 
   /**
-   * Return a tablet with the specified number of rows (maxBatchSize). Only call this constructor
-   * directly for testing purposes. Tablet should normally always be default size.
+   * Return a {@link Tablet} with the specified number of rows (maxBatchSize). Only call this
+   * constructor directly for testing purposes. {@link Tablet} should normally always be default
+   * size.
    *
    * @param deviceId the name of the device specified to be written in
-   * @param schemas the list of measurement schemas for creating the row batch, only measurementId
-   *     and type take effects
+   * @param schemas the list of {@link MeasurementSchema}s for creating the row batch, only
+   *     measurementId and type take effects
    * @param maxRowNumber the maximum number of rows for this tablet
    */
   public Tablet(String deviceId, List<MeasurementSchema> schemas, int maxRowNumber) {
@@ -106,16 +108,16 @@ public class Tablet {
   }
 
   /**
-   * Return a tablet with specified timestamps and values. Only call this constructor directly for
-   * Trigger.
+   * Return a {@link Tablet} with specified timestamps and values. Only call this constructor
+   * directly for Trigger.
    *
    * @param deviceId the name of the device specified to be written in
-   * @param schemas the list of measurement schemas for creating the row batch, only measurementId
-   *     and type take effects
+   * @param schemas the list of {@link MeasurementSchema}s for creating the row batch, only
+   *     measurementId and type take effects
    * @param timestamps given timestamps
    * @param values given values
-   * @param bitMaps given bitmaps
-   * @param maxRowNumber the maximum number of rows for this tablet
+   * @param bitMaps given {@link BitMap}s
+   * @param maxRowNumber the maximum number of rows for this {@link Tablet}
    */
   public Tablet(
       String deviceId,
@@ -173,14 +175,14 @@ public class Tablet {
       TSDataType dataType, int rowIndex, int indexOfSchema, Object value) {
 
     if (value == null) {
-      // init the bitMap to mark null value
+      // Init the bitMap to mark null value
       if (bitMaps == null) {
         bitMaps = new BitMap[values.length];
       }
       if (bitMaps[indexOfSchema] == null) {
         bitMaps[indexOfSchema] = new BitMap(maxRowNumber);
       }
-      // mark the null value position
+      // Mark the null value position
       bitMaps[indexOfSchema].mark(rowIndex);
     }
     switch (dataType) {
@@ -302,7 +304,7 @@ public class Tablet {
     return rowSize * 8;
   }
 
-  /** @return total bytes of values */
+  /** @return Total bytes of values */
   public int getTotalValueOccupation() {
     int valueOccupation = 0;
     int columnIndex = 0;
@@ -310,10 +312,10 @@ public class Tablet {
       valueOccupation += calOccupationOfOneColumn(schema.getType(), columnIndex);
       columnIndex++;
     }
-    // add bitmap size if the tablet has bitMaps
+    // Add bitmap size if the tablet has bitMaps
     if (bitMaps != null) {
       for (BitMap bitMap : bitMaps) {
-        // marker byte
+        // Marker byte
         valueOccupation++;
         if (bitMap != null && !bitMap.isAllUnmarked()) {
           valueOccupation += rowSize / Byte.SIZE + 1;
@@ -350,7 +352,7 @@ public class Tablet {
     return valueOccupation;
   }
 
-  /** serialize Tablet */
+  /** Serialize {@link Tablet} */
   public ByteBuffer serialize() throws IOException {
     try (PublicBAOS byteArrayOutputStream = new PublicBAOS();
         DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) {
@@ -368,7 +370,7 @@ public class Tablet {
     writeValues(stream);
   }
 
-  /** Serialize measurement schemas */
+  /** Serialize {@link MeasurementSchema}s */
   private void writeMeasurementSchemas(DataOutputStream stream) throws IOException {
     ReadWriteIOUtils.write(BytesUtils.boolToByte(schemas != null), stream);
     if (schemas != null) {
@@ -393,7 +395,7 @@ public class Tablet {
     }
   }
 
-  /** Serialize bitmaps */
+  /** Serialize {@link BitMap}s */
   private void writeBitMaps(DataOutputStream stream) throws IOException {
     ReadWriteIOUtils.write(BytesUtils.boolToByte(bitMaps != null), stream);
     if (bitMaps != null) {
@@ -607,13 +609,13 @@ public class Tablet {
   }
 
   /**
-   * Note that the function will judge 2 tablets to be equal when their contents are logically the
-   * same. Namely, a tablet with bitmap "null" may be equal to another tablet with 3 columns and
-   * bitmap "[null, null, null]", and a tablet with rowSize 2 is judged identical to other tablets
+   * Note that the function will judge 2 {@link Tablet}s to be equal when their contents are logically the
+   * same. Namely, a {@link Tablet} with {@link BitMap} "null" may be equal to another {@link Tablet} with 3 columns and
+   * {@link BitMap "[null, null, null]", and a {@link Tablet} with rowSize 2 is judged identical to other {@link Tablet}s
    * regardless of any timeStamps with indexes larger than or equal to 2.
    *
    * @param o the tablet to compare
-   * @return true if the tablets are logically equal
+   * @return {@code true} if the tablets are logically equal
    */
   @Override
   public boolean equals(Object o) {