You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ha...@apache.org on 2022/11/09 03:25:19 UTC

[iotdb] branch master updated: Remove old storage engine code and refactor UTs (#7932)

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

haonan 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 939bc592aa Remove old storage engine code and refactor UTs (#7932)
939bc592aa is described below

commit 939bc592aa4f489b41b40cff0f181ba5663465a8
Author: Haonan <hh...@outlook.com>
AuthorDate: Wed Nov 9 11:25:14 2022 +0800

    Remove old storage engine code and refactor UTs (#7932)
---
 .../IoTDBLoadExternalTsfileWithVirtualSGIT.java    |    4 +-
 .../db/integration/IoTDBRemovePartitionIT.java     |  622 ++++----
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |    4 +-
 .../org/apache/iotdb/db/engine/StorageEngine.java  | 1011 +------------
 .../apache/iotdb/db/engine/StorageEngineV2.java    |   42 +-
 .../iotdb/db/engine/memtable/AbstractMemTable.java |  195 ---
 .../apache/iotdb/db/engine/memtable/IMemTable.java |   23 +-
 .../iotdb/db/engine/storagegroup/DataRegion.java   |  501 +------
 .../db/engine/storagegroup/TsFileProcessor.java    |  163 +--
 .../dataregion/HashVirtualPartitioner.java         |   58 -
 .../dataregion/StorageGroupManager.java            |  505 -------
 .../dataregion/VirtualPartitioner.java             |   39 -
 .../storagegroup/timeindex/DeviceTimeIndex.java    |    4 +-
 .../storagegroup/timeindex/FileTimeIndex.java      |    6 +-
 .../schemaregion/SchemaRegionMemoryImpl.java       |    4 -
 .../schemaregion/SchemaRegionSchemaFileImpl.java   |    4 -
 .../apache/iotdb/db/metadata/tag/TagManager.java   |   58 +-
 .../planner/plan/node/write/InsertRowNode.java     |    2 +-
 .../planner/plan/node/write/InsertTabletNode.java  |    2 +-
 .../apache/iotdb/db/qp/executor/IPlanExecutor.java |   79 --
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  | 1374 ------------------
 .../java/org/apache/iotdb/db/service/IoTDB.java    |   10 -
 .../apache/iotdb/db/service/IoTDBShutdownHook.java |    9 +-
 .../org/apache/iotdb/db/service/MQTTService.java   |    6 +-
 .../org/apache/iotdb/db/service/SettleService.java |   36 +-
 .../org/apache/iotdb/db/service/UpgradeSevice.java |   16 +-
 .../db/sync/receiver/load/DeletionLoader.java      |    7 -
 .../iotdb/db/sync/receiver/load/TsFileLoader.java  |   15 -
 .../iotdb/db/tools/TsFileSplitByPartitionTool.java |   10 +-
 .../db/tools/dataregion/DeviceMappingViewer.java   |   63 -
 .../db/tools/upgrade/TsFileOnlineUpgradeTool.java  |    6 +-
 .../org/apache/iotdb/db/wal/buffer/WALEntry.java   |   23 +-
 .../apache/iotdb/db/wal/buffer/WALEntryType.java   |    3 +
 .../apache/iotdb/db/wal/buffer/WALInfoEntry.java   |   18 +-
 .../org/apache/iotdb/db/wal/node/IWALNode.java     |   11 -
 .../org/apache/iotdb/db/wal/node/WALFakeNode.java  |   19 -
 .../java/org/apache/iotdb/db/wal/node/WALNode.java |   42 +-
 .../db/wal/recover/file/TsFilePlanRedoer.java      |   73 -
 .../file/UnsealedTsFileRecoverPerformer.java       |   11 +-
 .../compaction/inner/InnerSeqCompactionTest.java   |    6 +-
 .../db/engine/memtable/MemTableFlushTaskTest.java  |    6 +-
 .../db/engine/memtable/MemTableTestUtils.java      |   69 +-
 .../db/engine/memtable/PrimitiveMemTableTest.java  |   65 +-
 .../db/engine/storagegroup/DataRegionTest.java     |   87 +-
 .../storagegroup/FileNodeManagerBenchmark.java     |  132 --
 .../storagegroup/StorageGroupProcessorTest.java    |  871 ------------
 .../iotdb/db/engine/storagegroup/TTLTest.java      |  151 +-
 .../engine/storagegroup/TsFileProcessorTest.java   |  476 -------
 .../dataregion/HashVirtualPartitionerTest.java     |   63 -
 .../org/apache/iotdb/db/metadata/TemplateTest.java |  137 +-
 .../db/metadata/idtable/IDTableFlushTimeTest.java  |  383 +++--
 .../db/metadata/idtable/IDTableRecoverTest.java    |  354 +++--
 .../db/metadata/idtable/LastQueryWithIDTable.java  |  285 ++--
 .../iotdb/db/qp/physical/ConcatOptimizerTest.java  |  140 --
 .../qp/physical/InsertRowsOfOneDevicePlanTest.java |   75 -
 .../db/qp/physical/PhysicalPlanSerializeTest.java  |  326 -----
 .../iotdb/db/qp/physical/PhysicalPlanTest.java     | 1490 --------------------
 .../iotdb/db/qp/physical/SerializationTest.java    |  151 --
 .../iotdb/db/qp/physical/SerializedSizeTest.java   |  130 --
 .../iotdb/db/query/dataset/SingleDataSetTest.java  |  199 ++-
 .../iotdb/db/tools/TsFileAndModSettleToolTest.java |  397 +++---
 .../org/apache/iotdb/db/tools/WalCheckerTest.java  |   16 +-
 .../apache/iotdb/db/utils/EnvironmentUtils.java    |   14 +-
 .../org/apache/iotdb/db/utils/MemUtilsTest.java    |   35 +-
 .../iotdb/db/utils/TsFileRewriteToolTest.java      |   11 +-
 .../org/apache/iotdb/db/wal/WALManagerTest.java    |   52 +-
 .../db/wal/allocation/ElasticStrategyTest.java     |   16 +-
 .../db/wal/allocation/FirstCreateStrategyTest.java |   18 +-
 .../db/wal/allocation/RoundRobinStrategyTest.java  |   16 +-
 .../iotdb/db/wal/buffer/WALBufferCommonTest.java   |   68 +-
 .../org/apache/iotdb/db/wal/io/WALFileTest.java    |  119 +-
 .../apache/iotdb/db/wal/node/WALFakeNodeTest.java  |   22 +-
 .../org/apache/iotdb/db/wal/node/WALNodeTest.java  |   90 +-
 .../db/wal/recover/WALRecoverManagerTest.java      |  100 +-
 .../db/wal/recover/file/TsFilePlanRedoerTest.java  |  240 ++--
 .../file/UnsealedTsFileRecoverPerformerTest.java   |   44 +-
 .../datanode3conf/iotdb-datanode.properties        |    2 +-
 .../apache/iotdb/spark/db/EnvironmentUtils.java    |   10 +-
 78 files changed, 1908 insertions(+), 10036 deletions(-)

diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileWithVirtualSGIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileWithVirtualSGIT.java
index 20f5040e91..2ece7adee3 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileWithVirtualSGIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileWithVirtualSGIT.java
@@ -22,7 +22,7 @@ import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.StorageEngineV2;
 import org.apache.iotdb.db.integration.sync.SyncTestUtil;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.itbase.category.LocalStandaloneTest;
@@ -51,7 +51,7 @@ public class IoTDBLoadExternalTsfileWithVirtualSGIT extends IoTDBLoadExternalTsf
     IoTDBDescriptor.getInstance().getConfig().setDataRegionNum(2);
     prevCompactionThread = IoTDBDescriptor.getInstance().getConfig().getCompactionThreadCount();
     EnvironmentUtils.envSetUp();
-    StorageEngine.getInstance().reset();
+    StorageEngineV2.getInstance().reset();
     Class.forName(Config.JDBC_DRIVER_NAME);
     prepareData(insertSequenceSqls);
   }
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBRemovePartitionIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBRemovePartitionIT.java
index b795b0319a..26daeb2532 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBRemovePartitionIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBRemovePartitionIT.java
@@ -19,315 +19,315 @@
 
 package org.apache.iotdb.db.integration;
 
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.itbase.category.LocalStandaloneTest;
-import org.apache.iotdb.jdbc.Config;
+// import org.apache.iotdb.commons.exception.IllegalPathException;
+// import org.apache.iotdb.commons.path.PartialPath;
+// import org.apache.iotdb.db.engine.StorageEngine;
+// import org.apache.iotdb.db.utils.EnvironmentUtils;
+// import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+// import org.apache.iotdb.jdbc.Config;
+//
+// import org.junit.After;
+// import org.junit.Before;
+// import org.junit.Test;
+// import org.junit.experimental.categories.Category;
+// import org.slf4j.Logger;
+// import org.slf4j.LoggerFactory;
+//
+// import java.sql.Connection;
+// import java.sql.DriverManager;
+// import java.sql.ResultSet;
+// import java.sql.Statement;
+// import java.util.ArrayList;
+// import java.util.Arrays;
+// import java.util.List;
+//
+// import static org.junit.Assert.assertEquals;
+// import static org.junit.Assert.assertFalse;
 
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-@Category({LocalStandaloneTest.class})
-public class IoTDBRemovePartitionIT {
-
-  private static final Logger logger = LoggerFactory.getLogger(IoTDBRemovePartitionIT.class);
-
-  private static int partitionInterval = 100;
-
-  @Before
-  public void setUp() throws Exception {
-    EnvironmentUtils.envSetUp();
-    StorageEngine.setEnablePartition(true);
-    StorageEngine.setTimePartitionInterval(partitionInterval);
-    insertData();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    StorageEngine.setEnablePartition(false);
-    StorageEngine.setTimePartitionInterval(-1);
-    EnvironmentUtils.cleanEnv();
-  }
-
-  @Test
-  public void testRemoveNoPartition() throws IllegalPathException {
-    logger.warn("running testRemoveNoPartition");
-    StorageEngine.getInstance()
-        .removePartitions(
-            new PartialPath("root.test1"), (storageGroupName, timePartitionId) -> false);
-
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-      try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.test1")) {
-        int count = 0;
-        while (resultSet.next()) {
-          assertEquals(count / 2 * 100L + count % 2 * 50, resultSet.getLong(1));
-          assertEquals(count / 2 * 100L + count % 2 * 50, resultSet.getLong(2));
-          count++;
-        }
-        assertEquals(20, count);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-
-  @Test
-  public void testRemovePartialPartition() throws IllegalPathException {
-    logger.warn("running testRemovePartialPartition");
-    StorageEngine.getInstance()
-        .removePartitions(
-            new PartialPath("root.test1"),
-            (storageGroupName, timePartitionId) -> timePartitionId >= 5);
-    StorageEngine.getInstance()
-        .removePartitions(
-            new PartialPath("root.test2"),
-            (storageGroupName, timePartitionId) -> timePartitionId < 5);
-
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-      try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.test1")) {
-        int count = 0;
-        while (resultSet.next()) {
-          assertEquals(count / 2 * 100 + count % 2 * 50, resultSet.getLong(1));
-          assertEquals(count / 2 * 100 + count % 2 * 50, resultSet.getLong(2));
-          count++;
-        }
-        assertEquals(10, count);
-      }
-
-      try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.test2")) {
-        int count = 0;
-        while (resultSet.next()) {
-          assertEquals(count / 2 * 100 + count % 2 * 50 + 500, resultSet.getLong(1));
-          assertEquals(count / 2 * 100 + count % 2 * 50 + 500, resultSet.getLong(2));
-          count++;
-        }
-        assertEquals(10, count);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-
-  @Test
-  public void testRemoveAllPartition() throws IllegalPathException {
-    logger.warn("running testRemoveAllPartition");
-    StorageEngine.getInstance()
-        .removePartitions(
-            new PartialPath("root.test1"), (storageGroupName, timePartitionId) -> true);
-
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-      try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.test1")) {
-        assertFalse(resultSet.next());
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-
-  @Test
-  public void testSQLRemovePartition() {
-    logger.warn("running testSQLRemovePartition");
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-      statement.execute("DELETE PARTITION root.test2 0,1,2,3,4");
-      try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.test2")) {
-        int count = 0;
-        while (resultSet.next()) {
-          assertEquals(count / 2 * 100 + count % 2 * 50 + 500, resultSet.getLong(1));
-          assertEquals(count / 2 * 100 + count % 2 * 50 + 500, resultSet.getLong(2));
-          count++;
-        }
-        assertEquals(10, count);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-
-  @Test
-  public void testRemoveOnePartitionAndInsertData() {
-    logger.warn("running testRemoveOnePartitionAndInsertData");
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-      statement.execute("set storage group to root.test");
-      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(1,true)");
-      statement.execute("select * from root.test.wf02.wt02");
-      statement.execute("DELETE PARTITION root.test 0");
-      statement.execute("select * from root.test.wf02.wt02");
-      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(1,true)");
-      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
-        assertEquals(true, resultSet.next());
-      }
-      statement.execute("flush");
-      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
-        assertEquals(true, resultSet.next());
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-
-  @Test
-  public void testRemovePartitionAndInsertUnSeqDataAndMerge() {
-    logger.warn("running testRemovePartitionAndInsertUnSeqDataAndMerge");
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-      statement.execute("set storage group to root.test");
-      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(2,true)");
-      statement.execute("select * from root.test.wf02.wt02");
-      statement.execute("DELETE PARTITION root.test 0");
-      statement.execute("select * from root.test.wf02.wt02");
-      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(1,true)");
-      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
-        assertEquals(true, resultSet.next());
-      }
-      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(3,true)");
-      statement.execute("merge");
-      int count = 0;
-      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
-        while (resultSet.next()) {
-          count++;
-        }
-        assertEquals(2, count);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-
-  @Test
-  public void testRemovePartitionAndInsertUnSeqDataAndUnSeqDataMerge() {
-    logger.warn("running testRemovePartitionAndInsertUnSeqDataAndUnSeqDataMerge");
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-      statement.execute("set storage group to root.test");
-      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(2,true)");
-      statement.execute("select * from root.test.wf02.wt02");
-      statement.execute("DELETE PARTITION root.test 0");
-      statement.execute("select * from root.test.wf02.wt02");
-      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(1,true)");
-      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
-        assertEquals(true, resultSet.next());
-      }
-      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(2,true)");
-      statement.execute("merge");
-      int count = 0;
-      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
-        while (resultSet.next()) {
-          count++;
-        }
-        assertEquals(2, count);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-
-  @Test
-  public void testFlushAndRemoveOnePartitionAndInsertData() {
-    logger.warn("running testFlushAndRemoveOnePartitionAndInsertData");
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-      statement.execute("set storage group to root.test");
-      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(1,true)");
-      statement.execute("flush");
-      statement.execute("DELETE PARTITION root.test 0");
-      statement.execute("select * from root.test.wf02.wt02");
-      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(1,true)");
-      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
-        assertEquals(true, resultSet.next());
-      }
-      statement.execute("flush");
-      int count = 0;
-      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
-        assertEquals(true, resultSet.next());
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-
-  private static void insertData() throws ClassNotFoundException {
-    List<String> sqls =
-        new ArrayList<>(
-            Arrays.asList(
-                "SET STORAGE GROUP TO root.test1",
-                "SET STORAGE GROUP TO root.test2",
-                "CREATE TIMESERIES root.test1.s0 WITH DATATYPE=INT64,ENCODING=PLAIN",
-                "CREATE TIMESERIES root.test2.s0 WITH DATATYPE=INT64,ENCODING=PLAIN"));
-    // 10 partitions, each one with one seq file and one unseq file
-    for (int i = 0; i < 10; i++) {
-      // seq files
-      for (int j = 1; j <= 2; j++) {
-        sqls.add(
-            String.format(
-                "INSERT INTO root.test%d(timestamp, s0) VALUES (%d, %d)",
-                j, i * partitionInterval + 50, i * partitionInterval + 50));
-      }
-      // last file is unclosed
-      if (i < 9) {
-        sqls.add("FLUSH");
-      }
-      // unseq files
-      for (int j = 1; j <= 2; j++) {
-        sqls.add(
-            String.format(
-                "INSERT INTO root.test%d(timestamp, s0) VALUES (%d, %d)",
-                j, i * partitionInterval, i * partitionInterval));
-      }
-      sqls.add("MERGE");
-      // last file is unclosed
-      if (i < 9) {
-        sqls.add("FLUSH");
-      }
-    }
-    Class.forName(Config.JDBC_DRIVER_NAME);
-    try (Connection connection =
-            DriverManager.getConnection(
-                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
-        Statement statement = connection.createStatement()) {
-
-      for (String sql : sqls) {
-        statement.execute(sql);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-}
+// @Category({LocalStandaloneTest.class})
+// public class IoTDBRemovePartitionIT {
+//
+//  private static final Logger logger = LoggerFactory.getLogger(IoTDBRemovePartitionIT.class);
+//
+//  private static int partitionInterval = 100;
+//
+//  @Before
+//  public void setUp() throws Exception {
+//    EnvironmentUtils.envSetUp();
+////    StorageEngine.setEnablePartition(true);
+////    StorageEngine.setTimePartitionInterval(partitionInterval);
+//    insertData();
+//  }
+//
+//  @After
+//  public void tearDown() throws Exception {
+////    StorageEngine.setEnablePartition(false);
+////    StorageEngine.setTimePartitionInterval(-1);
+//    EnvironmentUtils.cleanEnv();
+//  }
+//
+//  @Test
+//  public void testRemoveNoPartition() throws IllegalPathException {
+//    logger.warn("running testRemoveNoPartition");
+//    StorageEngine.getInstance()
+//        .removePartitions(
+//            new PartialPath("root.test1"), (storageGroupName, timePartitionId) -> false);
+//
+//    try (Connection connection =
+//            DriverManager.getConnection(
+//                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+//        Statement statement = connection.createStatement()) {
+//      try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.test1")) {
+//        int count = 0;
+//        while (resultSet.next()) {
+//          assertEquals(count / 2 * 100L + count % 2 * 50, resultSet.getLong(1));
+//          assertEquals(count / 2 * 100L + count % 2 * 50, resultSet.getLong(2));
+//          count++;
+//        }
+//        assertEquals(20, count);
+//      }
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//    }
+//  }
+//
+//  @Test
+//  public void testRemovePartialPartition() throws IllegalPathException {
+//    logger.warn("running testRemovePartialPartition");
+//    StorageEngine.getInstance()
+//        .removePartitions(
+//            new PartialPath("root.test1"),
+//            (storageGroupName, timePartitionId) -> timePartitionId >= 5);
+//    StorageEngine.getInstance()
+//        .removePartitions(
+//            new PartialPath("root.test2"),
+//            (storageGroupName, timePartitionId) -> timePartitionId < 5);
+//
+//    try (Connection connection =
+//            DriverManager.getConnection(
+//                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+//        Statement statement = connection.createStatement()) {
+//      try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.test1")) {
+//        int count = 0;
+//        while (resultSet.next()) {
+//          assertEquals(count / 2 * 100 + count % 2 * 50, resultSet.getLong(1));
+//          assertEquals(count / 2 * 100 + count % 2 * 50, resultSet.getLong(2));
+//          count++;
+//        }
+//        assertEquals(10, count);
+//      }
+//
+//      try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.test2")) {
+//        int count = 0;
+//        while (resultSet.next()) {
+//          assertEquals(count / 2 * 100 + count % 2 * 50 + 500, resultSet.getLong(1));
+//          assertEquals(count / 2 * 100 + count % 2 * 50 + 500, resultSet.getLong(2));
+//          count++;
+//        }
+//        assertEquals(10, count);
+//      }
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//    }
+//  }
+//
+//  @Test
+//  public void testRemoveAllPartition() throws IllegalPathException {
+//    logger.warn("running testRemoveAllPartition");
+//    StorageEngine.getInstance()
+//        .removePartitions(
+//            new PartialPath("root.test1"), (storageGroupName, timePartitionId) -> true);
+//
+//    try (Connection connection =
+//            DriverManager.getConnection(
+//                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+//        Statement statement = connection.createStatement()) {
+//      try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.test1")) {
+//        assertFalse(resultSet.next());
+//      }
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//    }
+//  }
+//
+//  @Test
+//  public void testSQLRemovePartition() {
+//    logger.warn("running testSQLRemovePartition");
+//    try (Connection connection =
+//            DriverManager.getConnection(
+//                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+//        Statement statement = connection.createStatement()) {
+//      statement.execute("DELETE PARTITION root.test2 0,1,2,3,4");
+//      try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.test2")) {
+//        int count = 0;
+//        while (resultSet.next()) {
+//          assertEquals(count / 2 * 100 + count % 2 * 50 + 500, resultSet.getLong(1));
+//          assertEquals(count / 2 * 100 + count % 2 * 50 + 500, resultSet.getLong(2));
+//          count++;
+//        }
+//        assertEquals(10, count);
+//      }
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//    }
+//  }
+//
+//  @Test
+//  public void testRemoveOnePartitionAndInsertData() {
+//    logger.warn("running testRemoveOnePartitionAndInsertData");
+//    try (Connection connection =
+//            DriverManager.getConnection(
+//                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+//        Statement statement = connection.createStatement()) {
+//      statement.execute("set storage group to root.test");
+//      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(1,true)");
+//      statement.execute("select * from root.test.wf02.wt02");
+//      statement.execute("DELETE PARTITION root.test 0");
+//      statement.execute("select * from root.test.wf02.wt02");
+//      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(1,true)");
+//      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
+//        assertEquals(true, resultSet.next());
+//      }
+//      statement.execute("flush");
+//      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
+//        assertEquals(true, resultSet.next());
+//      }
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//    }
+//  }
+//
+//  @Test
+//  public void testRemovePartitionAndInsertUnSeqDataAndMerge() {
+//    logger.warn("running testRemovePartitionAndInsertUnSeqDataAndMerge");
+//    try (Connection connection =
+//            DriverManager.getConnection(
+//                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+//        Statement statement = connection.createStatement()) {
+//      statement.execute("set storage group to root.test");
+//      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(2,true)");
+//      statement.execute("select * from root.test.wf02.wt02");
+//      statement.execute("DELETE PARTITION root.test 0");
+//      statement.execute("select * from root.test.wf02.wt02");
+//      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(1,true)");
+//      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
+//        assertEquals(true, resultSet.next());
+//      }
+//      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(3,true)");
+//      statement.execute("merge");
+//      int count = 0;
+//      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
+//        while (resultSet.next()) {
+//          count++;
+//        }
+//        assertEquals(2, count);
+//      }
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//    }
+//  }
+//
+//  @Test
+//  public void testRemovePartitionAndInsertUnSeqDataAndUnSeqDataMerge() {
+//    logger.warn("running testRemovePartitionAndInsertUnSeqDataAndUnSeqDataMerge");
+//    try (Connection connection =
+//            DriverManager.getConnection(
+//                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+//        Statement statement = connection.createStatement()) {
+//      statement.execute("set storage group to root.test");
+//      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(2,true)");
+//      statement.execute("select * from root.test.wf02.wt02");
+//      statement.execute("DELETE PARTITION root.test 0");
+//      statement.execute("select * from root.test.wf02.wt02");
+//      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(1,true)");
+//      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
+//        assertEquals(true, resultSet.next());
+//      }
+//      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(2,true)");
+//      statement.execute("merge");
+//      int count = 0;
+//      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
+//        while (resultSet.next()) {
+//          count++;
+//        }
+//        assertEquals(2, count);
+//      }
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//    }
+//  }
+//
+//  @Test
+//  public void testFlushAndRemoveOnePartitionAndInsertData() {
+//    logger.warn("running testFlushAndRemoveOnePartitionAndInsertData");
+//    try (Connection connection =
+//            DriverManager.getConnection(
+//                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+//        Statement statement = connection.createStatement()) {
+//      statement.execute("set storage group to root.test");
+//      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(1,true)");
+//      statement.execute("flush");
+//      statement.execute("DELETE PARTITION root.test 0");
+//      statement.execute("select * from root.test.wf02.wt02");
+//      statement.execute("insert into root.test.wf02.wt02(timestamp,status) values(1,true)");
+//      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
+//        assertEquals(true, resultSet.next());
+//      }
+//      statement.execute("flush");
+//      int count = 0;
+//      try (ResultSet resultSet = statement.executeQuery("select * from root.test.wf02.wt02")) {
+//        assertEquals(true, resultSet.next());
+//      }
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//    }
+//  }
+//
+//  private static void insertData() throws ClassNotFoundException {
+//    List<String> sqls =
+//        new ArrayList<>(
+//            Arrays.asList(
+//                "SET STORAGE GROUP TO root.test1",
+//                "SET STORAGE GROUP TO root.test2",
+//                "CREATE TIMESERIES root.test1.s0 WITH DATATYPE=INT64,ENCODING=PLAIN",
+//                "CREATE TIMESERIES root.test2.s0 WITH DATATYPE=INT64,ENCODING=PLAIN"));
+//    // 10 partitions, each one with one seq file and one unseq file
+//    for (int i = 0; i < 10; i++) {
+//      // seq files
+//      for (int j = 1; j <= 2; j++) {
+//        sqls.add(
+//            String.format(
+//                "INSERT INTO root.test%d(timestamp, s0) VALUES (%d, %d)",
+//                j, i * partitionInterval + 50, i * partitionInterval + 50));
+//      }
+//      // last file is unclosed
+//      if (i < 9) {
+//        sqls.add("FLUSH");
+//      }
+//      // unseq files
+//      for (int j = 1; j <= 2; j++) {
+//        sqls.add(
+//            String.format(
+//                "INSERT INTO root.test%d(timestamp, s0) VALUES (%d, %d)",
+//                j, i * partitionInterval, i * partitionInterval));
+//      }
+//      sqls.add("MERGE");
+//      // last file is unclosed
+//      if (i < 9) {
+//        sqls.add("FLUSH");
+//      }
+//    }
+//    Class.forName(Config.JDBC_DRIVER_NAME);
+//    try (Connection connection =
+//            DriverManager.getConnection(
+//                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+//        Statement statement = connection.createStatement()) {
+//
+//      for (String sql : sqls) {
+//        statement.execute(sql);
+//      }
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//    }
+//  }
+// }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index 4ce523aff3..10a94ad9fa 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -29,7 +29,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TCQConfig;
 import org.apache.iotdb.confignode.rpc.thrift.TGlobalConfig;
 import org.apache.iotdb.confignode.rpc.thrift.TRatisConfig;
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
-import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.StorageEngineV2;
 import org.apache.iotdb.db.engine.compaction.constant.CompactionPriority;
 import org.apache.iotdb.db.engine.compaction.constant.CrossCompactionPerformer;
 import org.apache.iotdb.db.engine.compaction.constant.CrossCompactionSelector;
@@ -1457,7 +1457,7 @@ public class IoTDBDescriptor {
 
       // update timed flush & close conf
       loadTimedService(properties);
-      StorageEngine.getInstance().rebootTimedService();
+      StorageEngineV2.getInstance().rebootTimedService();
 
       long seqTsFileSize =
           Long.parseLong(
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
index 6e0a7cf395..8f436ed5bf 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java
@@ -18,126 +18,34 @@
  */
 package org.apache.iotdb.db.engine;
 
-import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
-import org.apache.iotdb.commons.concurrent.ThreadName;
-import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil;
-import org.apache.iotdb.commons.conf.CommonDescriptor;
-import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.exception.ShutdownException;
-import org.apache.iotdb.commons.file.SystemFileFactory;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.service.IService;
 import org.apache.iotdb.commons.service.ServiceType;
-import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.conf.ServerConfigConsistent;
-import org.apache.iotdb.db.engine.flush.CloseFileListener;
-import org.apache.iotdb.db.engine.flush.FlushListener;
-import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy;
-import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy.DirectFlushPolicy;
 import org.apache.iotdb.db.engine.storagegroup.DataRegion;
-import org.apache.iotdb.db.engine.storagegroup.DataRegion.TimePartitionFilter;
-import org.apache.iotdb.db.engine.storagegroup.TsFileProcessor;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.engine.storagegroup.dataregion.StorageGroupManager;
-import org.apache.iotdb.db.exception.BatchProcessException;
 import org.apache.iotdb.db.exception.DataRegionException;
-import org.apache.iotdb.db.exception.LoadFileException;
 import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.TsFileProcessorException;
-import org.apache.iotdb.db.exception.WriteProcessException;
-import org.apache.iotdb.db.exception.WriteProcessRejectException;
-import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.exception.runtime.StorageEngineFailureException;
-import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowsOfOneDevicePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
-import org.apache.iotdb.db.rescon.SystemInfo;
 import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.utils.ThreadUtils;
-import org.apache.iotdb.db.utils.UpgradeUtils;
-import org.apache.iotdb.db.wal.exception.WALException;
-import org.apache.iotdb.db.wal.recover.WALRecoverManager;
-import org.apache.iotdb.rpc.RpcUtils;
-import org.apache.iotdb.rpc.TSStatusCode;
-import org.apache.iotdb.tsfile.utils.FilePathUtils;
 import org.apache.iotdb.tsfile.utils.Pair;
 
-import org.apache.commons.io.FileUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Comparator;
-import java.util.ConcurrentModificationException;
 import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
 
 public class StorageEngine implements IService {
   private static final Logger logger = LoggerFactory.getLogger(StorageEngine.class);
 
   private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
-  private static final long TTL_CHECK_INTERVAL = 60 * 1000L;
-
-  /**
-   * Time range for dividing storage group, the time unit is the same with IoTDB's
-   * TimestampPrecision
-   */
-  @ServerConfigConsistent private static long timePartitionInterval = -1;
-  /** whether enable data partition if disabled, all data belongs to partition 0 */
-  @ServerConfigConsistent private static boolean enablePartition = config.isEnablePartition();
-
-  private final boolean enableMemControl = config.isEnableMemControl();
-
-  /**
-   * a folder (system/storage_groups/ by default) that persist system info. Each Storage Processor
-   * will have a subfolder under the systemDir.
-   */
-  private final String systemDir =
-      FilePathUtils.regularizePath(config.getSystemDir()) + "storage_groups";
-
-  /** storage group name -> storage group processor */
-  private final ConcurrentHashMap<PartialPath, StorageGroupManager> processorMap =
-      new ConcurrentHashMap<>();
-
-  private AtomicBoolean isAllSgReady = new AtomicBoolean(false);
-
-  private ScheduledExecutorService ttlCheckThread;
-  private ScheduledExecutorService seqMemtableTimedFlushCheckThread;
-  private ScheduledExecutorService unseqMemtableTimedFlushCheckThread;
-
-  private TsFileFlushPolicy fileFlushPolicy = new DirectFlushPolicy();
-  private ExecutorService recoveryThreadPool;
-  // add customized listeners here for flush and close events
-  private List<CloseFileListener> customCloseFileListeners = new ArrayList<>();
-  private List<FlushListener> customFlushListeners = new ArrayList<>();
 
   private StorageEngine() {}
 
@@ -145,316 +53,20 @@ public class StorageEngine implements IService {
     return InstanceHolder.INSTANCE;
   }
 
-  private static void initTimePartition() {
-    timePartitionInterval =
-        IoTDBDescriptor.getInstance().getConfig().getTimePartitionIntervalForStorage();
-  }
-
-  public static long getTimePartitionInterval() {
-    if (timePartitionInterval == -1) {
-      initTimePartition();
-    }
-    return timePartitionInterval;
-  }
-
-  @TestOnly
-  public static void setTimePartitionInterval(long timePartitionInterval) {
-    StorageEngine.timePartitionInterval = timePartitionInterval;
-  }
-
-  public static long getTimePartition(long time) {
-    if (timePartitionInterval == -1) {
-      initTimePartition();
-    }
-    return enablePartition ? time / timePartitionInterval : 0;
-  }
-
-  public static boolean isEnablePartition() {
-    return enablePartition;
-  }
-
-  @TestOnly
-  public static void setEnablePartition(boolean enablePartition) {
-    StorageEngine.enablePartition = enablePartition;
-  }
-
-  /** block insertion if the insertion is rejected by memory control */
-  public static void blockInsertionIfReject(TsFileProcessor tsFileProcessor)
-      throws WriteProcessRejectException {
-    long startTime = System.currentTimeMillis();
-    while (SystemInfo.getInstance().isRejected()) {
-      if (tsFileProcessor != null && tsFileProcessor.shouldFlush()) {
-        break;
-      }
-      try {
-        TimeUnit.MILLISECONDS.sleep(config.getCheckPeriodWhenInsertBlocked());
-        if (System.currentTimeMillis() - startTime > config.getMaxWaitingTimeWhenInsertBlocked()) {
-          throw new WriteProcessRejectException(
-              "System rejected over " + (System.currentTimeMillis() - startTime) + "ms");
-        }
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-      }
-    }
-  }
-
-  public boolean isAllSgReady() {
-    return isAllSgReady.get();
-  }
-
-  public void setAllSgReady(boolean allSgReady) {
-    isAllSgReady.set(allSgReady);
-  }
-
-  public void recover() {
-    setAllSgReady(false);
-    recoveryThreadPool =
-        IoTDBThreadPoolFactory.newCachedThreadPool(
-            ThreadName.STORAGE_ENGINE_CACHED_SERVICE.getName());
-
-    List<IStorageGroupMNode> sgNodes = IoTDB.schemaProcessor.getAllStorageGroupNodes();
-    // init wal recover manager
-    WALRecoverManager.getInstance()
-        .setAllDataRegionScannedLatch(
-            new CountDownLatch(sgNodes.size() * config.getDataRegionNum()));
-    // recover all logic storage groups
-    List<Future<Void>> futures = new LinkedList<>();
-    for (IStorageGroupMNode storageGroup : sgNodes) {
-      StorageGroupManager storageGroupManager =
-          processorMap.computeIfAbsent(
-              storageGroup.getPartialPath(), id -> new StorageGroupManager(true));
-
-      // recover all virtual storage groups in each logic storage group
-      storageGroupManager.asyncRecover(storageGroup, recoveryThreadPool, futures);
-    }
-
-    // wait until wal is recovered
-    try {
-      WALRecoverManager.getInstance().recover();
-    } catch (WALException e) {
-      logger.error("Fail to recover wal.", e);
-    }
-
-    // operations after all virtual storage groups are recovered
-    Thread recoverEndTrigger =
-        new Thread(
-            () -> {
-              for (Future<Void> future : futures) {
-                try {
-                  future.get();
-                } catch (ExecutionException e) {
-                  throw new StorageEngineFailureException("StorageEngine failed to recover.", e);
-                } catch (InterruptedException e) {
-                  Thread.currentThread().interrupt();
-                  throw new StorageEngineFailureException("StorageEngine failed to recover.", e);
-                }
-              }
-              recoveryThreadPool.shutdown();
-              setAllSgReady(true);
-            });
-    recoverEndTrigger.start();
-  }
-
   @Override
-  public void start() {
-    // build time Interval to divide time partition
-    if (!enablePartition) {
-      timePartitionInterval = Long.MAX_VALUE;
-    } else {
-      initTimePartition();
-    }
-
-    // create systemDir
-    try {
-      FileUtils.forceMkdir(SystemFileFactory.INSTANCE.getFile(systemDir));
-    } catch (IOException e) {
-      throw new StorageEngineFailureException(e);
-    }
-
-    // recover upgrade process
-    UpgradeUtils.recoverUpgrade();
-
-    recover();
-
-    ttlCheckThread = IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor("TTL-Check");
-    ScheduledExecutorUtil.safelyScheduleAtFixedRate(
-        ttlCheckThread,
-        this::checkTTL,
-        TTL_CHECK_INTERVAL,
-        TTL_CHECK_INTERVAL,
-        TimeUnit.MILLISECONDS);
-    logger.info("start ttl check thread successfully.");
-
-    startTimedService();
-  }
-
-  private void checkTTL() {
-    try {
-      for (StorageGroupManager processor : processorMap.values()) {
-        processor.checkTTL();
-      }
-    } catch (ConcurrentModificationException e) {
-      // ignore
-    } catch (Exception e) {
-      logger.error("An error occurred when checking TTL", e);
-    }
-  }
-
-  private void startTimedService() {
-    // timed flush sequence memtable
-    if (config.isEnableTimedFlushSeqMemtable()) {
-      seqMemtableTimedFlushCheckThread =
-          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor(
-              ThreadName.TIMED_FlUSH_SEQ_MEMTABLE.getName());
-      ScheduledExecutorUtil.safelyScheduleAtFixedRate(
-          seqMemtableTimedFlushCheckThread,
-          this::timedFlushSeqMemTable,
-          config.getSeqMemtableFlushCheckInterval(),
-          config.getSeqMemtableFlushCheckInterval(),
-          TimeUnit.MILLISECONDS);
-      logger.info("start sequence memtable timed flush check thread successfully.");
-    }
-    // timed flush unsequence memtable
-    if (config.isEnableTimedFlushUnseqMemtable()) {
-      unseqMemtableTimedFlushCheckThread =
-          IoTDBThreadPoolFactory.newSingleThreadScheduledExecutor(
-              ThreadName.TIMED_FlUSH_UNSEQ_MEMTABLE.getName());
-      ScheduledExecutorUtil.safelyScheduleAtFixedRate(
-          unseqMemtableTimedFlushCheckThread,
-          this::timedFlushUnseqMemTable,
-          config.getUnseqMemtableFlushCheckInterval(),
-          config.getUnseqMemtableFlushCheckInterval(),
-          TimeUnit.MILLISECONDS);
-      logger.info("start unsequence memtable timed flush check thread successfully.");
-    }
-  }
-
-  private void timedFlushSeqMemTable() {
-    for (StorageGroupManager processor : processorMap.values()) {
-      processor.timedFlushSeqMemTable();
-    }
-  }
-
-  private void timedFlushUnseqMemTable() {
-    for (StorageGroupManager processor : processorMap.values()) {
-      processor.timedFlushUnseqMemTable();
-    }
-  }
+  public void start() {}
 
   @Override
-  public void stop() {
-    for (StorageGroupManager storageGroupManager : processorMap.values()) {
-      storageGroupManager.stopSchedulerPool();
-    }
-    syncCloseAllProcessor();
-    ThreadUtils.stopThreadPool(ttlCheckThread, ThreadName.TTL_CHECK_SERVICE);
-    ThreadUtils.stopThreadPool(
-        seqMemtableTimedFlushCheckThread, ThreadName.TIMED_FlUSH_SEQ_MEMTABLE);
-    ThreadUtils.stopThreadPool(
-        unseqMemtableTimedFlushCheckThread, ThreadName.TIMED_FlUSH_UNSEQ_MEMTABLE);
-    recoveryThreadPool.shutdownNow();
-    processorMap.clear();
-  }
+  public void stop() {}
 
   @Override
-  public void shutdown(long milliseconds) throws ShutdownException {
-    try {
-      for (StorageGroupManager storageGroupManager : processorMap.values()) {
-        storageGroupManager.stopSchedulerPool();
-      }
-      forceCloseAllProcessor();
-    } catch (TsFileProcessorException e) {
-      throw new ShutdownException(e);
-    }
-    shutdownTimedService(ttlCheckThread, "TTlCheckThread");
-    shutdownTimedService(seqMemtableTimedFlushCheckThread, "SeqMemtableTimedFlushCheckThread");
-    shutdownTimedService(unseqMemtableTimedFlushCheckThread, "UnseqMemtableTimedFlushCheckThread");
-    recoveryThreadPool.shutdownNow();
-    processorMap.clear();
-  }
-
-  private void shutdownTimedService(ScheduledExecutorService pool, String poolName) {
-    if (pool != null) {
-      pool.shutdownNow();
-      try {
-        pool.awaitTermination(30, TimeUnit.SECONDS);
-      } catch (InterruptedException e) {
-        logger.warn("{} still doesn't exit after 30s", poolName);
-        Thread.currentThread().interrupt();
-      }
-    }
-  }
-
-  /** reboot timed flush sequence/unsequence memetable thread */
-  public void rebootTimedService() throws ShutdownException {
-    logger.info("Start rebooting all timed service.");
-
-    // exclude ttl check thread
-    stopTimedServiceAndThrow(seqMemtableTimedFlushCheckThread, "SeqMemtableTimedFlushCheckThread");
-    stopTimedServiceAndThrow(
-        unseqMemtableTimedFlushCheckThread, "UnseqMemtableTimedFlushCheckThread");
-
-    logger.info("Stop all timed service successfully, and now restart them.");
-
-    startTimedService();
-
-    logger.info("Reboot all timed service successfully");
-  }
-
-  private void stopTimedServiceAndThrow(ScheduledExecutorService pool, String poolName)
-      throws ShutdownException {
-    if (pool != null) {
-      pool.shutdownNow();
-      try {
-        pool.awaitTermination(30, TimeUnit.SECONDS);
-      } catch (InterruptedException e) {
-        logger.warn("{} still doesn't exit after 30s", poolName);
-        throw new ShutdownException(e);
-      }
-    }
-  }
+  public void shutdown(long milliseconds) throws ShutdownException {}
 
   @Override
   public ServiceType getID() {
     return ServiceType.STORAGE_ENGINE_SERVICE;
   }
 
-  /**
-   * This method is for sync, delete tsfile or sth like them, just get storage group directly by sg
-   * name
-   *
-   * @param path storage group path
-   * @return storage group processor
-   */
-  public DataRegion getProcessorDirectly(PartialPath path) throws StorageEngineException {
-    PartialPath storageGroupPath;
-    try {
-      IStorageGroupMNode storageGroupMNode = IoTDB.schemaProcessor.getStorageGroupNodeByPath(path);
-      storageGroupPath = storageGroupMNode.getPartialPath();
-      return getStorageGroupProcessorByPath(storageGroupPath, storageGroupMNode);
-    } catch (DataRegionException | MetadataException e) {
-      throw new StorageEngineException(e);
-    }
-  }
-
-  /**
-   * This method is for sync, delete tsfile or sth like them, just get storage group directly by
-   * dataRegionId
-   *
-   * @param path storage group path
-   * @param dataRegionId dataRegionId
-   * @return storage group processor
-   */
-  public DataRegion getProcessorDirectly(PartialPath path, int dataRegionId)
-      throws StorageEngineException {
-    try {
-      IStorageGroupMNode storageGroupMNode = IoTDB.schemaProcessor.getStorageGroupNodeByPath(path);
-      return getStorageGroupProcessorById(dataRegionId, storageGroupMNode);
-    } catch (DataRegionException | MetadataException e) {
-      throw new StorageEngineException(e);
-    }
-  }
-
   /**
    * This method is for insert and query or sth like them, this may get a virtual storage group
    *
@@ -470,37 +82,6 @@ public class StorageEngine implements IService {
     }
   }
 
-  public DataRegion getProcessorByDataRegionId(PartialPath path, int dataRegionId)
-      throws StorageEngineException {
-    try {
-      IStorageGroupMNode storageGroupMNode = IoTDB.schemaProcessor.getStorageGroupNodeByPath(path);
-      return getStorageGroupManager(storageGroupMNode)
-          .getProcessor(storageGroupMNode, dataRegionId);
-    } catch (DataRegionException | MetadataException e) {
-      throw new StorageEngineException(e);
-    }
-  }
-
-  /**
-   * get lock holder for each sg
-   *
-   * @return storage group processor
-   */
-  public List<String> getLockInfo(List<PartialPath> pathList) throws StorageEngineException {
-    try {
-      List<String> lockHolderList = new ArrayList<>(pathList.size());
-      for (PartialPath path : pathList) {
-        IStorageGroupMNode storageGroupMNode =
-            IoTDB.schemaProcessor.getStorageGroupNodeByPath(path);
-        DataRegion dataRegion = getStorageGroupProcessorByPath(path, storageGroupMNode);
-        lockHolderList.add(dataRegion.getInsertWriteLockHolder());
-      }
-      return lockHolderList;
-    } catch (DataRegionException | MetadataException e) {
-      throw new StorageEngineException(e);
-    }
-  }
-
   /**
    * get storage group processor by device path
    *
@@ -510,562 +91,8 @@ public class StorageEngine implements IService {
    * @return found or new storage group processor
    */
   private DataRegion getStorageGroupProcessorByPath(
-      PartialPath devicePath, IStorageGroupMNode storageGroupMNode)
-      throws DataRegionException, StorageEngineException {
-    return getStorageGroupManager(storageGroupMNode).getProcessor(devicePath, storageGroupMNode);
-  }
-
-  /**
-   * get storage group processor by dataRegionId
-   *
-   * @param dataRegionId dataRegionId
-   * @param storageGroupMNode mnode of the storage group, we need synchronize this to avoid
-   *     modification in mtree
-   * @return found or new storage group processor
-   */
-  private DataRegion getStorageGroupProcessorById(
-      int dataRegionId, IStorageGroupMNode storageGroupMNode)
-      throws DataRegionException, StorageEngineException {
-    return getStorageGroupManager(storageGroupMNode).getProcessor(dataRegionId, storageGroupMNode);
-  }
-  /**
-   * get storage group manager by storage group mnode
-   *
-   * @param storageGroupMNode mnode of the storage group, we need synchronize this to avoid
-   *     modification in mtree
-   * @return found or new storage group manager
-   */
-  @SuppressWarnings("java:S2445")
-  // actually storageGroupMNode is a unique object on the mtree, synchronize it is reasonable
-  private StorageGroupManager getStorageGroupManager(IStorageGroupMNode storageGroupMNode) {
-    StorageGroupManager storageGroupManager = processorMap.get(storageGroupMNode.getPartialPath());
-    if (storageGroupManager == null) {
-      synchronized (this) {
-        storageGroupManager = processorMap.get(storageGroupMNode.getPartialPath());
-        if (storageGroupManager == null) {
-          storageGroupManager = new StorageGroupManager();
-          processorMap.put(storageGroupMNode.getPartialPath(), storageGroupManager);
-        }
-      }
-    }
-    return storageGroupManager;
-  }
-
-  /**
-   * build a new storage group processor
-   *
-   * @param virtualStorageGroupId virtual storage group id e.g. 1
-   * @param logicalStorageGroupName logical storage group name e.g. root.sg1
-   */
-  public DataRegion buildNewStorageGroupProcessor(
-      PartialPath logicalStorageGroupName,
-      IStorageGroupMNode storageGroupMNode,
-      String virtualStorageGroupId)
-      throws DataRegionException {
-    DataRegion processor;
-    logger.info(
-        "construct a processor instance, the storage group is {}, Thread is {}",
-        logicalStorageGroupName,
-        Thread.currentThread().getId());
-    processor =
-        new DataRegion(
-            systemDir + File.separator + logicalStorageGroupName,
-            virtualStorageGroupId,
-            fileFlushPolicy,
-            storageGroupMNode.getFullPath());
-    processor.setDataTTLWithTimePrecisionCheck(storageGroupMNode.getDataTTL());
-    processor.setCustomFlushListeners(customFlushListeners);
-    processor.setCustomCloseFileListeners(customCloseFileListeners);
-    return processor;
-  }
-
-  /** This function is just for unit test. */
-  @TestOnly
-  public synchronized void reset() {
-    for (StorageGroupManager storageGroupManager : processorMap.values()) {
-      storageGroupManager.reset();
-    }
-  }
-
-  /**
-   * insert an InsertRowPlan to a storage group.
-   *
-   * @param insertRowPlan physical plan of insertion
-   */
-  public void insert(InsertRowPlan insertRowPlan) throws StorageEngineException, MetadataException {
-    if (enableMemControl) {
-      try {
-        blockInsertionIfReject(null);
-      } catch (WriteProcessException e) {
-        throw new StorageEngineException(e);
-      }
-    }
-
-    DataRegion dataRegion = getProcessor(insertRowPlan.getDevicePath());
-    getSeriesSchemas(insertRowPlan, dataRegion);
-    try {
-      insertRowPlan.transferType();
-    } catch (QueryProcessException e) {
-      throw new StorageEngineException(e);
-    }
-
-    try {
-      dataRegion.insert(insertRowPlan);
-    } catch (WriteProcessException e) {
-      throw new StorageEngineException(e);
-    }
-  }
-
-  public void insert(InsertRowsOfOneDevicePlan insertRowsOfOneDevicePlan)
-      throws StorageEngineException, MetadataException {
-    if (enableMemControl) {
-      try {
-        blockInsertionIfReject(null);
-      } catch (WriteProcessException e) {
-        throw new StorageEngineException(e);
-      }
-    }
-
-    DataRegion dataRegion = getProcessor(insertRowsOfOneDevicePlan.getDevicePath());
-
-    for (InsertRowPlan plan : insertRowsOfOneDevicePlan.getRowPlans()) {
-      plan.setMeasurementMNodes(new IMeasurementMNode[plan.getMeasurements().length]);
-      // check whether types are match
-      getSeriesSchemas(plan, dataRegion);
-    }
-
-    // TODO monitor: update statistics
-    try {
-      dataRegion.insert(insertRowsOfOneDevicePlan);
-    } catch (WriteProcessException e) {
-      throw new StorageEngineException(e);
-    }
-  }
-
-  /** insert a InsertTabletPlan to a storage group */
-  public void insertTablet(InsertTabletPlan insertTabletPlan)
-      throws StorageEngineException, BatchProcessException, MetadataException {
-    if (enableMemControl) {
-      try {
-        blockInsertionIfReject(null);
-      } catch (WriteProcessRejectException e) {
-        TSStatus[] results = new TSStatus[insertTabletPlan.getRowCount()];
-        Arrays.fill(results, RpcUtils.getStatus(TSStatusCode.WRITE_PROCESS_REJECT));
-        throw new BatchProcessException(results);
-      }
-    }
-    DataRegion dataRegion;
-    try {
-      dataRegion = getProcessor(insertTabletPlan.getDevicePath());
-    } catch (StorageEngineException e) {
-      throw new StorageEngineException(
-          String.format(
-              "Get StorageGroupProcessor of device %s " + "failed",
-              insertTabletPlan.getDevicePath()),
-          e);
-    }
-
-    getSeriesSchemas(insertTabletPlan, dataRegion);
-    dataRegion.insertTablet(insertTabletPlan);
-  }
-
-  /** flush command Sync asyncCloseOneProcessor all file node processors. */
-  public void syncCloseAllProcessor() {
-    logger.info("Start closing all storage group processor");
-    for (StorageGroupManager processor : processorMap.values()) {
-      processor.syncCloseAllWorkingTsFileProcessors();
-    }
-  }
-
-  public void forceCloseAllProcessor() throws TsFileProcessorException {
-    logger.info("Start force closing all storage group processor");
-    for (StorageGroupManager processor : processorMap.values()) {
-      processor.forceCloseAllWorkingTsFileProcessors();
-    }
-  }
-
-  public void closeStorageGroupProcessor(
-      PartialPath storageGroupPath, boolean isSeq, boolean isSync) {
-    if (!processorMap.containsKey(storageGroupPath)) {
-      return;
-    }
-
-    StorageGroupManager storageGroupManager = processorMap.get(storageGroupPath);
-    storageGroupManager.closeStorageGroupProcessor(isSeq, isSync);
-  }
-
-  /**
-   * @param storageGroupPath the storage group name
-   * @param partitionId the partition id
-   * @param isSeq is sequence tsfile or unsequence tsfile
-   * @param isSync close tsfile synchronously or asynchronously
-   * @throws StorageGroupNotSetException
-   */
-  public void closeStorageGroupProcessor(
-      PartialPath storageGroupPath, long partitionId, boolean isSeq, boolean isSync)
-      throws StorageGroupNotSetException {
-    if (!processorMap.containsKey(storageGroupPath)) {
-      throw new StorageGroupNotSetException(storageGroupPath.getFullPath());
-    }
-
-    StorageGroupManager storageGroupManager = processorMap.get(storageGroupPath);
-    storageGroupManager.closeStorageGroupProcessor(partitionId, isSeq, isSync);
-  }
-
-  public void delete(
-      PartialPath path,
-      long startTime,
-      long endTime,
-      long planIndex,
-      TimePartitionFilter timePartitionFilter)
-      throws StorageEngineException {
-    try {
-      List<PartialPath> sgPaths = IoTDB.schemaProcessor.getBelongedStorageGroups(path);
-      for (PartialPath storageGroupPath : sgPaths) {
-        // storage group has no data
-        if (!processorMap.containsKey(storageGroupPath)) {
-          continue;
-        }
-
-        List<PartialPath> possiblePaths = path.alterPrefixPath(storageGroupPath);
-        for (PartialPath possiblePath : possiblePaths) {
-          processorMap
-              .get(storageGroupPath)
-              .delete(possiblePath, startTime, endTime, planIndex, timePartitionFilter);
-        }
-      }
-    } catch (IOException | MetadataException e) {
-      throw new StorageEngineException(e.getMessage());
-    }
-  }
-
-  /** delete data of timeseries "{deviceId}.{measurementId}" */
-  public void deleteTimeseries(
-      PartialPath path, long planIndex, TimePartitionFilter timePartitionFilter)
-      throws StorageEngineException {
-    try {
-      List<PartialPath> sgPaths = IoTDB.schemaProcessor.getBelongedStorageGroups(path);
-      for (PartialPath storageGroupPath : sgPaths) {
-        // storage group has no data
-        if (!processorMap.containsKey(storageGroupPath)) {
-          continue;
-        }
-
-        List<PartialPath> possiblePaths = path.alterPrefixPath(storageGroupPath);
-        for (PartialPath possiblePath : possiblePaths) {
-          processorMap
-              .get(storageGroupPath)
-              .delete(possiblePath, Long.MIN_VALUE, Long.MAX_VALUE, planIndex, timePartitionFilter);
-        }
-      }
-    } catch (IOException | MetadataException e) {
-      throw new StorageEngineException(e.getMessage());
-    }
-  }
-
-  /**
-   * count all Tsfiles which need to be upgraded
-   *
-   * @return total num of the tsfiles which need to be upgraded
-   */
-  public int countUpgradeFiles() {
-    int totalUpgradeFileNum = 0;
-    for (StorageGroupManager storageGroupManager : processorMap.values()) {
-      totalUpgradeFileNum += storageGroupManager.countUpgradeFiles();
-    }
-    return totalUpgradeFileNum;
-  }
-
-  /**
-   * upgrade all storage groups.
-   *
-   * @throws StorageEngineException StorageEngineException
-   */
-  public void upgradeAll() throws StorageEngineException {
-    if (CommonDescriptor.getInstance().getConfig().isReadOnly()) {
-      throw new StorageEngineException(
-          "Current system mode is read only, does not support file upgrade");
-    }
-    for (StorageGroupManager storageGroupManager : processorMap.values()) {
-      storageGroupManager.upgradeAll();
-    }
-  }
-
-  public void getResourcesToBeSettled(
-      PartialPath sgPath,
-      List<TsFileResource> seqResourcesToBeSettled,
-      List<TsFileResource> unseqResourcesToBeSettled,
-      List<String> tsFilePaths)
-      throws StorageEngineException {
-    StorageGroupManager storageGroupManager = processorMap.get(sgPath);
-    if (storageGroupManager == null) {
-      throw new StorageEngineException(
-          "The Storage Group " + sgPath.toString() + " is not existed.");
-    }
-    if (!storageGroupManager.getIsSettling().compareAndSet(false, true)) {
-      throw new StorageEngineException(
-          "Storage Group " + sgPath.getFullPath() + " is already being settled now.");
-    }
-    storageGroupManager.getResourcesToBeSettled(
-        seqResourcesToBeSettled, unseqResourcesToBeSettled, tsFilePaths);
-  }
-
-  public void setSettling(PartialPath sgPath, boolean isSettling) {
-    if (processorMap.get(sgPath) == null) {
-      return;
-    }
-    processorMap.get(sgPath).setSettling(isSettling);
-  }
-
-  /**
-   * merge all storage groups.
-   *
-   * @throws StorageEngineException StorageEngineException
-   */
-  public void mergeAll() throws StorageEngineException {
-    if (CommonDescriptor.getInstance().getConfig().isReadOnly()) {
-      throw new StorageEngineException("Current system mode is read only, does not support merge");
-    }
-
-    for (StorageGroupManager storageGroupManager : processorMap.values()) {
-      storageGroupManager.mergeAll();
-    }
-  }
-
-  /**
-   * delete all data files (both memory data and file on disk) in a storage group. It is used when
-   * there is no timeseries (which are all deleted) in this storage group)
-   */
-  public void deleteAllDataFilesInOneStorageGroup(PartialPath storageGroupPath) {
-    if (processorMap.containsKey(storageGroupPath)) {
-      syncDeleteDataFiles(storageGroupPath);
-    }
-  }
-
-  private void syncDeleteDataFiles(PartialPath storageGroupPath) {
-    logger.info("Force to delete the data in storage group processor {}", storageGroupPath);
-    processorMap.get(storageGroupPath).syncDeleteDataFiles();
-  }
-
-  /** delete all data of storage groups' timeseries. */
-  @TestOnly
-  public synchronized boolean deleteAll() {
-    logger.info("Start deleting all storage groups' timeseries");
-    syncCloseAllProcessor();
-    for (PartialPath storageGroup : IoTDB.schemaProcessor.getAllStorageGroupPaths()) {
-      this.deleteAllDataFilesInOneStorageGroup(storageGroup);
-    }
-    processorMap.clear();
-    return true;
-  }
-
-  public void setTTL(PartialPath storageGroup, long dataTTL) {
-    // storage group has no data
-    if (!processorMap.containsKey(storageGroup)) {
-      return;
-    }
-
-    processorMap.get(storageGroup).setTTL(dataTTL);
-  }
-
-  public void deleteStorageGroup(PartialPath storageGroupPath) {
-    if (!processorMap.containsKey(storageGroupPath)) {
-      return;
-    }
-    abortCompactionTaskForStorageGroup(storageGroupPath);
-    deleteAllDataFilesInOneStorageGroup(storageGroupPath);
-    StorageGroupManager storageGroupManager = processorMap.remove(storageGroupPath);
-    storageGroupManager.deleteStorageGroupSystemFolder(systemDir);
-    storageGroupManager.stopSchedulerPool();
-  }
-
-  private void abortCompactionTaskForStorageGroup(PartialPath storageGroupPath) {
-    if (!processorMap.containsKey(storageGroupPath)) {
-      return;
-    }
-
-    StorageGroupManager manager = processorMap.get(storageGroupPath);
-    manager.setAllowCompaction(false);
-    manager.abortCompaction();
-  }
-
-  public void loadNewTsFile(TsFileResource newTsFileResource, boolean deleteOriginFile)
-      throws LoadFileException, StorageEngineException, MetadataException {
-    Set<String> deviceSet = newTsFileResource.getDevices();
-    if (deviceSet == null || deviceSet.isEmpty()) {
-      throw new StorageEngineException("The TsFile is empty, cannot be loaded.");
-    }
-    String device = deviceSet.iterator().next();
-    PartialPath devicePath = new PartialPath(device);
-    PartialPath storageGroupPath = IoTDB.schemaProcessor.getBelongedStorageGroup(devicePath);
-    getProcessorDirectly(storageGroupPath).loadNewTsFile(newTsFileResource, deleteOriginFile);
-  }
-
-  public boolean deleteTsfile(File deletedTsfile)
-      throws StorageEngineException, IllegalPathException {
-    return getProcessorDirectly(
-            new PartialPath(getSgByEngineFile(deletedTsfile, true)),
-            getDataRegionIdByEngineFile(deletedTsfile, true))
-        .deleteTsfile(deletedTsfile);
-  }
-
-  public boolean unloadTsfile(File tsfileToBeUnloaded, File targetDir)
-      throws StorageEngineException, IllegalPathException {
-    return getProcessorDirectly(
-            new PartialPath(getSgByEngineFile(tsfileToBeUnloaded, true)),
-            getDataRegionIdByEngineFile(tsfileToBeUnloaded, true))
-        .unloadTsfile(tsfileToBeUnloaded, targetDir);
-  }
-
-  /**
-   * The internal file means that the file is in the engine, which is different from those external
-   * files which are not loaded.
-   *
-   * @param file internal file
-   * @param needCheck check if the tsfile is an internal TsFile. If you make sure it is inside, no
-   *     need to check
-   * @return sg name
-   * @throws IllegalPathException throw if tsfile is not an internal TsFile
-   */
-  public String getSgByEngineFile(File file, boolean needCheck) throws IllegalPathException {
-    if (needCheck) {
-      File dataDir =
-          file.getParentFile().getParentFile().getParentFile().getParentFile().getParentFile();
-      if (dataDir.exists()) {
-        String[] dataDirs = IoTDBDescriptor.getInstance().getConfig().getDataDirs();
-        for (String dir : dataDirs) {
-          try {
-            if (Files.isSameFile(Paths.get(dir), dataDir.toPath())) {
-              return file.getParentFile().getParentFile().getParentFile().getName();
-            }
-          } catch (IOException e) {
-            throw new IllegalPathException(file.getAbsolutePath(), e.getMessage());
-          }
-        }
-      }
-      throw new IllegalPathException(file.getAbsolutePath(), "it's not an internal tsfile.");
-    } else {
-      return file.getParentFile().getParentFile().getParentFile().getName();
-    }
-  }
-
-  /**
-   * The internal file means that the file is in the engine, which is different from those external
-   * files which are not loaded.
-   *
-   * @param file internal file
-   * @param needCheck check if the tsfile is an internal TsFile. If you make sure it is inside, no
-   *     need to check
-   * @return dataRegionId
-   * @throws IllegalPathException throw if tsfile is not an internal TsFile
-   */
-  public int getDataRegionIdByEngineFile(File file, boolean needCheck) throws IllegalPathException {
-    if (needCheck) {
-      File dataDir =
-          file.getParentFile().getParentFile().getParentFile().getParentFile().getParentFile();
-      if (dataDir.exists()) {
-        String[] dataDirs = IoTDBDescriptor.getInstance().getConfig().getDataDirs();
-        for (String dir : dataDirs) {
-          try {
-            if (Files.isSameFile(Paths.get(dir), dataDir.toPath())) {
-              return Integer.parseInt(file.getParentFile().getParentFile().getName());
-            }
-          } catch (IOException e) {
-            throw new IllegalPathException(file.getAbsolutePath(), e.getMessage());
-          }
-        }
-      }
-      throw new IllegalPathException(file.getAbsolutePath(), "it's not an internal tsfile.");
-    } else {
-      return Integer.parseInt(file.getParentFile().getParentFile().getName());
-    }
-  }
-
-  /**
-   * Get all the closed tsfiles of each storage group.
-   *
-   * @return TsFiles (seq or unseq) grouped by their storage group and partition number.
-   */
-  public Map<PartialPath, Map<Long, List<TsFileResource>>> getAllClosedStorageGroupTsFile() {
-    Map<PartialPath, Map<Long, List<TsFileResource>>> ret = new HashMap<>();
-    for (Entry<PartialPath, StorageGroupManager> entry : processorMap.entrySet()) {
-      entry.getValue().getAllClosedStorageGroupTsFile(entry.getKey(), ret);
-    }
-    return ret;
-  }
-
-  public void setFileFlushPolicy(TsFileFlushPolicy fileFlushPolicy) {
-    this.fileFlushPolicy = fileFlushPolicy;
-  }
-
-  public boolean isFileAlreadyExist(
-      TsFileResource tsFileResource, PartialPath storageGroup, long partitionNum) {
-    StorageGroupManager storageGroupManager = processorMap.get(storageGroup);
-    if (storageGroupManager == null) {
-      return false;
-    }
-
-    Iterator<String> partialPathIterator = tsFileResource.getDevices().iterator();
-    try {
-      return getProcessor(new PartialPath(partialPathIterator.next()))
-          .isFileAlreadyExist(tsFileResource, partitionNum);
-    } catch (StorageEngineException | IllegalPathException e) {
-      logger.error("can't find processor with: " + tsFileResource, e);
-    }
-
-    return false;
-  }
-
-  /**
-   * Set the version of given partition to newMaxVersion if it is larger than the current version.
-   */
-  public void setPartitionVersionToMax(
-      PartialPath storageGroup, long partitionId, long newMaxVersion) {
-    processorMap.get(storageGroup).setPartitionVersionToMax(partitionId, newMaxVersion);
-  }
-
-  public void removePartitions(PartialPath storageGroupPath, TimePartitionFilter filter) {
-    if (processorMap.get(storageGroupPath) != null) {
-      processorMap.get(storageGroupPath).removePartitions(filter);
-    }
-  }
-
-  public Map<PartialPath, StorageGroupManager> getProcessorMap() {
-    return processorMap;
-  }
-
-  /**
-   * Get a map indicating which storage groups have working TsFileProcessors and its associated
-   * partitionId and whether it is sequence or not.
-   *
-   * @return storage group -> a list of partitionId-isSequence pairs
-   */
-  public Map<String, List<Pair<Long, Boolean>>> getWorkingStorageGroupPartitions() {
-    Map<String, List<Pair<Long, Boolean>>> res = new ConcurrentHashMap<>();
-    for (Entry<PartialPath, StorageGroupManager> entry : processorMap.entrySet()) {
-      entry.getValue().getWorkingStorageGroupPartitions(entry.getKey().getFullPath(), res);
-    }
-    return res;
-  }
-
-  /**
-   * Add a listener to listen flush start/end events. Notice that this addition only applies to
-   * TsFileProcessors created afterwards.
-   *
-   * @param listener
-   */
-  public void registerFlushListener(FlushListener listener) {
-    customFlushListeners.add(listener);
-  }
-
-  /**
-   * Add a listener to listen file close events. Notice that this addition only applies to
-   * TsFileProcessors created afterwards.
-   *
-   * @param listener
-   */
-  public void registerCloseFileListener(CloseFileListener listener) {
-    customCloseFileListeners.add(listener);
+      PartialPath devicePath, IStorageGroupMNode storageGroupMNode) throws DataRegionException {
+    return null;
   }
 
   /** get all merge lock of the storage group processor related to the query */
@@ -1089,32 +116,8 @@ public class StorageEngine implements IService {
     list.forEach(DataRegion::readUnlock);
   }
 
-  /**
-   * Get the virtual storage group name.
-   *
-   * @return virtual storage group name, like root.sg1/0
-   */
-  public String getStorageGroupPath(PartialPath path) throws StorageEngineException {
-    PartialPath deviceId = path.getDevicePath();
-    DataRegion storageGroupProcessor = getProcessor(deviceId);
-    return storageGroupProcessor.getStorageGroupName()
-        + File.separator
-        + storageGroupProcessor.getDataRegionId();
-  }
-
-  protected void getSeriesSchemas(InsertPlan insertPlan, DataRegion processor)
-      throws StorageEngineException, MetadataException {
-    try {
-      if (config.isEnableIDTable()) {
-        processor.getIdTable().getSeriesSchemas(insertPlan);
-      } else {
-        IoTDB.schemaProcessor.getSeriesSchemasAndReadLockDevice(insertPlan);
-        insertPlan.setDeviceID(
-            DeviceIDFactory.getInstance().getDeviceID(insertPlan.getDevicePath()));
-      }
-    } catch (IOException e) {
-      throw new StorageEngineException(e);
-    }
+  public String getStorageGroupPath(PartialPath selectedPath) {
+    return null;
   }
 
   static class InstanceHolder {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java
index 4bf7e2c453..b7bc628b12 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngineV2.java
@@ -687,19 +687,6 @@ public class StorageEngineV2 implements IService {
     dataRegionMap.put(regionId, newRegion);
   }
 
-  //  public TSStatus setTTL(TSetTTLReq req) {
-  //    Map<String, List<DataRegionId>> localDataRegionInfo =
-  //        StorageEngineV2.getInstance().getLocalDataRegionInfo();
-  //    List<DataRegionId> dataRegionIdList = localDataRegionInfo.get(req.storageGroup);
-  //    for (DataRegionId dataRegionId : dataRegionIdList) {
-  //      DataRegion dataRegion = dataRegionMap.get(dataRegionId);
-  //      if (dataRegion != null) {
-  //        dataRegion.setDataTTL(req.TTL);
-  //      }
-  //    }
-  //    return RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS);
-  //  }
-
   public TSStatus setTTL(TSetTTLReq req) {
     Map<String, List<DataRegionId>> localDataRegionInfo =
         StorageEngineV2.getInstance().getLocalDataRegionInfo();
@@ -792,6 +779,35 @@ public class StorageEngineV2 implements IService {
     return status;
   }
 
+  /** reboot timed flush sequence/unsequence memetable thread */
+  public void rebootTimedService() throws ShutdownException {
+    logger.info("Start rebooting all timed service.");
+
+    // exclude ttl check thread
+    stopTimedServiceAndThrow(seqMemtableTimedFlushCheckThread, "SeqMemtableTimedFlushCheckThread");
+    stopTimedServiceAndThrow(
+        unseqMemtableTimedFlushCheckThread, "UnseqMemtableTimedFlushCheckThread");
+
+    logger.info("Stop all timed service successfully, and now restart them.");
+
+    startTimedService();
+
+    logger.info("Reboot all timed service successfully");
+  }
+
+  private void stopTimedServiceAndThrow(ScheduledExecutorService pool, String poolName)
+      throws ShutdownException {
+    if (pool != null) {
+      pool.shutdownNow();
+      try {
+        pool.awaitTermination(30, TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        logger.warn("{} still doesn't exit after 30s", poolName);
+        throw new ShutdownException(e);
+      }
+    }
+  }
+
   static class InstanceHolder {
 
     private static final StorageEngineV2 INSTANCE = new StorageEngineV2();
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
index 2f2379f16e..e34b79a173 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
@@ -35,8 +35,6 @@ import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
 import org.apache.iotdb.db.metadata.utils.ResourceByPathUtils;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.utils.MemUtils;
 import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
 import org.apache.iotdb.db.wal.utils.WALWriteUtils;
@@ -158,55 +156,6 @@ public abstract class AbstractMemTable implements IMemTable {
     return memChunkGroup;
   }
 
-  @Override
-  public void insert(InsertRowPlan insertRowPlan) {
-    // if this insert plan isn't from storage engine (mainly from test), we should set a temp device
-    // id for it
-    if (insertRowPlan.getDeviceID() == null) {
-      insertRowPlan.setDeviceID(deviceIDFactory.getDeviceID(insertRowPlan.getDevicePath()));
-    }
-
-    String[] measurements = insertRowPlan.getMeasurements();
-    Object[] values = insertRowPlan.getValues();
-
-    List<IMeasurementSchema> schemaList = new ArrayList<>();
-    List<TSDataType> dataTypes = new ArrayList<>();
-    int nullPointsNumber = 0;
-    for (int i = 0; i < insertRowPlan.getMeasurements().length; i++) {
-      // use measurements[i] to ignore failed partial insert
-      if (measurements[i] == null) {
-        schemaList.add(null);
-        continue;
-      }
-      // use values[i] to ignore null value
-      if (values[i] == null) {
-        schemaList.add(null);
-        nullPointsNumber++;
-        continue;
-      }
-      IMeasurementSchema schema = insertRowPlan.getMeasurementMNodes()[i].getSchema();
-      schemaList.add(schema);
-      dataTypes.add(schema.getType());
-    }
-    memSize += MemUtils.getRecordsSize(dataTypes, values, disableMemControl);
-    write(insertRowPlan.getDeviceID(), schemaList, insertRowPlan.getTime(), values);
-
-    int pointsInserted =
-        insertRowPlan.getMeasurements().length
-            - insertRowPlan.getFailedMeasurementNumber()
-            - nullPointsNumber;
-
-    totalPointsNum += pointsInserted;
-
-    MetricService.getInstance()
-        .count(
-            pointsInserted,
-            Metric.QUANTITY.toString(),
-            MetricLevel.IMPORTANT,
-            Tag.NAME.toString(),
-            METRIC_POINT_IN);
-  }
-
   @Override
   public void insert(InsertRowNode insertRowNode) {
     // if this insert plan isn't from storage engine (mainly from test), we should set a temp device
@@ -256,46 +205,6 @@ public abstract class AbstractMemTable implements IMemTable {
             METRIC_POINT_IN);
   }
 
-  @Override
-  public void insertAlignedRow(InsertRowPlan insertRowPlan) {
-    // if this insert plan isn't from storage engine, we should set a temp device id for it
-    if (insertRowPlan.getDeviceID() == null) {
-      insertRowPlan.setDeviceID(deviceIDFactory.getDeviceID(insertRowPlan.getDevicePath()));
-    }
-
-    String[] measurements = insertRowPlan.getMeasurements();
-    Object[] values = insertRowPlan.getValues();
-
-    List<IMeasurementSchema> schemaList = new ArrayList<>();
-    List<TSDataType> dataTypes = new ArrayList<>();
-    for (int i = 0; i < insertRowPlan.getMeasurements().length; i++) {
-      // use measurements[i] to ignore failed partial insert
-      if (measurements[i] == null) {
-        schemaList.add(null);
-        continue;
-      }
-      IMeasurementSchema schema = insertRowPlan.getMeasurementMNodes()[i].getSchema();
-      schemaList.add(schema);
-      dataTypes.add(schema.getType());
-    }
-    if (schemaList.isEmpty()) {
-      return;
-    }
-    memSize += MemUtils.getAlignedRecordsSize(dataTypes, values, disableMemControl);
-    writeAlignedRow(insertRowPlan.getDeviceID(), schemaList, insertRowPlan.getTime(), values);
-    int pointsInserted =
-        insertRowPlan.getMeasurements().length - insertRowPlan.getFailedMeasurementNumber();
-    totalPointsNum += pointsInserted;
-
-    MetricService.getInstance()
-        .count(
-            pointsInserted,
-            Metric.QUANTITY.toString(),
-            MetricLevel.IMPORTANT,
-            Tag.NAME.toString(),
-            METRIC_POINT_IN);
-  }
-
   @Override
   public void insertAlignedRow(InsertRowNode insertRowNode) {
     // if this insert node isn't from storage engine, we should set a temp device id for it
@@ -335,50 +244,6 @@ public abstract class AbstractMemTable implements IMemTable {
             METRIC_POINT_IN);
   }
 
-  @Override
-  public void insertTablet(InsertTabletPlan insertTabletPlan, int start, int end)
-      throws WriteProcessException {
-    try {
-      write(insertTabletPlan, start, end);
-      memSize += MemUtils.getTabletSize(insertTabletPlan, start, end, disableMemControl);
-      int pointsInserted =
-          (insertTabletPlan.getDataTypes().length - insertTabletPlan.getFailedMeasurementNumber())
-              * (end - start);
-      totalPointsNum += pointsInserted;
-      MetricService.getInstance()
-          .count(
-              pointsInserted,
-              Metric.QUANTITY.toString(),
-              MetricLevel.IMPORTANT,
-              Tag.NAME.toString(),
-              METRIC_POINT_IN);
-    } catch (RuntimeException e) {
-      throw new WriteProcessException(e);
-    }
-  }
-
-  @Override
-  public void insertAlignedTablet(InsertTabletPlan insertTabletPlan, int start, int end)
-      throws WriteProcessException {
-    try {
-      writeAlignedTablet(insertTabletPlan, start, end);
-      memSize += MemUtils.getAlignedTabletSize(insertTabletPlan, start, end, disableMemControl);
-      int pointsInserted =
-          (insertTabletPlan.getDataTypes().length - insertTabletPlan.getFailedMeasurementNumber())
-              * (end - start);
-      totalPointsNum += pointsInserted;
-      MetricService.getInstance()
-          .count(
-              pointsInserted,
-              Metric.QUANTITY.toString(),
-              MetricLevel.IMPORTANT,
-              Tag.NAME.toString(),
-              METRIC_POINT_IN);
-    } catch (RuntimeException e) {
-      throw new WriteProcessException(e);
-    }
-  }
-
   @Override
   public void insertTablet(InsertTabletNode insertTabletNode, int start, int end)
       throws WriteProcessException {
@@ -449,35 +314,6 @@ public abstract class AbstractMemTable implements IMemTable {
     }
   }
 
-  @SuppressWarnings("squid:S3776") // high Cognitive Complexity
-  @Override
-  public void write(InsertTabletPlan insertTabletPlan, int start, int end) {
-    // if this insert plan isn't from storage engine, we should set a temp device id for it
-    if (insertTabletPlan.getDeviceID() == null) {
-      insertTabletPlan.setDeviceID(deviceIDFactory.getDeviceID(insertTabletPlan.getDevicePath()));
-    }
-
-    List<IMeasurementSchema> schemaList = new ArrayList<>();
-    for (int i = 0; i < insertTabletPlan.getMeasurements().length; i++) {
-      if (insertTabletPlan.getColumns()[i] == null) {
-        schemaList.add(null);
-      } else {
-        schemaList.add(insertTabletPlan.getMeasurementMNodes()[i].getSchema());
-      }
-    }
-    IWritableMemChunkGroup memChunkGroup =
-        createMemChunkGroupIfNotExistAndGet(insertTabletPlan.getDeviceID(), schemaList);
-    if (memChunkGroup.writeValuesWithFlushCheck(
-        insertTabletPlan.getTimes(),
-        insertTabletPlan.getColumns(),
-        insertTabletPlan.getBitMaps(),
-        schemaList,
-        start,
-        end)) {
-      shouldFlush = true;
-    }
-  }
-
   public void write(InsertTabletNode insertTabletNode, int start, int end) {
     // if this insert plan isn't from storage engine, we should set a temp device id for it
     if (insertTabletNode.getDeviceID() == null) {
@@ -505,37 +341,6 @@ public abstract class AbstractMemTable implements IMemTable {
     }
   }
 
-  @Override
-  public void writeAlignedTablet(InsertTabletPlan insertTabletPlan, int start, int end) {
-    // if this insert plan isn't from storage engine, we should set a temp device id for it
-    if (insertTabletPlan.getDeviceID() == null) {
-      insertTabletPlan.setDeviceID(deviceIDFactory.getDeviceID(insertTabletPlan.getDevicePath()));
-    }
-
-    List<IMeasurementSchema> schemaList = new ArrayList<>();
-    for (int i = 0; i < insertTabletPlan.getMeasurements().length; i++) {
-      if (insertTabletPlan.getColumns()[i] == null) {
-        schemaList.add(null);
-      } else {
-        schemaList.add(insertTabletPlan.getMeasurementMNodes()[i].getSchema());
-      }
-    }
-    if (schemaList.isEmpty()) {
-      return;
-    }
-    IWritableMemChunkGroup memChunkGroup =
-        createAlignedMemChunkGroupIfNotExistAndGet(insertTabletPlan.getDeviceID(), schemaList);
-    if (memChunkGroup.writeValuesWithFlushCheck(
-        insertTabletPlan.getTimes(),
-        insertTabletPlan.getColumns(),
-        insertTabletPlan.getBitMaps(),
-        schemaList,
-        start,
-        end)) {
-      shouldFlush = true;
-    }
-  }
-
   public void writeAlignedTablet(InsertTabletNode insertTabletNode, int start, int end) {
     // if this insert plan isn't from storage engine, we should set a temp device id for it
     if (insertTabletNode.getDeviceID() == null) {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
index adf995e948..9101efdf7c 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java
@@ -28,8 +28,6 @@ import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.wal.buffer.WALEntryValue;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
@@ -61,13 +59,6 @@ public interface IMemTable extends WALEntryValue {
       List<IMeasurementSchema> schemaList,
       long insertTime,
       Object[] objectValue);
-  /**
-   * write data in the range [start, end). Null value in each column values will be replaced by the
-   * subsequent non-null value, e.g., {1, null, 3, null, 5} will be {1, 3, 5, null, 5}
-   */
-  void write(InsertTabletPlan insertTabletPlan, int start, int end);
-
-  void writeAlignedTablet(InsertTabletPlan insertTabletPlan, int start, int end);
 
   /** @return the number of points */
   long size();
@@ -98,12 +89,8 @@ public interface IMemTable extends WALEntryValue {
   /**
    * insert into this memtable
    *
-   * @param insertRowPlan insertRowPlan
+   * @param insertRowNode insertRowNode
    */
-  void insert(InsertRowPlan insertRowPlan);
-
-  void insertAlignedRow(InsertRowPlan insertRowPlan);
-
   void insert(InsertRowNode insertRowNode);
 
   void insertAlignedRow(InsertRowNode insertRowNode);
@@ -113,16 +100,10 @@ public interface IMemTable extends WALEntryValue {
    * value in each column values will be replaced by the subsequent non-null value, e.g., {1, null,
    * 3, null, 5} will be {1, 3, 5, null, 5}
    *
-   * @param insertTabletPlan insertTabletPlan
+   * @param insertTabletNode insertTabletNode
    * @param start included
    * @param end excluded
    */
-  void insertTablet(InsertTabletPlan insertTabletPlan, int start, int end)
-      throws WriteProcessException;
-
-  void insertAlignedTablet(InsertTabletPlan insertTabletPlan, int start, int end)
-      throws WriteProcessException;
-
   void insertTablet(InsertTabletNode insertTabletNode, int start, int end)
       throws WriteProcessException;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
index 9dfe87c498..aef9ff8ac5 100755
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java
@@ -35,7 +35,6 @@ import org.apache.iotdb.consensus.ConsensusFactory;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.directories.DirectoryManager;
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.StorageEngineV2;
 import org.apache.iotdb.db.engine.TsFileMetricManager;
 import org.apache.iotdb.db.engine.compaction.CompactionRecoverManager;
@@ -50,8 +49,6 @@ import org.apache.iotdb.db.engine.memtable.IMemTable;
 import org.apache.iotdb.db.engine.modification.Deletion;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
-import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
-import org.apache.iotdb.db.engine.trigger.executor.TriggerEvent;
 import org.apache.iotdb.db.engine.upgrade.UpgradeCheckStatus;
 import org.apache.iotdb.db.engine.upgrade.UpgradeLog;
 import org.apache.iotdb.db.engine.version.SimpleFileVersionController;
@@ -69,7 +66,6 @@ import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.cache.DataNodeSchemaCache;
 import org.apache.iotdb.db.metadata.idtable.IDTable;
 import org.apache.iotdb.db.metadata.idtable.IDTableManager;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.DeleteDataNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertMultiTabletsNode;
@@ -77,10 +73,6 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowsNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowsOfOneDeviceNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowsOfOneDevicePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.qp.utils.DateTimeUtils;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.FileReaderManager;
@@ -540,9 +532,7 @@ public class DataRegion {
     // recover and start timed compaction thread
     initCompaction();
 
-    if (config.isMppMode()
-        ? StorageEngineV2.getInstance().isAllSgReady()
-        : StorageEngine.getInstance().isAllSgReady()) {
+    if (StorageEngineV2.getInstance().isAllSgReady()) {
       logger.info("The data region {}[{}] is created successfully", storageGroupName, dataRegionId);
     } else {
       logger.info(
@@ -610,12 +600,12 @@ public class DataRegion {
     for (TsFileResource resource : upgradeSeqFileList) {
       for (String deviceId : resource.getDevices()) {
         long endTime = resource.getEndTime(deviceId);
-        long endTimePartitionId = StorageEngine.getTimePartition(endTime);
+        long endTimePartitionId = StorageEngineV2.getTimePartition(endTime);
         lastFlushTimeManager.setOneDeviceLastTime(endTimePartitionId, deviceId, endTime);
         lastFlushTimeManager.setOneDeviceGlobalFlushedTime(deviceId, endTime);
 
         // set all the covered partition's LatestFlushedTime
-        long partitionId = StorageEngine.getTimePartition(resource.getStartTime(deviceId));
+        long partitionId = StorageEngineV2.getTimePartition(resource.getStartTime(deviceId));
         while (partitionId <= endTimePartitionId) {
           lastFlushTimeManager.setOneDeviceFlushedTime(partitionId, deviceId, endTime);
           if (!timePartitionIdVersionControllerMap.containsKey(partitionId)) {
@@ -845,49 +835,6 @@ public class DataRegion {
     }
   }
 
-  /**
-   * insert one row of data
-   *
-   * @param insertRowPlan one row of data
-   */
-  public void insert(InsertRowPlan insertRowPlan)
-      throws WriteProcessException, TriggerExecutionException {
-    // reject insertions that are out of ttl
-    if (!isAlive(insertRowPlan.getTime())) {
-      throw new OutOfTTLException(insertRowPlan.getTime(), (DateTimeUtils.currentTime() - dataTTL));
-    }
-    writeLock("InsertRow");
-    try {
-      // init map
-      long timePartitionId = StorageEngine.getTimePartition(insertRowPlan.getTime());
-
-      lastFlushTimeManager.ensureFlushedTimePartition(timePartitionId);
-
-      boolean isSequence =
-          insertRowPlan.getTime()
-              > lastFlushTimeManager.getFlushedTime(
-                  timePartitionId, insertRowPlan.getDevicePath().getFullPath());
-
-      // is unsequence and user set config to discard out of order data
-      if (!isSequence
-          && IoTDBDescriptor.getInstance().getConfig().isEnableDiscardOutOfOrderData()) {
-        return;
-      }
-
-      lastFlushTimeManager.ensureLastTimePartition(timePartitionId);
-
-      // fire trigger before insertion
-      TriggerEngine.fire(TriggerEvent.BEFORE_INSERT, insertRowPlan);
-      // insert to sequence or unSequence file
-      insertToTsFileProcessor(insertRowPlan, isSequence, timePartitionId);
-      // fire trigger after insertion
-      TriggerEngine.fire(TriggerEvent.AFTER_INSERT, insertRowPlan);
-    } finally {
-      writeUnlock();
-    }
-  }
-
-  // TODO: (New Insert)
   /**
    * insert one row of data
    *
@@ -933,125 +880,6 @@ public class DataRegion {
     }
   }
 
-  /**
-   * Insert a tablet (rows belonging to the same devices) into this storage group.
-   *
-   * @throws BatchProcessException if some of the rows failed to be inserted
-   */
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  public void insertTablet(InsertTabletPlan insertTabletPlan)
-      throws BatchProcessException, TriggerExecutionException {
-
-    writeLock("insertTablet");
-    try {
-      TSStatus[] results = new TSStatus[insertTabletPlan.getRowCount()];
-      Arrays.fill(results, RpcUtils.SUCCESS_STATUS);
-      boolean noFailure = true;
-
-      /*
-       * assume that batch has been sorted by client
-       */
-      int loc = 0;
-      while (loc < insertTabletPlan.getRowCount()) {
-        long currTime = insertTabletPlan.getTimes()[loc];
-        // skip points that do not satisfy TTL
-        if (!isAlive(currTime)) {
-          results[loc] =
-              RpcUtils.getStatus(
-                  TSStatusCode.OUT_OF_TTL_ERROR,
-                  "time " + currTime + " in current line is out of TTL: " + dataTTL);
-          loc++;
-          noFailure = false;
-        } else {
-          break;
-        }
-      }
-      // loc pointing at first legal position
-      if (loc == insertTabletPlan.getRowCount()) {
-        throw new BatchProcessException(results);
-      }
-
-      // fire trigger before insertion
-      final int firePosition = loc;
-      TriggerEngine.fire(TriggerEvent.BEFORE_INSERT, insertTabletPlan, firePosition);
-
-      // before is first start point
-      int before = loc;
-      // before time partition
-      long beforeTimePartition =
-          StorageEngine.getTimePartition(insertTabletPlan.getTimes()[before]);
-      // init map
-      long lastFlushTime =
-          lastFlushTimeManager.ensureFlushedTimePartitionAndInit(
-              beforeTimePartition, insertTabletPlan.getDevicePath().getFullPath(), Long.MIN_VALUE);
-      // if is sequence
-      boolean isSequence = false;
-      while (loc < insertTabletPlan.getRowCount()) {
-        long time = insertTabletPlan.getTimes()[loc];
-        long curTimePartition = StorageEngine.getTimePartition(time);
-        // start next partition
-        if (curTimePartition != beforeTimePartition) {
-          // insert last time partition
-          if (isSequence
-              || !IoTDBDescriptor.getInstance().getConfig().isEnableDiscardOutOfOrderData()) {
-            noFailure =
-                insertTabletToTsFileProcessor(
-                        insertTabletPlan, before, loc, isSequence, results, beforeTimePartition)
-                    && noFailure;
-          }
-          // re initialize
-          before = loc;
-          beforeTimePartition = curTimePartition;
-          lastFlushTime =
-              lastFlushTimeManager.ensureFlushedTimePartitionAndInit(
-                  beforeTimePartition,
-                  insertTabletPlan.getDevicePath().getFullPath(),
-                  Long.MIN_VALUE);
-
-          isSequence = false;
-        }
-        // still in this partition
-        else {
-          // judge if we should insert sequence
-          if (!isSequence && time > lastFlushTime) {
-            // insert into unsequence and then start sequence
-            if (!IoTDBDescriptor.getInstance().getConfig().isEnableDiscardOutOfOrderData()) {
-              noFailure =
-                  insertTabletToTsFileProcessor(
-                          insertTabletPlan, before, loc, false, results, beforeTimePartition)
-                      && noFailure;
-            }
-            before = loc;
-            isSequence = true;
-          }
-          loc++;
-        }
-      }
-
-      // do not forget last part
-      if (before < loc
-          && (isSequence
-              || !IoTDBDescriptor.getInstance().getConfig().isEnableDiscardOutOfOrderData())) {
-        noFailure =
-            insertTabletToTsFileProcessor(
-                    insertTabletPlan, before, loc, isSequence, results, beforeTimePartition)
-                && noFailure;
-      }
-      long globalLatestFlushedTime =
-          lastFlushTimeManager.getGlobalFlushedTime(insertTabletPlan.getDevicePath().getFullPath());
-      tryToUpdateBatchInsertLastCache(insertTabletPlan, globalLatestFlushedTime);
-
-      if (!noFailure) {
-        throw new BatchProcessException(results);
-      }
-
-      // fire trigger after insertion
-      TriggerEngine.fire(TriggerEvent.AFTER_INSERT, insertTabletPlan, firePosition);
-    } finally {
-      writeUnlock();
-    }
-  }
-
   /**
    * Insert a tablet (rows belonging to the same devices) into this storage group.
    *
@@ -1160,68 +988,6 @@ public class DataRegion {
     return dataTTL == Long.MAX_VALUE || (DateTimeUtils.currentTime() - time) <= dataTTL;
   }
 
-  /**
-   * insert batch to tsfile processor thread-safety that the caller need to guarantee The rows to be
-   * inserted are in the range [start, end) Null value in each column values will be replaced by the
-   * subsequent non-null value, e.g., {1, null, 3, null, 5} will be {1, 3, 5, null, 5}
-   *
-   * @param insertTabletPlan insert a tablet of a device
-   * @param sequence whether is sequence
-   * @param start start index of rows to be inserted in insertTabletPlan
-   * @param end end index of rows to be inserted in insertTabletPlan
-   * @param results result array
-   * @param timePartitionId time partition id
-   * @return false if any failure occurs when inserting the tablet, true otherwise
-   */
-  private boolean insertTabletToTsFileProcessor(
-      InsertTabletPlan insertTabletPlan,
-      int start,
-      int end,
-      boolean sequence,
-      TSStatus[] results,
-      long timePartitionId) {
-    // return when start >= end
-    if (start >= end) {
-      return true;
-    }
-
-    TsFileProcessor tsFileProcessor = getOrCreateTsFileProcessor(timePartitionId, sequence);
-    if (tsFileProcessor == null) {
-      for (int i = start; i < end; i++) {
-        results[i] =
-            RpcUtils.getStatus(
-                TSStatusCode.INTERNAL_SERVER_ERROR,
-                "can not create TsFileProcessor, timePartitionId: " + timePartitionId);
-      }
-      return false;
-    }
-
-    try {
-      tsFileProcessor.insertTablet(insertTabletPlan, start, end, results);
-    } catch (WriteProcessRejectException e) {
-      logger.warn("insert to TsFileProcessor rejected, {}", e.getMessage());
-      return false;
-    } catch (WriteProcessException e) {
-      logger.error("insert to TsFileProcessor error ", e);
-      return false;
-    }
-
-    lastFlushTimeManager.ensureLastTimePartition(timePartitionId);
-    // try to update the latest time of the device of this tsRecord
-    if (sequence) {
-      lastFlushTimeManager.updateLastTime(
-          timePartitionId,
-          insertTabletPlan.getDevicePath().getFullPath(),
-          insertTabletPlan.getTimes()[end - 1]);
-    }
-
-    // check memtable size and may async try to flush the work memtable
-    if (tsFileProcessor.shouldFlush()) {
-      fileFlushPolicy.apply(this, tsFileProcessor, sequence);
-    }
-    return true;
-  }
-
   /**
    * insert batch to tsfile processor thread-safety that the caller need to guarantee The rows to be
    * inserted are in the range [start, end) Null value in each column values will be replaced by the
@@ -1284,31 +1050,6 @@ public class DataRegion {
     return true;
   }
 
-  private void tryToUpdateBatchInsertLastCache(InsertTabletPlan plan, Long latestFlushedTime) {
-    if (!IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled()) {
-      return;
-    }
-    IMeasurementMNode[] mNodes = plan.getMeasurementMNodes();
-    for (int i = 0; i < mNodes.length; i++) {
-      if (plan.getColumns()[i] == null) {
-        continue;
-      }
-      // Update cached last value with high priority
-      if (mNodes[i] == null) {
-        IoTDB.schemaProcessor.updateLastCache(
-            plan.getDevicePath().concatNode(plan.getMeasurements()[i]),
-            plan.composeLastTimeValuePair(i),
-            true,
-            latestFlushedTime);
-      } else {
-        // in stand alone version, the seriesPath is not needed, just use measurementMNodes[i] to
-        // update last cache
-        IoTDB.schemaProcessor.updateLastCache(
-            mNodes[i], plan.composeLastTimeValuePair(i), true, latestFlushedTime);
-      }
-    }
-  }
-
   private void tryToUpdateBatchInsertLastCache(InsertTabletNode node, long latestFlushedTime) {
     if (!IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled()) {
       return;
@@ -1327,31 +1068,6 @@ public class DataRegion {
     }
   }
 
-  private void insertToTsFileProcessor(
-      InsertRowPlan insertRowPlan, boolean sequence, long timePartitionId)
-      throws WriteProcessException {
-    TsFileProcessor tsFileProcessor = getOrCreateTsFileProcessor(timePartitionId, sequence);
-    if (tsFileProcessor == null) {
-      return;
-    }
-
-    tsFileProcessor.insert(insertRowPlan);
-
-    // try to update the latest time of the device of this tsRecord
-    lastFlushTimeManager.updateLastTime(
-        timePartitionId, insertRowPlan.getDevicePath().getFullPath(), insertRowPlan.getTime());
-
-    long globalLatestFlushTime =
-        lastFlushTimeManager.getGlobalFlushedTime(insertRowPlan.getDevicePath().getFullPath());
-
-    tryToUpdateInsertLastCache(insertRowPlan, globalLatestFlushTime);
-
-    // check memtable size and may asyncTryToFlush the work memtable
-    if (tsFileProcessor.shouldFlush()) {
-      fileFlushPolicy.apply(this, tsFileProcessor, sequence);
-    }
-  }
-
   private void insertToTsFileProcessor(
       InsertRowNode insertRowNode, boolean sequence, long timePartitionId)
       throws WriteProcessException {
@@ -1377,31 +1093,6 @@ public class DataRegion {
     }
   }
 
-  private void tryToUpdateInsertLastCache(InsertRowPlan plan, Long latestFlushedTime) {
-    if (!IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled()) {
-      return;
-    }
-    IMeasurementMNode[] mNodes = plan.getMeasurementMNodes();
-    for (int i = 0; i < mNodes.length; i++) {
-      if (plan.getValues()[i] == null) {
-        continue;
-      }
-      // Update cached last value with high priority
-      if (mNodes[i] == null) {
-        IoTDB.schemaProcessor.updateLastCache(
-            plan.getDevicePath().concatNode(plan.getMeasurements()[i]),
-            plan.composeTimeValuePair(i),
-            true,
-            latestFlushedTime);
-      } else {
-        // in stand alone version, the seriesPath is not needed, just use measurementMNodes[i] to
-        // update last cache
-        IoTDB.schemaProcessor.updateLastCache(
-            mNodes[i], plan.composeTimeValuePair(i), true, latestFlushedTime);
-      }
-    }
-  }
-
   private void tryToUpdateInsertLastCache(InsertRowNode node, long latestFlushedTime) {
     if (!IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled()) {
       return;
@@ -2079,87 +1770,6 @@ public class DataRegion {
     return tsfileResourcesForQuery;
   }
 
-  /**
-   * Delete data whose timestamp <= 'timestamp' and belongs to the time series
-   * deviceId.measurementId.
-   *
-   * @param path the timeseries path of the to be deleted.
-   * @param startTime the startTime of delete range.
-   * @param endTime the endTime of delete range.
-   * @param timePartitionFilter
-   */
-  public void delete(
-      PartialPath path,
-      long startTime,
-      long endTime,
-      long planIndex,
-      TimePartitionFilter timePartitionFilter)
-      throws IOException {
-    // If there are still some old version tsfiles, the delete won't succeeded.
-    if (upgradeFileCount.get() != 0) {
-      throw new IOException(
-          "Delete failed. " + "Please do not delete until the old files upgraded.");
-    }
-    if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) {
-      throw new IOException(
-          "Delete failed. " + "Please do not delete until the old files settled.");
-    }
-    // TODO: how to avoid partial deletion?
-    // FIXME: notice that if we may remove a SGProcessor out of memory, we need to close all opened
-    // mod files in mergingModification, sequenceFileList, and unsequenceFileList
-    writeLock("delete");
-
-    // record files which are updated so that we can roll back them in case of exception
-    List<ModificationFile> updatedModFiles = new ArrayList<>();
-    boolean hasReleasedLock = false;
-    try {
-      Set<PartialPath> devicePaths = IoTDB.schemaProcessor.getBelongedDevices(path);
-      for (PartialPath device : devicePaths) {
-        // delete Last cache record if necessary
-        tryToDeleteLastCache(device, path, startTime, endTime);
-      }
-
-      // write log to impacted working TsFileProcessors
-      List<WALFlushListener> walListeners =
-          logDeleteInWAL(startTime, endTime, path, timePartitionFilter);
-
-      for (WALFlushListener walFlushListener : walListeners) {
-        if (walFlushListener.waitForResult() == WALFlushListener.Status.FAILURE) {
-          logger.error("Fail to log delete to wal.", walFlushListener.getCause());
-          throw walFlushListener.getCause();
-        }
-      }
-
-      Deletion deletion = new Deletion(path, MERGE_MOD_START_VERSION_NUM, startTime, endTime);
-
-      List<TsFileResource> sealedTsFileResource = new ArrayList<>();
-      List<TsFileResource> unsealedTsFileResource = new ArrayList<>();
-      separateTsFile(sealedTsFileResource, unsealedTsFileResource);
-
-      deleteDataInFiles(
-          unsealedTsFileResource, deletion, devicePaths, updatedModFiles, timePartitionFilter);
-
-      writeUnlock();
-      hasReleasedLock = true;
-
-      deleteDataInFiles(
-          sealedTsFileResource, deletion, devicePaths, updatedModFiles, timePartitionFilter);
-
-    } catch (Exception e) {
-      // roll back
-      for (ModificationFile modFile : updatedModFiles) {
-        modFile.abort();
-        // remember to close mod file
-        modFile.close();
-      }
-      throw new IOException(e);
-    } finally {
-      if (!hasReleasedLock) {
-        writeUnlock();
-      }
-    }
-  }
-
   /** Seperate tsfiles in TsFileManager to sealedList and unsealedList. */
   private void separateTsFile(
       List<TsFileResource> sealedResource, List<TsFileResource> unsealedResource) {
@@ -2266,44 +1876,14 @@ public class DataRegion {
     }
   }
 
-  private List<WALFlushListener> logDeleteInWAL(
-      long startTime, long endTime, PartialPath path, TimePartitionFilter timePartitionFilter) {
-    long timePartitionStartId = StorageEngine.getTimePartition(startTime);
-    long timePartitionEndId = StorageEngine.getTimePartition(endTime);
-    List<WALFlushListener> walFlushListeners = new ArrayList<>();
-    if (config.getWalMode() == WALMode.DISABLE) {
-      return walFlushListeners;
-    }
-    DeletePlan deletionPlan = new DeletePlan(startTime, endTime, path);
-    for (Map.Entry<Long, TsFileProcessor> entry : workSequenceTsFileProcessors.entrySet()) {
-      if (timePartitionStartId <= entry.getKey()
-          && entry.getKey() <= timePartitionEndId
-          && (timePartitionFilter == null
-              || timePartitionFilter.satisfy(storageGroupName, entry.getKey()))) {
-        WALFlushListener walFlushListener = entry.getValue().logDeleteInWAL(deletionPlan);
-        walFlushListeners.add(walFlushListener);
-      }
-    }
-    for (Map.Entry<Long, TsFileProcessor> entry : workUnsequenceTsFileProcessors.entrySet()) {
-      if (timePartitionStartId <= entry.getKey()
-          && entry.getKey() <= timePartitionEndId
-          && (timePartitionFilter == null
-              || timePartitionFilter.satisfy(storageGroupName, entry.getKey()))) {
-        WALFlushListener walFlushListener = entry.getValue().logDeleteInWAL(deletionPlan);
-        walFlushListeners.add(walFlushListener);
-      }
-    }
-    return walFlushListeners;
-  }
-
   private List<WALFlushListener> logDeletionInWAL(
       long startTime,
       long endTime,
       long searchIndex,
       PartialPath path,
       TimePartitionFilter timePartitionFilter) {
-    long timePartitionStartId = StorageEngine.getTimePartition(startTime);
-    long timePartitionEndId = StorageEngine.getTimePartition(endTime);
+    long timePartitionStartId = StorageEngineV2.getTimePartition(startTime);
+    long timePartitionEndId = StorageEngineV2.getTimePartition(endTime);
     List<WALFlushListener> walFlushListeners = new ArrayList<>();
     if (config.getWalMode() == WALMode.DISABLE) {
       return walFlushListeners;
@@ -2671,24 +2251,11 @@ public class DataRegion {
     }
   }
 
-  private void resetLastCacheWhenLoadingTsFile(TsFileResource resource)
-      throws IllegalPathException {
+  private void resetLastCacheWhenLoadingTsFile() throws IllegalPathException {
     if (!IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled()) {
       return;
     }
-
-    if (config.isMppMode()) {
-      // TODO: implement more precise process
-      DataNodeSchemaCache.getInstance().cleanUp();
-    } else {
-      for (String device : resource.getDevices()) {
-        try {
-          IoTDB.schemaProcessor.deleteLastCacheByDevice(new PartialPath(device));
-        } catch (MetadataException e) {
-          logger.warn(String.format("Create device %s error.", device));
-        }
-      }
-    }
+    DataNodeSchemaCache.getInstance().cleanUp();
   }
 
   /**
@@ -2741,7 +2308,7 @@ public class DataRegion {
           insertPos,
           deleteOriginFile);
 
-      resetLastCacheWhenLoadingTsFile(newTsFileResource); // update last cache
+      resetLastCacheWhenLoadingTsFile(); // update last cache
       updateLastFlushTime(newTsFileResource); // update last flush time
       long partitionNum = newTsFileResource.getTimePartition();
       updatePartitionFileVersion(partitionNum, newTsFileResource.getVersion());
@@ -3471,58 +3038,6 @@ public class DataRegion {
     return tsFileManager;
   }
 
-  /**
-   * insert batch of rows belongs to one device
-   *
-   * @param insertRowsOfOneDevicePlan batch of rows belongs to one device
-   */
-  public void insert(InsertRowsOfOneDevicePlan insertRowsOfOneDevicePlan)
-      throws WriteProcessException, TriggerExecutionException {
-    writeLock("InsertRowsOfOneDevice");
-    try {
-      boolean isSequence = false;
-      InsertRowPlan[] rowPlans = insertRowsOfOneDevicePlan.getRowPlans();
-      for (int i = 0, rowPlansLength = rowPlans.length; i < rowPlansLength; i++) {
-
-        InsertRowPlan plan = rowPlans[i];
-        if (!isAlive(plan.getTime()) || insertRowsOfOneDevicePlan.isExecuted(i)) {
-          // we do not need to write these part of data, as they can not be queried
-          // or the sub-plan has already been executed, we are retrying other sub-plans
-          continue;
-        }
-        // init map
-        long timePartitionId = StorageEngine.getTimePartition(plan.getTime());
-
-        lastFlushTimeManager.ensureFlushedTimePartition(timePartitionId);
-        // as the plans have been ordered, and we have get the write lock,
-        // So, if a plan is sequenced, then all the rest plans are sequenced.
-        //
-        if (!isSequence) {
-          isSequence =
-              plan.getTime()
-                  > lastFlushTimeManager.getFlushedTime(
-                      timePartitionId, plan.getDevicePath().getFullPath());
-        }
-        // is unsequence and user set config to discard out of order data
-        if (!isSequence
-            && IoTDBDescriptor.getInstance().getConfig().isEnableDiscardOutOfOrderData()) {
-          return;
-        }
-
-        lastFlushTimeManager.ensureLastTimePartition(timePartitionId);
-
-        // fire trigger before insertion
-        TriggerEngine.fire(TriggerEvent.BEFORE_INSERT, plan);
-        // insert to sequence or unSequence file
-        insertToTsFileProcessor(plan, isSequence, timePartitionId);
-        // fire trigger before insertion
-        TriggerEngine.fire(TriggerEvent.AFTER_INSERT, plan);
-      }
-    } finally {
-      writeUnlock();
-    }
-  }
-
   /**
    * insert batch of rows belongs to one device
    *
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
index 122348b5e0..35eda60ca8 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
@@ -28,7 +28,7 @@ import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.adapter.CompressionRatio;
-import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.StorageEngineV2;
 import org.apache.iotdb.db.engine.TsFileMetricManager;
 import org.apache.iotdb.db.engine.flush.CloseFileListener;
 import org.apache.iotdb.db.engine.flush.FlushListener;
@@ -52,9 +52,6 @@ import org.apache.iotdb.db.metadata.utils.ResourceByPathUtils;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.DeleteDataNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.rescon.MemTableManager;
 import org.apache.iotdb.db.rescon.PrimitiveArrayManager;
@@ -215,66 +212,6 @@ public class TsFileProcessor {
     logger.info("reopen a tsfile processor {}", tsFileResource.getTsFile());
   }
 
-  /**
-   * insert data in an InsertRowPlan into the workingMemtable.
-   *
-   * @param insertRowPlan physical plan of insertion
-   */
-  public void insert(InsertRowPlan insertRowPlan) throws WriteProcessException {
-
-    if (workMemTable == null) {
-      createNewWorkingMemTable();
-    }
-
-    long[] memIncrements = null;
-    if (enableMemControl) {
-      if (insertRowPlan.isAligned()) {
-        memIncrements =
-            checkAlignedMemCostAndAddToTspInfo(
-                insertRowPlan.getDevicePath().getFullPath(), insertRowPlan.getMeasurements(),
-                insertRowPlan.getDataTypes(), insertRowPlan.getValues());
-      } else {
-        memIncrements =
-            checkMemCostAndAddToTspInfo(
-                insertRowPlan.getDevicePath().getFullPath(), insertRowPlan.getMeasurements(),
-                insertRowPlan.getDataTypes(), insertRowPlan.getValues());
-      }
-    }
-
-    try {
-      WALFlushListener walFlushListener = walNode.log(workMemTable.getMemTableId(), insertRowPlan);
-      if (walFlushListener.waitForResult() == WALFlushListener.Status.FAILURE) {
-        throw walFlushListener.getCause();
-      }
-    } catch (Exception e) {
-      if (enableMemControl && memIncrements != null) {
-        rollbackMemoryInfo(memIncrements);
-      }
-      throw new WriteProcessException(
-          String.format(
-              "%s: %s write WAL failed",
-              storageGroupName, tsFileResource.getTsFile().getAbsolutePath()),
-          e);
-    }
-
-    if (insertRowPlan.isAligned()) {
-      workMemTable.insertAlignedRow(insertRowPlan);
-    } else {
-      workMemTable.insert(insertRowPlan);
-    }
-
-    // update start time of this memtable
-    tsFileResource.updateStartTime(
-        insertRowPlan.getDeviceID().toStringID(), insertRowPlan.getTime());
-    // for sequence tsfile, we update the endTime only when the file is prepared to be closed.
-    // for unsequence tsfile, we have to update the endTime for each insertion.
-    if (!sequence) {
-      tsFileResource.updateEndTime(
-          insertRowPlan.getDeviceID().toStringID(), insertRowPlan.getTime());
-    }
-    tsFileResource.updatePlanIndexes(insertRowPlan.getIndex());
-  }
-
   /**
    * insert data in an InsertRowNode into the workingMemtable.
    *
@@ -335,98 +272,6 @@ public class TsFileProcessor {
     // tsFileResource.updatePlanIndexes(insertRowNode.getIndex());
   }
 
-  /**
-   * insert batch data of insertTabletPlan into the workingMemtable. The rows to be inserted are in
-   * the range [start, end). Null value in each column values will be replaced by the subsequent
-   * non-null value, e.g., {1, null, 3, null, 5} will be {1, 3, 5, null, 5}
-   *
-   * @param insertTabletPlan insert a tablet of a device
-   * @param start start index of rows to be inserted in insertTabletPlan
-   * @param end end index of rows to be inserted in insertTabletPlan
-   * @param results result array
-   */
-  public void insertTablet(
-      InsertTabletPlan insertTabletPlan, int start, int end, TSStatus[] results)
-      throws WriteProcessException {
-
-    if (workMemTable == null) {
-      createNewWorkingMemTable();
-    }
-
-    long[] memIncrements = null;
-    try {
-      if (enableMemControl) {
-        if (insertTabletPlan.isAligned()) {
-          memIncrements =
-              checkAlignedMemCostAndAddToTsp(
-                  insertTabletPlan.getDevicePath().getFullPath(),
-                  insertTabletPlan.getMeasurements(),
-                  insertTabletPlan.getDataTypes(),
-                  insertTabletPlan.getColumns(),
-                  start,
-                  end);
-        } else {
-          memIncrements =
-              checkMemCostAndAddToTspInfo(
-                  insertTabletPlan.getDevicePath().getFullPath(),
-                  insertTabletPlan.getMeasurements(),
-                  insertTabletPlan.getDataTypes(),
-                  insertTabletPlan.getColumns(),
-                  start,
-                  end);
-        }
-      }
-    } catch (WriteProcessException e) {
-      for (int i = start; i < end; i++) {
-        results[i] = RpcUtils.getStatus(TSStatusCode.WRITE_PROCESS_REJECT, e.getMessage());
-      }
-      throw new WriteProcessException(e);
-    }
-
-    try {
-      WALFlushListener walFlushListener =
-          walNode.log(workMemTable.getMemTableId(), insertTabletPlan, start, end);
-      if (walFlushListener.waitForResult() == WALFlushListener.Status.FAILURE) {
-        throw walFlushListener.getCause();
-      }
-    } catch (Exception e) {
-      for (int i = start; i < end; i++) {
-        results[i] = RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR, e.getMessage());
-      }
-      if (enableMemControl && memIncrements != null) {
-        rollbackMemoryInfo(memIncrements);
-      }
-      throw new WriteProcessException(e);
-    }
-
-    try {
-      if (insertTabletPlan.isAligned()) {
-        workMemTable.insertAlignedTablet(insertTabletPlan, start, end);
-      } else {
-        workMemTable.insertTablet(insertTabletPlan, start, end);
-      }
-    } catch (WriteProcessException e) {
-      for (int i = start; i < end; i++) {
-        results[i] = RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR, e.getMessage());
-      }
-      throw new WriteProcessException(e);
-    }
-
-    for (int i = start; i < end; i++) {
-      results[i] = RpcUtils.SUCCESS_STATUS;
-    }
-    tsFileResource.updateStartTime(
-        insertTabletPlan.getDeviceID().toStringID(), insertTabletPlan.getTimes()[start]);
-
-    // for sequence tsfile, we update the endTime only when the file is prepared to be closed.
-    // for unsequence tsfile, we have to update the endTime for each insertion.
-    if (!sequence) {
-      tsFileResource.updateEndTime(
-          insertTabletPlan.getDeviceID().toStringID(), insertTabletPlan.getTimes()[end - 1]);
-    }
-    tsFileResource.updatePlanIndexes(insertTabletPlan.getIndex());
-  }
-
   private void createNewWorkingMemTable() throws WriteProcessException {
     workMemTable = MemTableManager.getInstance().getAvailableMemTable(storageGroupName);
     walNode.onMemTableCreated(workMemTable, tsFileResource.getTsFilePath());
@@ -792,7 +637,7 @@ public class TsFileProcessor {
     if (dataRegionInfo.needToReportToSystem()) {
       try {
         if (!SystemInfo.getInstance().reportStorageGroupStatus(dataRegionInfo, this)) {
-          StorageEngine.blockInsertionIfReject(this);
+          StorageEngineV2.blockInsertionIfReject(this);
         }
       } catch (WriteProcessRejectException e) {
         dataRegionInfo.releaseStorageGroupMemCost(memTableIncrement);
@@ -855,10 +700,6 @@ public class TsFileProcessor {
     }
   }
 
-  WALFlushListener logDeleteInWAL(DeletePlan deletePlan) {
-    return walNode.log(workMemTable.getMemTableId(), deletePlan);
-  }
-
   WALFlushListener logDeleteDataNodeInWAL(DeleteDataNode deleteDataNode) {
     return walNode.log(workMemTable.getMemTableId(), deleteDataNode);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/dataregion/HashVirtualPartitioner.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/dataregion/HashVirtualPartitioner.java
deleted file mode 100644
index 66dd554bae..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/dataregion/HashVirtualPartitioner.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.engine.storagegroup.dataregion;
-
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-
-public class HashVirtualPartitioner implements VirtualPartitioner {
-
-  /** total number of virtual storage groups */
-  public static int STORAGE_GROUP_NUM =
-      IoTDBDescriptor.getInstance().getConfig().getDataRegionNum();
-
-  private HashVirtualPartitioner() {}
-
-  public static HashVirtualPartitioner getInstance() {
-    return HashVirtualPartitionerHolder.INSTANCE;
-  }
-
-  @Override
-  public int deviceToDataRegionId(PartialPath deviceId) {
-    return toStorageGroupId(deviceId);
-  }
-
-  @Override
-  public int getPartitionCount() {
-    return STORAGE_GROUP_NUM;
-  }
-
-  private int toStorageGroupId(PartialPath deviceId) {
-    return Math.abs(deviceId.getFullPath().hashCode() % STORAGE_GROUP_NUM);
-  }
-
-  private static class HashVirtualPartitionerHolder {
-
-    private static final HashVirtualPartitioner INSTANCE = new HashVirtualPartitioner();
-
-    private HashVirtualPartitionerHolder() {
-      // allowed to do nothing
-    }
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/dataregion/StorageGroupManager.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/dataregion/StorageGroupManager.java
deleted file mode 100644
index 91717b2e61..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/dataregion/StorageGroupManager.java
+++ /dev/null
@@ -1,505 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.engine.storagegroup.dataregion;
-
-import org.apache.iotdb.commons.concurrent.ThreadName;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.engine.storagegroup.DataRegion;
-import org.apache.iotdb.db.engine.storagegroup.DataRegion.TimePartitionFilter;
-import org.apache.iotdb.db.engine.storagegroup.TsFileProcessor;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.exception.DataRegionException;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.StorageGroupNotReadyException;
-import org.apache.iotdb.db.exception.TsFileProcessorException;
-import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
-import org.apache.iotdb.db.utils.ThreadUtils;
-import org.apache.iotdb.rpc.TSStatusCode;
-import org.apache.iotdb.tsfile.utils.Pair;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/** Each storage group that set by users corresponds to a StorageGroupManager */
-public class StorageGroupManager {
-
-  /** logger of this class */
-  private static final Logger logger = LoggerFactory.getLogger(StorageGroupManager.class);
-
-  /** virtual storage group partitioner */
-  VirtualPartitioner partitioner = HashVirtualPartitioner.getInstance();
-
-  /** all virtual storage group processor */
-  DataRegion[] dataRegion;
-
-  /**
-   * recover status of each virtual storage group processor, null if this logical storage group is
-   * new created
-   */
-  private final AtomicBoolean[] isDataRegionReady;
-
-  /** number of ready virtual storage group processors */
-  private AtomicInteger readyDataRegionNum;
-
-  private final AtomicBoolean isSettling = new AtomicBoolean();
-
-  /** value of root.stats."root.sg".TOTAL_POINTS */
-  private long monitorSeriesValue;
-
-  public StorageGroupManager() {
-    this(false);
-  }
-
-  public StorageGroupManager(boolean needRecovering) {
-    dataRegion = new DataRegion[partitioner.getPartitionCount()];
-    isDataRegionReady = new AtomicBoolean[partitioner.getPartitionCount()];
-    boolean recoverReady = !needRecovering;
-    for (int i = 0; i < partitioner.getPartitionCount(); i++) {
-      isDataRegionReady[i] = new AtomicBoolean(recoverReady);
-    }
-  }
-
-  /** push forceCloseAllWorkingTsFileProcessors down to all sg */
-  public void forceCloseAllWorkingTsFileProcessors() throws TsFileProcessorException {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        dataRegion.forceCloseAllWorkingTsFileProcessors();
-      }
-    }
-  }
-
-  /** push syncCloseAllWorkingTsFileProcessors down to all sg */
-  public void syncCloseAllWorkingTsFileProcessors() {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        dataRegion.syncCloseAllWorkingTsFileProcessors();
-      }
-    }
-  }
-
-  /** push check ttl down to all sg */
-  public void checkTTL() {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        dataRegion.checkFilesTTL();
-      }
-    }
-  }
-
-  /** push check sequence memtable flush interval down to all sg */
-  public void timedFlushSeqMemTable() {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        dataRegion.timedFlushSeqMemTable();
-      }
-    }
-  }
-
-  /** push check unsequence memtable flush interval down to all sg */
-  public void timedFlushUnseqMemTable() {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        dataRegion.timedFlushUnseqMemTable();
-      }
-    }
-  }
-
-  /**
-   * get processor from device id
-   *
-   * @param partialPath device path
-   * @return virtual storage group processor
-   */
-  public DataRegion getProcessor(PartialPath partialPath, IStorageGroupMNode storageGroupMNode)
-      throws DataRegionException, StorageEngineException {
-    int dataRegionId = partitioner.deviceToDataRegionId(partialPath);
-    return getProcessor(storageGroupMNode, dataRegionId);
-  }
-
-  /**
-   * get processor from data region id
-   *
-   * @param dataRegionId dataRegionId
-   * @return virtual storage group processor
-   */
-  public DataRegion getProcessor(int dataRegionId, IStorageGroupMNode storageGroupMNode)
-      throws DataRegionException, StorageEngineException {
-    return getProcessor(storageGroupMNode, dataRegionId);
-  }
-
-  @SuppressWarnings("java:S2445")
-  public DataRegion getProcessor(IStorageGroupMNode storageGroupMNode, int dataRegionId)
-      throws DataRegionException, StorageEngineException {
-    DataRegion processor = dataRegion[dataRegionId];
-    if (processor == null) {
-      // if finish recover
-      if (isDataRegionReady[dataRegionId].get()) {
-        // it's unsafe to synchronize MNode here because
-        // concurrent deletions and creations will create a new MNode
-        synchronized (isDataRegionReady[dataRegionId]) {
-          processor = dataRegion[dataRegionId];
-          if (processor == null) {
-            processor =
-                StorageEngine.getInstance()
-                    .buildNewStorageGroupProcessor(
-                        storageGroupMNode.getPartialPath(),
-                        storageGroupMNode,
-                        String.valueOf(dataRegionId));
-            dataRegion[dataRegionId] = processor;
-          }
-        }
-      } else {
-        // not finished recover, refuse the request
-        throw new StorageGroupNotReadyException(
-            storageGroupMNode.getFullPath(), TSStatusCode.STORAGE_GROUP_NOT_READY.getStatusCode());
-      }
-    }
-
-    return processor;
-  }
-
-  /**
-   * async recover all virtual storage groups in this logical storage group
-   *
-   * @param storageGroupMNode logical sg mnode
-   * @param pool thread pool to run virtual storage group recover task
-   * @param futures virtual storage group recover tasks
-   */
-  public void asyncRecover(
-      IStorageGroupMNode storageGroupMNode, ExecutorService pool, List<Future<Void>> futures) {
-    readyDataRegionNum = new AtomicInteger(0);
-    for (int i = 0; i < partitioner.getPartitionCount(); i++) {
-      int cur = i;
-      Callable<Void> recoverVsgTask =
-          () -> {
-            isDataRegionReady[cur].set(false);
-            DataRegion processor = null;
-            try {
-              processor =
-                  StorageEngine.getInstance()
-                      .buildNewStorageGroupProcessor(
-                          storageGroupMNode.getPartialPath(),
-                          storageGroupMNode,
-                          String.valueOf(cur));
-            } catch (DataRegionException e) {
-              logger.error(
-                  "Failed to recover virtual storage group {}[{}]",
-                  storageGroupMNode.getFullPath(),
-                  cur,
-                  e);
-            }
-
-            dataRegion[cur] = processor;
-            isDataRegionReady[cur].set(true);
-            logger.info(
-                "Storage Group {} has been recovered {}/{}",
-                storageGroupMNode.getFullPath(),
-                readyDataRegionNum.incrementAndGet(),
-                partitioner.getPartitionCount());
-            return null;
-          };
-      futures.add(pool.submit(recoverVsgTask));
-    }
-  }
-
-  public long getMonitorSeriesValue() {
-    return monitorSeriesValue;
-  }
-
-  public void setMonitorSeriesValue(long monitorSeriesValue) {
-    this.monitorSeriesValue = monitorSeriesValue;
-  }
-
-  public void updateMonitorSeriesValue(int successPointsNum) {
-    this.monitorSeriesValue += successPointsNum;
-  }
-
-  /** push closeStorageGroupProcessor operation down to all virtual storage group processors */
-  public void closeStorageGroupProcessor(boolean isSeq, boolean isSync) {
-    for (DataRegion processor : dataRegion) {
-      if (processor == null) {
-        continue;
-      }
-
-      if (logger.isInfoEnabled()) {
-        logger.info(
-            "{} closing sg processor is called for closing {}, seq = {}",
-            isSync ? "sync" : "async",
-            processor.getDataRegionId() + "-" + processor.getStorageGroupName(),
-            isSeq);
-      }
-
-      processor.writeLock("VirtualCloseStorageGroupProcessor-204");
-      try {
-        if (isSeq) {
-          // to avoid concurrent modification problem, we need a new array list
-          for (TsFileProcessor tsfileProcessor :
-              new ArrayList<>(processor.getWorkSequenceTsFileProcessors())) {
-            if (isSync) {
-              processor.syncCloseOneTsFileProcessor(true, tsfileProcessor);
-            } else {
-              processor.asyncCloseOneTsFileProcessor(true, tsfileProcessor);
-            }
-          }
-        } else {
-          // to avoid concurrent modification problem, we need a new array list
-          for (TsFileProcessor tsfileProcessor :
-              new ArrayList<>(processor.getWorkUnsequenceTsFileProcessors())) {
-            if (isSync) {
-              processor.syncCloseOneTsFileProcessor(false, tsfileProcessor);
-            } else {
-              processor.asyncCloseOneTsFileProcessor(false, tsfileProcessor);
-            }
-          }
-        }
-      } finally {
-        processor.writeUnlock();
-      }
-    }
-  }
-
-  /** push closeStorageGroupProcessor operation down to all virtual storage group processors */
-  public void closeStorageGroupProcessor(long partitionId, boolean isSeq, boolean isSync) {
-    for (DataRegion processor : dataRegion) {
-      if (processor != null) {
-        logger.info(
-            "async closing sg processor is called for closing {}, seq = {}, partitionId = {}",
-            processor.getDataRegionId() + "-" + processor.getStorageGroupName(),
-            isSeq,
-            partitionId);
-        processor.writeLock("VirtualCloseStorageGroupProcessor-242");
-        try {
-          // to avoid concurrent modification problem, we need a new array list
-          List<TsFileProcessor> processors =
-              isSeq
-                  ? new ArrayList<>(processor.getWorkSequenceTsFileProcessors())
-                  : new ArrayList<>(processor.getWorkUnsequenceTsFileProcessors());
-          for (TsFileProcessor tsfileProcessor : processors) {
-            if (tsfileProcessor.getTimeRangeId() == partitionId) {
-              if (isSync) {
-                processor.syncCloseOneTsFileProcessor(isSeq, tsfileProcessor);
-              } else {
-                processor.asyncCloseOneTsFileProcessor(isSeq, tsfileProcessor);
-              }
-              break;
-            }
-          }
-        } finally {
-          processor.writeUnlock();
-        }
-      }
-    }
-  }
-
-  /** push delete operation down to all virtual storage group processors */
-  public void delete(
-      PartialPath path,
-      long startTime,
-      long endTime,
-      long planIndex,
-      TimePartitionFilter timePartitionFilter)
-      throws IOException {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        dataRegion.delete(path, startTime, endTime, planIndex, timePartitionFilter);
-      }
-    }
-  }
-
-  /** push countUpgradeFiles operation down to all virtual storage group processors */
-  public int countUpgradeFiles() {
-    int totalUpgradeFileNum = 0;
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        totalUpgradeFileNum += dataRegion.countUpgradeFiles();
-      }
-    }
-
-    return totalUpgradeFileNum;
-  }
-
-  /** push upgradeAll operation down to all virtual storage group processors */
-  public void upgradeAll() {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        dataRegion.upgrade();
-      }
-    }
-  }
-
-  public void getResourcesToBeSettled(
-      List<TsFileResource> seqResourcesToBeSettled,
-      List<TsFileResource> unseqResourcesToBeSettled,
-      List<String> tsFilePaths) {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        dataRegion.addSettleFilesToList(
-            seqResourcesToBeSettled, unseqResourcesToBeSettled, tsFilePaths);
-      }
-    }
-  }
-
-  /** push mergeAll operation down to all virtual storage group processors */
-  public void mergeAll() {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        dataRegion.compact();
-      }
-    }
-  }
-
-  /** push syncDeleteDataFiles operation down to all virtual storage group processors */
-  public void syncDeleteDataFiles() {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        dataRegion.syncDeleteDataFiles();
-      }
-    }
-  }
-
-  /** push setTTL operation down to all virtual storage group processors */
-  public void setTTL(long dataTTL) {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        dataRegion.setDataTTLWithTimePrecisionCheck(dataTTL);
-      }
-    }
-  }
-
-  /** push deleteStorageGroup operation down to all virtual storage group processors */
-  public void deleteStorageGroupSystemFolder(String systemDir) {
-    for (DataRegion processor : dataRegion) {
-      if (processor != null) {
-        processor.deleteFolder(systemDir);
-      }
-    }
-  }
-
-  /** push getAllClosedStorageGroupTsFile operation down to all virtual storage group processors */
-  public void getAllClosedStorageGroupTsFile(
-      PartialPath storageGroupName, Map<PartialPath, Map<Long, List<TsFileResource>>> ret) {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        List<TsFileResource> allResources = dataRegion.getSequenceFileList();
-        allResources.addAll(dataRegion.getUnSequenceFileList());
-        for (TsFileResource tsfile : allResources) {
-          if (!tsfile.isClosed()) {
-            continue;
-          }
-          long partitionNum = tsfile.getTimePartition();
-          Map<Long, List<TsFileResource>> storageGroupFiles =
-              ret.computeIfAbsent(storageGroupName, n -> new HashMap<>());
-          storageGroupFiles.computeIfAbsent(partitionNum, n -> new ArrayList<>()).add(tsfile);
-        }
-      }
-    }
-  }
-
-  /** push setPartitionVersionToMax operation down to all virtual storage group processors */
-  public void setPartitionVersionToMax(long partitionId, long newMaxVersion) {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        dataRegion.setPartitionFileVersionToMax(partitionId, newMaxVersion);
-      }
-    }
-  }
-
-  /** push removePartitions operation down to all virtual storage group processors */
-  public void removePartitions(TimePartitionFilter filter) {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        dataRegion.removePartitions(filter);
-      }
-    }
-  }
-
-  /**
-   * push getWorkingStorageGroupPartitions operation down to all virtual storage group processors
-   */
-  public void getWorkingStorageGroupPartitions(
-      String storageGroupName, Map<String, List<Pair<Long, Boolean>>> res) {
-    for (DataRegion dataRegion : this.dataRegion) {
-      if (dataRegion != null) {
-        List<Pair<Long, Boolean>> partitionIdList = new ArrayList<>();
-        for (TsFileProcessor tsFileProcessor : dataRegion.getWorkSequenceTsFileProcessors()) {
-          Pair<Long, Boolean> tmpPair = new Pair<>(tsFileProcessor.getTimeRangeId(), true);
-          partitionIdList.add(tmpPair);
-        }
-
-        for (TsFileProcessor tsFileProcessor : dataRegion.getWorkUnsequenceTsFileProcessors()) {
-          Pair<Long, Boolean> tmpPair = new Pair<>(tsFileProcessor.getTimeRangeId(), false);
-          partitionIdList.add(tmpPair);
-        }
-
-        res.put(storageGroupName, partitionIdList);
-      }
-    }
-  }
-
-  /** only for test */
-  public void reset() {
-    Arrays.fill(dataRegion, null);
-  }
-
-  public void stopSchedulerPool() {
-    for (DataRegion vsg : this.dataRegion) {
-      if (vsg != null) {
-        ThreadUtils.stopThreadPool(
-            vsg.getTimedCompactionScheduleTask(), ThreadName.COMPACTION_SCHEDULE);
-      }
-    }
-  }
-
-  public void setSettling(boolean settling) {
-    isSettling.set(settling);
-  }
-
-  public void setAllowCompaction(boolean allowCompaction) {
-    for (DataRegion processor : dataRegion) {
-      if (processor != null) {
-        processor.setAllowCompaction(allowCompaction);
-      }
-    }
-  }
-
-  public void abortCompaction() {
-    for (DataRegion processor : dataRegion) {
-      if (processor != null) {
-        processor.abortCompaction();
-      }
-    }
-  }
-
-  public AtomicBoolean getIsSettling() {
-    return isSettling;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/dataregion/VirtualPartitioner.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/dataregion/VirtualPartitioner.java
deleted file mode 100644
index e1c66463e1..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/dataregion/VirtualPartitioner.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.engine.storagegroup.dataregion;
-
-import org.apache.iotdb.commons.path.PartialPath;
-
-public interface VirtualPartitioner {
-
-  /**
-   * use device id to determine storage group id
-   *
-   * @param deviceId device id
-   * @return data region id
-   */
-  int deviceToDataRegionId(PartialPath deviceId);
-
-  /**
-   * get total number of data region
-   *
-   * @return total number of data region
-   */
-  int getPartitionCount();
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java
index 41c354c50d..2c5ea3439b 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java
@@ -22,7 +22,6 @@ package org.apache.iotdb.db.engine.storagegroup.timeindex;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.utils.SerializeUtils;
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.StorageEngineV2;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.PartitionViolationException;
@@ -235,7 +234,8 @@ public class DeviceTimeIndex implements ITimeIndex {
   public long getTimePartition(String tsFilePath) {
     try {
       if (deviceToIndex != null && !deviceToIndex.isEmpty()) {
-        return StorageEngine.getTimePartition(startTimes[deviceToIndex.values().iterator().next()]);
+        return StorageEngineV2.getTimePartition(
+            startTimes[deviceToIndex.values().iterator().next()]);
       }
       String[] filePathSplits = FilePathUtils.splitTsFilePath(tsFilePath);
       return Long.parseLong(filePathSplits[filePathSplits.length - 2]);
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java
index ed7b6bf18e..dfa80b25a6 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java
@@ -20,7 +20,7 @@
 package org.apache.iotdb.db.engine.storagegroup.timeindex;
 
 import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.StorageEngineV2;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.PartitionViolationException;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
@@ -139,8 +139,8 @@ public class FileTimeIndex implements ITimeIndex {
   }
 
   private long getTimePartitionWithCheck() {
-    long startPartitionId = StorageEngine.getTimePartition(startTime);
-    long endPartitionId = StorageEngine.getTimePartition(endTime);
+    long startPartitionId = StorageEngineV2.getTimePartition(startTime);
+    long endPartitionId = StorageEngineV2.getTimePartition(endTime);
     if (startPartitionId == endPartitionId) {
       return startPartitionId;
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
index 069620f3f8..040250dc4c 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java
@@ -29,7 +29,6 @@ import org.apache.iotdb.commons.utils.FileUtils;
 import org.apache.iotdb.consensus.ConsensusFactory;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
 import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
 import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
@@ -902,9 +901,6 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
     try {
       PartialPath emptyStorageGroup = deleteOneTimeseriesUpdateStatisticsAndDropTrigger(p);
       if (!isRecovering) {
-        if (emptyStorageGroup != null) {
-          StorageEngine.getInstance().deleteAllDataFilesInOneStorageGroup(emptyStorageGroup);
-        }
         writeToMLog(SchemaRegionPlanFactory.getDeleteTimeSeriesPlan(Collections.singletonList(p)));
       }
     } catch (DeleteFailedException e) {
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
index a606f26f97..972a1e3e82 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java
@@ -27,7 +27,6 @@ import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.path.PathPatternTree;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.trigger.executor.TriggerEngine;
 import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
@@ -879,9 +878,6 @@ public class SchemaRegionSchemaFileImpl implements ISchemaRegion {
     try {
       PartialPath emptyStorageGroup = deleteOneTimeseriesUpdateStatisticsAndDropTrigger(p);
       if (!isRecovering) {
-        if (emptyStorageGroup != null) {
-          StorageEngine.getInstance().deleteAllDataFilesInOneStorageGroup(emptyStorageGroup);
-        }
         writeToMLog(SchemaRegionPlanFactory.getDeleteTimeSeriesPlan(Collections.singletonList(p)));
       }
     } catch (DeleteFailedException e) {
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/tag/TagManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/tag/TagManager.java
index a8029aed74..8c4a26e60e 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/tag/TagManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/tag/TagManager.java
@@ -23,17 +23,11 @@ import org.apache.iotdb.commons.file.SystemFileFactory;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.engine.storagegroup.DataRegion;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.MetadataConstant;
-import org.apache.iotdb.db.metadata.lastCache.LastCacheManager;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
 import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.tsfile.utils.Pair;
 
 import org.apache.commons.io.FileUtils;
@@ -232,53 +226,11 @@ public class TagManager {
         }
       }
     }
-
-    // if ordered by heat, we sort all the timeseries by the descending order of the last insert
-    // timestamp
-    if (plan.isOrderByHeat()) {
-      List<DataRegion> list;
-      try {
-        Pair<List<DataRegion>, Map<DataRegion, List<PartialPath>>>
-            lockListAndProcessorToSeriesMapPair =
-                StorageEngine.getInstance()
-                    .mergeLock(
-                        allMatchedNodes.stream()
-                            .map(IMeasurementMNode::getMeasurementPath)
-                            .collect(toList()));
-        list = lockListAndProcessorToSeriesMapPair.left;
-        Map<DataRegion, List<PartialPath>> processorToSeriesMap =
-            lockListAndProcessorToSeriesMapPair.right;
-
-        try {
-          // init QueryDataSource cache
-          QueryResourceManager.getInstance()
-              .initQueryDataSourceCache(processorToSeriesMap, context, null);
-        } catch (Exception e) {
-          logger.error("Meet error when init QueryDataSource ", e);
-          throw new QueryProcessException("Meet error when init QueryDataSource.", e);
-        } finally {
-          StorageEngine.getInstance().mergeUnLock(list);
-        }
-
-        allMatchedNodes =
-            allMatchedNodes.stream()
-                .sorted(
-                    Comparator.comparingLong(
-                            (IMeasurementMNode mNode) ->
-                                LastCacheManager.getLastTimeStamp(mNode, context))
-                        .reversed()
-                        .thenComparing(IMNode::getFullPath))
-                .collect(toList());
-      } catch (StorageEngineException | QueryProcessException e) {
-        throw new MetadataException(e);
-      }
-    } else {
-      // otherwise, we just sort them by the alphabetical order
-      allMatchedNodes =
-          allMatchedNodes.stream()
-              .sorted(Comparator.comparing(IMNode::getFullPath))
-              .collect(toList());
-    }
+    // we just sort them by the alphabetical order
+    allMatchedNodes =
+        allMatchedNodes.stream()
+            .sorted(Comparator.comparing(IMNode::getFullPath))
+            .collect(toList());
 
     return allMatchedNodes;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/write/InsertRowNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/write/InsertRowNode.java
index 8b544b5c17..b1e824c157 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/write/InsertRowNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/write/InsertRowNode.java
@@ -235,7 +235,7 @@ public class InsertRowNode extends InsertNode implements WALEntryValue {
    * Notice: measurementSchemas must be initialized before calling this method
    */
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  private void transferType() throws QueryProcessException {
+  public void transferType() throws QueryProcessException {
 
     for (int i = 0; i < measurementSchemas.length; i++) {
       // null when time series doesn't exist
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/write/InsertTabletNode.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/write/InsertTabletNode.java
index 29f563dd20..606ee5dd93 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/write/InsertTabletNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/node/write/InsertTabletNode.java
@@ -1010,7 +1010,7 @@ public class InsertTabletNode extends InsertNode implements WALEntryValue {
     int result = Objects.hash(super.hashCode(), rowCount, range);
     result = 31 * result + Arrays.hashCode(times);
     result = 31 * result + Arrays.hashCode(bitMaps);
-    result = 31 * result + Arrays.hashCode(columns);
+    result = 31 * result + Arrays.deepHashCode(columns);
     return result;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/IPlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/IPlanExecutor.java
index 3b3d299434..879e4ed94d 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/IPlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/IPlanExecutor.java
@@ -19,19 +19,10 @@
 package org.apache.iotdb.db.qp.executor;
 
 import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.engine.storagegroup.DataRegion.TimePartitionFilter;
-import org.apache.iotdb.db.exception.BatchProcessException;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertMultiTabletsPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowsOfOneDevicePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowsPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
@@ -61,74 +52,4 @@ public interface IPlanExecutor {
    */
   boolean processNonQuery(PhysicalPlan plan)
       throws QueryProcessException, StorageGroupNotSetException, StorageEngineException;
-
-  /**
-   * execute update command and return whether the operator is successful.
-   *
-   * @param path : update series seriesPath
-   * @param startTime start time in update command
-   * @param endTime end time in update command
-   * @param value - in type of string
-   */
-  void update(PartialPath path, long startTime, long endTime, String value)
-      throws QueryProcessException;
-
-  /**
-   * execute delete command and return whether the operator is successful.
-   *
-   * @param deletePlan physical delete plan
-   */
-  void delete(DeletePlan deletePlan) throws QueryProcessException;
-
-  /**
-   * execute delete command and return whether the operator is successful.
-   *
-   * @param path delete series seriesPath
-   * @param startTime start time in delete command
-   * @param endTime end time in delete command
-   * @param planIndex index of the deletion plan
-   * @param partitionFilter specify involving time partitions, if null, all partitions are involved
-   */
-  void delete(
-      PartialPath path,
-      long startTime,
-      long endTime,
-      long planIndex,
-      TimePartitionFilter partitionFilter)
-      throws QueryProcessException;
-
-  /**
-   * execute insert command and return whether the operator is successful.
-   *
-   * @param insertRowPlan physical insert plan
-   */
-  void insert(InsertRowPlan insertRowPlan) throws QueryProcessException;
-
-  /**
-   * execute insert command and return whether the operator is successful.
-   *
-   * @param insertRowsPlan physical insert rows plan, which contains multi insertRowPlans
-   */
-  void insert(InsertRowsPlan insertRowsPlan) throws QueryProcessException;
-
-  /**
-   * execute insert command and return whether the operator is successful.
-   *
-   * @param insertRowsOfOneDevicePlan physical insert plan
-   */
-  void insert(InsertRowsOfOneDevicePlan insertRowsOfOneDevicePlan) throws QueryProcessException;
-
-  /**
-   * execute batch insert plan
-   *
-   * @throws BatchProcessException when some of the rows failed
-   */
-  void insertTablet(InsertTabletPlan insertTabletPlan) throws QueryProcessException;
-
-  /**
-   * execute multi batch insert plan
-   *
-   * @throws QueryProcessException when some of the rows failed
-   */
-  void insertTablet(InsertMultiTabletsPlan insertMultiTabletsPlan) throws QueryProcessException;
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
index 344d44f41b..fa78332e35 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
@@ -18,21 +18,15 @@
  */
 package org.apache.iotdb.db.qp.executor;
 
-import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.common.rpc.thrift.TSchemaNode;
 import org.apache.iotdb.commons.auth.AuthException;
 import org.apache.iotdb.commons.auth.entity.PathPrivilege;
 import org.apache.iotdb.commons.auth.entity.PrivilegeType;
 import org.apache.iotdb.commons.auth.entity.Role;
 import org.apache.iotdb.commons.auth.entity.User;
-import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
-import org.apache.iotdb.commons.concurrent.ThreadName;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.commons.exception.sync.PipeException;
-import org.apache.iotdb.commons.exception.sync.PipeSinkException;
 import org.apache.iotdb.commons.path.MeasurementPath;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.sync.pipesink.PipeSink;
@@ -42,90 +36,34 @@ import org.apache.iotdb.commons.udf.service.UDFManagementService;
 import org.apache.iotdb.commons.utils.AuthUtils;
 import org.apache.iotdb.db.auth.AuthorityChecker;
 import org.apache.iotdb.db.auth.AuthorizerManager;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.cache.BloomFilterCache;
 import org.apache.iotdb.db.engine.cache.ChunkCache;
 import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
 import org.apache.iotdb.db.engine.flush.pool.FlushTaskPoolManager;
-import org.apache.iotdb.db.engine.storagegroup.DataRegion.TimePartitionFilter;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus;
 import org.apache.iotdb.db.engine.trigger.service.TriggerRegistrationService;
-import org.apache.iotdb.db.exception.BatchProcessException;
-import org.apache.iotdb.db.exception.QueryIdNotExsitException;
 import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.TriggerExecutionException;
-import org.apache.iotdb.db.exception.TriggerManagementException;
-import org.apache.iotdb.db.exception.WriteProcessException;
-import org.apache.iotdb.db.exception.metadata.PathNotExistException;
-import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
 import org.apache.iotdb.db.metadata.mnode.MNodeType;
-import org.apache.iotdb.db.metadata.utils.MetaUtils;
-import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.qp.logical.sys.AuthorOperator;
 import org.apache.iotdb.db.qp.logical.sys.AuthorOperator.AuthorType;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
 import org.apache.iotdb.db.qp.physical.crud.AlignByDevicePlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePartitionPlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
 import org.apache.iotdb.db.qp.physical.crud.FillQueryPlan;
 import org.apache.iotdb.db.qp.physical.crud.GroupByTimeFillPlan;
 import org.apache.iotdb.db.qp.physical.crud.GroupByTimePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertMultiTabletsPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowsOfOneDevicePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowsPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.qp.physical.crud.LastQueryPlan;
 import org.apache.iotdb.db.qp.physical.crud.QueryIndexPlan;
 import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
 import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
 import org.apache.iotdb.db.qp.physical.crud.UDAFPlan;
 import org.apache.iotdb.db.qp.physical.crud.UDTFPlan;
-import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
-import org.apache.iotdb.db.qp.physical.sys.AlterTimeSeriesPlan;
-import org.apache.iotdb.db.qp.physical.sys.AppendTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
 import org.apache.iotdb.db.qp.physical.sys.CountPlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateContinuousQueryPlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateFunctionPlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateMultiTimeSeriesPlan;
-import org.apache.iotdb.db.qp.physical.sys.CreatePipePlan;
-import org.apache.iotdb.db.qp.physical.sys.CreatePipeSinkPlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateTemplatePlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateTriggerPlan;
-import org.apache.iotdb.db.qp.physical.sys.DataAuthPlan;
-import org.apache.iotdb.db.qp.physical.sys.DeleteStorageGroupPlan;
-import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
-import org.apache.iotdb.db.qp.physical.sys.DropContinuousQueryPlan;
-import org.apache.iotdb.db.qp.physical.sys.DropFunctionPlan;
-import org.apache.iotdb.db.qp.physical.sys.DropPipeSinkPlan;
-import org.apache.iotdb.db.qp.physical.sys.DropTemplatePlan;
-import org.apache.iotdb.db.qp.physical.sys.DropTriggerPlan;
-import org.apache.iotdb.db.qp.physical.sys.FlushPlan;
-import org.apache.iotdb.db.qp.physical.sys.KillQueryPlan;
-import org.apache.iotdb.db.qp.physical.sys.LoadConfigurationPlan;
-import org.apache.iotdb.db.qp.physical.sys.OperateFilePlan;
-import org.apache.iotdb.db.qp.physical.sys.OperatePipePlan;
-import org.apache.iotdb.db.qp.physical.sys.PruneTemplatePlan;
-import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
-import org.apache.iotdb.db.qp.physical.sys.SetSystemModePlan;
-import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
-import org.apache.iotdb.db.qp.physical.sys.SetTemplatePlan;
-import org.apache.iotdb.db.qp.physical.sys.SettlePlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowChildNodesPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowChildPathsPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
-import org.apache.iotdb.db.qp.physical.sys.ShowLockInfoPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowNodesInTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowPathsSetTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowPathsUsingTemplatePlan;
@@ -135,9 +73,6 @@ import org.apache.iotdb.db.qp.physical.sys.ShowPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTTLPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
-import org.apache.iotdb.db.qp.physical.sys.StartTriggerPlan;
-import org.apache.iotdb.db.qp.physical.sys.StopTriggerPlan;
-import org.apache.iotdb.db.qp.physical.sys.UnsetTemplatePlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.FileReaderManager;
 import org.apache.iotdb.db.query.control.QueryResourceManager;
@@ -150,42 +85,21 @@ import org.apache.iotdb.db.query.dataset.SingleDataSet;
 import org.apache.iotdb.db.query.executor.IQueryRouter;
 import org.apache.iotdb.db.query.executor.QueryRouter;
 import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.service.SettleService;
 import org.apache.iotdb.db.sync.SyncService;
-import org.apache.iotdb.db.tools.TsFileSplitByPartitionTool;
-import org.apache.iotdb.db.utils.FileLoaderUtils;
-import org.apache.iotdb.db.utils.TypeInferenceUtils;
-import org.apache.iotdb.db.utils.UpgradeUtils;
-import org.apache.iotdb.db.utils.sync.SyncPipeUtil;
-import org.apache.iotdb.db.wal.WALManager;
-import org.apache.iotdb.rpc.RpcUtils;
-import org.apache.iotdb.rpc.TSStatusCode;
-import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
-import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetadata;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
-import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
-import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.read.TsFileCheckStatus;
-import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.common.Field;
-import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.common.RowRecord;
 import org.apache.iotdb.tsfile.read.query.dataset.EmptyDataSet;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
-import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
-import org.apache.iotdb.udf.api.exception.UDFManagementException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.util.ArrayList;
@@ -194,14 +108,12 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Date;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeSet;
-import java.util.concurrent.Future;
 import java.util.concurrent.ThreadPoolExecutor;
 
 import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_CHILD_NODES;
@@ -214,7 +126,6 @@ import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_FUNCTION_CLASS;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_FUNCTION_NAME;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_FUNCTION_TYPE;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_ITEM;
-import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_LOCK_INFO;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PIPESINK_ATTRIBUTES;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PIPESINK_NAME;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.COLUMN_PIPESINK_TYPE;
@@ -239,11 +150,8 @@ import static org.apache.iotdb.commons.conf.IoTDBConstant.FUNCTION_TYPE_BUILTIN_
 import static org.apache.iotdb.commons.conf.IoTDBConstant.FUNCTION_TYPE_EXTERNAL_UDAF;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.FUNCTION_TYPE_EXTERNAL_UDTF;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.FUNCTION_TYPE_NATIVE;
-import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.QUERY_ID;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.STATEMENT;
-import static org.apache.iotdb.rpc.TSStatusCode.INTERNAL_SERVER_ERROR;
-import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
 
 public class PlanExecutor implements IPlanExecutor {
 
@@ -283,355 +191,15 @@ public class PlanExecutor implements IPlanExecutor {
   @Override
   public boolean processNonQuery(PhysicalPlan plan)
       throws QueryProcessException, StorageGroupNotSetException, StorageEngineException {
-    switch (plan.getOperatorType()) {
-      case DELETE:
-        delete((DeletePlan) plan);
-        return true;
-      case INSERT:
-        insert((InsertRowPlan) plan);
-        return true;
-      case BATCH_INSERT_ONE_DEVICE:
-        insert((InsertRowsOfOneDevicePlan) plan);
-        return true;
-      case BATCH_INSERT_ROWS:
-        insert((InsertRowsPlan) plan);
-        return true;
-      case BATCH_INSERT:
-        insertTablet((InsertTabletPlan) plan);
-        return true;
-      case MULTI_BATCH_INSERT:
-        insertTablet((InsertMultiTabletsPlan) plan);
-        return true;
-      case CREATE_ROLE:
-      case DELETE_ROLE:
-      case CREATE_USER:
-      case REVOKE_USER_ROLE:
-      case REVOKE_ROLE_PRIVILEGE:
-      case REVOKE_USER_PRIVILEGE:
-      case GRANT_ROLE_PRIVILEGE:
-      case GRANT_USER_PRIVILEGE:
-      case GRANT_USER_ROLE:
-      case MODIFY_PASSWORD:
-      case DELETE_USER:
-        AuthorPlan author = (AuthorPlan) plan;
-        return operateAuthor(author);
-      case GRANT_WATERMARK_EMBEDDING:
-        return operateWatermarkEmbedding(((DataAuthPlan) plan).getUsers(), true);
-      case REVOKE_WATERMARK_EMBEDDING:
-        return operateWatermarkEmbedding(((DataAuthPlan) plan).getUsers(), false);
-      case DELETE_TIMESERIES:
-        return deleteTimeSeries((DeleteTimeSeriesPlan) plan);
-      case CREATE_TIMESERIES:
-        return createTimeSeries((CreateTimeSeriesPlan) plan);
-      case CREATE_ALIGNED_TIMESERIES:
-        return createAlignedTimeSeries((CreateAlignedTimeSeriesPlan) plan);
-      case CREATE_MULTI_TIMESERIES:
-        return createMultiTimeSeries((CreateMultiTimeSeriesPlan) plan);
-      case ALTER_TIMESERIES:
-        return alterTimeSeries((AlterTimeSeriesPlan) plan);
-      case SET_STORAGE_GROUP:
-        return setStorageGroup((SetStorageGroupPlan) plan);
-      case DELETE_STORAGE_GROUP:
-        return deleteStorageGroups((DeleteStorageGroupPlan) plan);
-      case TTL:
-        operateTTL((SetTTLPlan) plan);
-        return true;
-      case LOAD_CONFIGURATION:
-        loadConfiguration((LoadConfigurationPlan) plan);
-        return true;
-      case LOAD_FILES:
-        operateLoadFiles((OperateFilePlan) plan);
-        return true;
-      case REMOVE_FILE:
-        operateRemoveFile((OperateFilePlan) plan);
-        return true;
-      case UNLOAD_FILE:
-        operateUnloadFile((OperateFilePlan) plan);
-        return true;
-      case FLUSH:
-        operateFlush((FlushPlan) plan);
-        return true;
-      case MERGE:
-      case FULL_MERGE:
-        operateMerge();
-        return true;
-      case SET_SYSTEM_MODE:
-        operateSetSystemMode((SetSystemModePlan) plan);
-        return true;
-      case CLEAR_CACHE:
-        operateClearCache();
-        return true;
-      case DELETE_PARTITION:
-        DeletePartitionPlan p = (DeletePartitionPlan) plan;
-        TimePartitionFilter filter =
-            (storageGroupName, partitionId) ->
-                storageGroupName.equals(
-                        ((DeletePartitionPlan) plan).getStorageGroupName().getFullPath())
-                    && p.getPartitionId().contains(partitionId);
-        StorageEngine.getInstance()
-            .removePartitions(((DeletePartitionPlan) plan).getStorageGroupName(), filter);
-        return true;
-      case CREATE_FUNCTION:
-        return operateCreateFunction((CreateFunctionPlan) plan);
-      case DROP_FUNCTION:
-        return operateDropFunction((DropFunctionPlan) plan);
-      case CREATE_TRIGGER:
-        return operateCreateTrigger((CreateTriggerPlan) plan);
-      case DROP_TRIGGER:
-        return operateDropTrigger((DropTriggerPlan) plan);
-      case START_TRIGGER:
-        return operateStartTrigger((StartTriggerPlan) plan);
-      case STOP_TRIGGER:
-        return operateStopTrigger((StopTriggerPlan) plan);
-      case KILL:
-        try {
-          operateKillQuery((KillQueryPlan) plan);
-        } catch (QueryIdNotExsitException e) {
-          throw new QueryProcessException(e.getMessage());
-        }
-        return true;
-      case CREATE_TEMPLATE:
-        return createTemplate((CreateTemplatePlan) plan);
-      case DROP_TEMPLATE:
-        return dropTemplate((DropTemplatePlan) plan);
-      case APPEND_TEMPLATE:
-        return appendTemplate((AppendTemplatePlan) plan);
-      case PRUNE_TEMPLATE:
-        return pruneTemplate((PruneTemplatePlan) plan);
-      case SET_TEMPLATE:
-        return setTemplate((SetTemplatePlan) plan);
-      case ACTIVATE_TEMPLATE:
-        return activateTemplate((ActivateTemplatePlan) plan);
-      case UNSET_TEMPLATE:
-        return unsetTemplate((UnsetTemplatePlan) plan);
-      case CREATE_CONTINUOUS_QUERY:
-        return operateCreateContinuousQuery((CreateContinuousQueryPlan) plan);
-      case DROP_CONTINUOUS_QUERY:
-        return operateDropContinuousQuery((DropContinuousQueryPlan) plan);
-      case SETTLE:
-        settle((SettlePlan) plan);
-        return true;
-      case SHOW_QUERY_RESOURCE:
-        return processShowQueryResource();
-      case CREATE_PIPESINK:
-        createPipeSink((CreatePipeSinkPlan) plan);
-        return true;
-      case DROP_PIPESINK:
-        dropPipeSink((DropPipeSinkPlan) plan);
-        return true;
-      case CREATE_PIPE:
-        createPipe((CreatePipePlan) plan);
-        return true;
-      case STOP_PIPE:
-      case START_PIPE:
-      case DROP_PIPE:
-        operatePipe((OperatePipePlan) plan);
-        return true;
-      default:
-        throw new UnsupportedOperationException(
-            String.format("operation %s is not supported", plan.getOperatorName()));
-    }
-  }
-
-  private boolean createTemplate(CreateTemplatePlan createTemplatePlan)
-      throws QueryProcessException {
-    try {
-      IoTDB.schemaProcessor.createSchemaTemplate(createTemplatePlan);
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    }
-    return true;
-  }
-
-  private boolean dropTemplate(DropTemplatePlan dropTemplatePlan) throws QueryProcessException {
-    try {
-      IoTDB.schemaProcessor.dropSchemaTemplate(dropTemplatePlan);
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    }
-    return true;
-  }
-
-  private boolean appendTemplate(AppendTemplatePlan plan) throws QueryProcessException {
-    try {
-      IoTDB.schemaProcessor.appendSchemaTemplate(plan);
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    }
-    return true;
-  }
-
-  private boolean pruneTemplate(PruneTemplatePlan plan) throws QueryProcessException {
-    try {
-      IoTDB.schemaProcessor.pruneSchemaTemplate(plan);
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    }
-    return true;
-  }
-
-  private boolean setTemplate(SetTemplatePlan setTemplatePlan) throws QueryProcessException {
-    try {
-      IoTDB.schemaProcessor.setSchemaTemplate(setTemplatePlan);
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    }
-    return true;
-  }
-
-  private boolean activateTemplate(ActivateTemplatePlan activateTemplatePlan)
-      throws QueryProcessException {
-    try {
-      IoTDB.schemaProcessor.setUsingSchemaTemplate(activateTemplatePlan);
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    }
-    return true;
-  }
-
-  private boolean unsetTemplate(UnsetTemplatePlan unsetTemplatePlan) throws QueryProcessException {
-    try {
-      IoTDB.schemaProcessor.unsetSchemaTemplate(unsetTemplatePlan);
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    }
-    return true;
-  }
-
-  private boolean operateCreateFunction(CreateFunctionPlan plan) throws UDFManagementException {
-    try {
-      UDFManagementService.getInstance()
-          .register(new UDFInformation(plan.getUdfName(), plan.getClassName()));
-    } catch (Exception e) {
-      throw new UDFManagementException(e.getMessage());
-    }
-    return true;
-  }
-
-  private boolean operateDropFunction(DropFunctionPlan plan) throws UDFManagementException {
-    try {
-      UDFManagementService.getInstance().deregister(plan.getUdfName(), false);
-    } catch (Exception e) {
-      throw new UDFManagementException(e.getMessage());
-    }
-    return true;
-  }
-
-  private boolean operateCreateTrigger(CreateTriggerPlan plan)
-      throws TriggerManagementException, TriggerExecutionException {
-    TriggerRegistrationService.getInstance().register(plan);
-    return true;
-  }
-
-  private boolean operateDropTrigger(DropTriggerPlan plan) throws TriggerManagementException {
-    TriggerRegistrationService.getInstance().deregister(plan);
     return true;
   }
 
-  private boolean operateStartTrigger(StartTriggerPlan plan)
-      throws TriggerManagementException, TriggerExecutionException {
-    TriggerRegistrationService.getInstance().activate(plan);
-    return true;
-  }
-
-  private boolean operateStopTrigger(StopTriggerPlan plan) throws TriggerManagementException {
-    TriggerRegistrationService.getInstance().inactivate(plan);
-    return true;
-  }
-
-  private void operateMerge() throws StorageEngineException {
-    StorageEngine.getInstance().mergeAll();
-  }
-
   public static void operateClearCache() {
     ChunkCache.getInstance().clear();
     TimeSeriesMetadataCache.getInstance().clear();
     BloomFilterCache.getInstance().clear();
   }
 
-  private void operateKillQuery(KillQueryPlan killQueryPlan) throws QueryIdNotExsitException {
-    QueryTimeManager queryTimeManager = QueryTimeManager.getInstance();
-    long killQueryId = killQueryPlan.getQueryId();
-    if (killQueryId != -1) {
-      if (queryTimeManager.getQueryContextMap().get(killQueryId) != null) {
-        queryTimeManager.killQuery(killQueryId);
-      } else {
-        throw new QueryIdNotExsitException(
-            String.format(
-                "Query Id %d is not exist, please check it.", killQueryPlan.getQueryId()));
-      }
-    } else {
-      // if queryId is not specified, kill all running queries
-      if (!queryTimeManager.getQueryContextMap().isEmpty()) {
-        synchronized (queryTimeManager.getQueryContextMap()) {
-          List<Long> queryIdList = new ArrayList<>(queryTimeManager.getQueryContextMap().keySet());
-          for (Long queryId : queryIdList) {
-            queryTimeManager.killQuery(queryId);
-          }
-        }
-      }
-    }
-  }
-
-  private void operateSetSystemMode(SetSystemModePlan plan) {
-    CommonDescriptor.getInstance().getConfig().setNodeStatus(plan.getStatus());
-  }
-
-  private void operateFlush(FlushPlan plan) throws StorageGroupNotSetException {
-    if (plan.getPaths().isEmpty()) {
-      StorageEngine.getInstance().syncCloseAllProcessor();
-      WALManager.getInstance().deleteOutdatedWALFiles();
-    } else {
-      flushSpecifiedStorageGroups(plan);
-    }
-
-    if (!plan.getPaths().isEmpty()) {
-      List<PartialPath> noExistSg = checkStorageGroupExist(plan.getPaths());
-      if (!noExistSg.isEmpty()) {
-        StringBuilder sb = new StringBuilder();
-        noExistSg.forEach(storageGroup -> sb.append(storageGroup.getFullPath()).append(","));
-        throw new StorageGroupNotSetException(sb.subSequence(0, sb.length() - 1).toString(), true);
-      }
-    }
-  }
-
-  private boolean operateCreateContinuousQuery(CreateContinuousQueryPlan plan) {
-    throw new UnsupportedOperationException();
-  }
-
-  private boolean operateDropContinuousQuery(DropContinuousQueryPlan plan) {
-    throw new UnsupportedOperationException();
-  }
-
-  public static void flushSpecifiedStorageGroups(FlushPlan plan)
-      throws StorageGroupNotSetException {
-    Map<PartialPath, List<Pair<Long, Boolean>>> storageGroupMap =
-        plan.getStorageGroupPartitionIds();
-    for (Entry<PartialPath, List<Pair<Long, Boolean>>> entry : storageGroupMap.entrySet()) {
-      PartialPath storageGroupName = entry.getKey();
-      // normal flush
-      if (entry.getValue() == null) {
-        if (plan.isSeq() == null) {
-          StorageEngine.getInstance()
-              .closeStorageGroupProcessor(storageGroupName, true, plan.isSync());
-          StorageEngine.getInstance()
-              .closeStorageGroupProcessor(storageGroupName, false, plan.isSync());
-        } else {
-          StorageEngine.getInstance()
-              .closeStorageGroupProcessor(storageGroupName, plan.isSeq(), plan.isSync());
-        }
-      }
-      // partition specified flush, for snapshot flush plan
-      else {
-        List<Pair<Long, Boolean>> partitionIdSequencePairs = entry.getValue();
-        for (Pair<Long, Boolean> pair : partitionIdSequencePairs) {
-          StorageEngine.getInstance()
-              .closeStorageGroupProcessor(storageGroupName, pair.left, pair.right, true);
-        }
-      }
-    }
-  }
-
   protected QueryDataSet processDataQuery(QueryPlan queryPlan, QueryContext context)
       throws StorageEngineException, QueryFilterOptimizationException, QueryProcessException,
           IOException, InterruptedException {
@@ -695,8 +263,6 @@ public class PlanExecutor implements IPlanExecutor {
         return processShowTimeseries((ShowTimeSeriesPlan) showPlan, context);
       case STORAGE_GROUP:
         return processShowStorageGroup((ShowStorageGroupPlan) showPlan);
-      case LOCK_INFO:
-        return processShowLockInfo((ShowLockInfoPlan) showPlan);
       case DEVICES:
         return processShowDevices((ShowDevicesPlan) showPlan);
       case CHILD_PATH:
@@ -940,25 +506,6 @@ public class PlanExecutor implements IPlanExecutor {
     }
   }
 
-  private QueryDataSet processShowLockInfo(ShowLockInfoPlan showLockInfoPlan)
-      throws MetadataException {
-    ListDataSet listDataSet =
-        new ListDataSet(
-            Arrays.asList(
-                new PartialPath(COLUMN_STORAGE_GROUP, false),
-                new PartialPath(COLUMN_LOCK_INFO, false)),
-            Arrays.asList(TSDataType.TEXT, TSDataType.TEXT));
-    try {
-      List<PartialPath> storageGroupList =
-          getStorageGroupNames(showLockInfoPlan.getPath(), showLockInfoPlan.isPrefixMatch());
-      List<String> lockHolderList = StorageEngine.getInstance().getLockInfo(storageGroupList);
-      addLockInfoToDataSet(storageGroupList, lockHolderList, listDataSet);
-    } catch (StorageEngineException e) {
-      throw new MetadataException(e);
-    }
-    return listDataSet;
-  }
-
   private void addLockInfoToDataSet(
       List<PartialPath> paths, List<String> lockHolderList, ListDataSet dataSet) {
     for (int i = 0; i < paths.size(); i++) {
@@ -1277,821 +824,7 @@ public class PlanExecutor implements IPlanExecutor {
     return listDataSet;
   }
 
-  @Override
-  public void delete(DeletePlan deletePlan) throws QueryProcessException {
-    AUDIT_LOGGER.info(
-        "delete data from {} in [{},{}]",
-        deletePlan.getPaths(),
-        deletePlan.getDeleteStartTime(),
-        deletePlan.getDeleteEndTime());
-    for (PartialPath path : deletePlan.getPaths()) {
-      delete(
-          path,
-          deletePlan.getDeleteStartTime(),
-          deletePlan.getDeleteEndTime(),
-          deletePlan.getIndex(),
-          deletePlan.getPartitionFilter());
-    }
-  }
-
-  private void operateLoadFiles(OperateFilePlan plan) throws QueryProcessException {
-    File file = plan.getFile();
-    if (!file.exists()) {
-      throw new QueryProcessException(
-          String.format("File path '%s' doesn't exists.", file.getPath()));
-    }
-
-    List<File> tsFiles = new ArrayList<>();
-    findAllTsFiles(file, tsFiles);
-    tsFiles.sort(
-        (o1, o2) -> {
-          String file1Name = o1.getName();
-          String file2Name = o2.getName();
-          try {
-            return TsFileResource.checkAndCompareFileName(file1Name, file2Name);
-          } catch (IOException e) {
-            return file1Name.compareTo(file2Name);
-          }
-        });
-    for (File tsFile : tsFiles) {
-      loadFile(tsFile, plan);
-    }
-  }
-
-  private void findAllTsFiles(File curFile, List<File> files) {
-    if (curFile.isFile()) {
-      if (curFile.getName().endsWith(TSFILE_SUFFIX)) {
-        files.add(curFile);
-      }
-    } else {
-      for (File file : curFile.listFiles()) {
-        findAllTsFiles(file, files);
-      }
-    }
-  }
-
-  private void loadFile(File file, OperateFilePlan plan) throws QueryProcessException {
-    if (!file.getName().endsWith(TSFILE_SUFFIX)) {
-      return;
-    }
-    TsFileResource tsFileResource = new TsFileResource(file);
-    tsFileResource.setStatus(TsFileResourceStatus.CLOSED);
-    try {
-      // check file
-      RestorableTsFileIOWriter restorableTsFileIOWriter = new RestorableTsFileIOWriter(file);
-      if (restorableTsFileIOWriter.hasCrashed()) {
-        restorableTsFileIOWriter.close();
-        throw new QueryProcessException(
-            String.format(
-                "Cannot load file %s because the file has crashed.", file.getAbsolutePath()));
-      }
-      Map<Path, IMeasurementSchema> schemaMap = new HashMap<>();
-
-      List<ChunkGroupMetadata> chunkGroupMetadataList = new ArrayList<>();
-      try (TsFileSequenceReader reader = new TsFileSequenceReader(file.getAbsolutePath(), false)) {
-        if (reader.selfCheck(schemaMap, chunkGroupMetadataList, false)
-            != TsFileCheckStatus.COMPLETE_FILE) {
-          throw new QueryProcessException(
-              String.format(
-                  "Cannot load file %s because the file has crashed.", file.getAbsolutePath()));
-        }
-        if (plan.getVerifyMetadata()) {
-          loadNewTsFileVerifyMetadata(reader);
-        }
-      } catch (IOException e) {
-        logger.warn("can not get timeseries metadata from {}.", file.getAbsoluteFile());
-        throw new QueryProcessException(e.getMessage());
-      }
-
-      FileLoaderUtils.loadOrGenerateResource(tsFileResource);
-      if (UpgradeUtils.isNeedUpgrade(tsFileResource)) {
-        throw new QueryProcessException(
-            String.format(
-                "Cannot load file %s because the file's version is old which needs to be upgraded.",
-                file.getAbsolutePath()));
-      }
-
-      // create schemas if they doesn't exist
-      if (plan.isAutoCreateSchema()) {
-        createSchemaAutomatically(chunkGroupMetadataList, schemaMap, plan.getSgLevel());
-      }
-
-      List<TsFileResource> splitResources = new ArrayList();
-      if (tsFileResource.isSpanMultiTimePartitions()) {
-        logger.info(
-            "try to split the tsFile={} du to it spans multi partitions",
-            tsFileResource.getTsFile().getPath());
-        TsFileSplitByPartitionTool.rewriteTsFile(tsFileResource, splitResources);
-        tsFileResource.writeLock();
-        tsFileResource.removeModFile();
-        tsFileResource.writeUnlock();
-        logger.info(
-            "after split, the old tsFile was split to {} new tsFiles", splitResources.size());
-      }
-
-      if (splitResources.isEmpty()) {
-        splitResources.add(tsFileResource);
-      }
-
-      for (TsFileResource resource : splitResources) {
-        StorageEngine.getInstance().loadNewTsFile(resource, plan.isDeleteAfterLoad());
-      }
-    } catch (Exception e) {
-      logger.error("fail to load file {}", file.getName(), e);
-      throw new QueryProcessException(
-          String.format("Cannot load file %s because %s", file.getAbsolutePath(), e.getMessage()));
-    }
-  }
-
-  private void loadNewTsFileVerifyMetadata(TsFileSequenceReader tsFileSequenceReader)
-      throws MetadataException, QueryProcessException, IOException {
-    Map<String, List<TimeseriesMetadata>> metadataSet =
-        tsFileSequenceReader.getAllTimeseriesMetadata(false);
-    for (Map.Entry<String, List<TimeseriesMetadata>> entry : metadataSet.entrySet()) {
-      String deviceId = entry.getKey();
-      PartialPath devicePath = new PartialPath(deviceId);
-      if (!IoTDB.schemaProcessor.isPathExist(devicePath)) {
-        continue;
-      }
-      boolean isAlignedInIoTDB =
-          IoTDB.schemaProcessor.getDeviceNode(devicePath).getAsEntityMNode().isAligned();
-      boolean isAlignedInTsFile = false;
-      for (TimeseriesMetadata metadata : entry.getValue()) {
-        if ("".equals(metadata.getMeasurementId())) {
-          isAlignedInTsFile = true;
-          continue;
-        }
-        PartialPath fullPath =
-            new PartialPath(deviceId + TsFileConstant.PATH_SEPARATOR + metadata.getMeasurementId());
-        if (IoTDB.schemaProcessor.isPathExist(fullPath)) {
-          TSDataType dataType = IoTDB.schemaProcessor.getSeriesType(fullPath);
-          if (dataType != metadata.getTSDataType()) {
-            throw new QueryProcessException(
-                String.format(
-                    "%s is %s in the loading TsFile but is %s in IoTDB.",
-                    fullPath.getFullPath(), metadata.getTSDataType().name(), dataType.name()));
-          }
-        }
-      }
-      if (isAlignedInIoTDB != isAlignedInTsFile) {
-        throw new QueryProcessException(
-            String.format(
-                "%s is%s aligned in the loading TsFile but is%s aligned in IoTDB.",
-                devicePath.getFullPath(),
-                isAlignedInTsFile ? "" : " not",
-                isAlignedInIoTDB ? "" : " not"));
-      }
-    }
-  }
-
-  // Suppress high Cognitive Complexity warning
-  private void createSchemaAutomatically(
-      List<ChunkGroupMetadata> chunkGroupMetadataList,
-      Map<Path, IMeasurementSchema> knownSchemas,
-      int sgLevel)
-      throws MetadataException {
-    if (chunkGroupMetadataList.isEmpty()) {
-      return;
-    }
-
-    Set<PartialPath> registeredSeries = new HashSet<>();
-    for (ChunkGroupMetadata chunkGroupMetadata : chunkGroupMetadataList) {
-      String device = chunkGroupMetadata.getDevice();
-      boolean isAligned = false;
-      Set<String> existSeriesSet = new HashSet<>();
-      List<PartialPath> needRegisterSeries = new ArrayList<>();
-      List<IMeasurementSchema> needRegisterSchema = new ArrayList<>();
-      PartialPath devicePath = new PartialPath(device);
-      PartialPath storageGroupPath = MetaUtils.getStorageGroupPathByLevel(devicePath, sgLevel);
-      try {
-        IoTDB.schemaProcessor.setStorageGroup(storageGroupPath);
-      } catch (StorageGroupAlreadySetException alreadySetException) {
-        if (!alreadySetException.getStorageGroupPath().equals(storageGroupPath.getFullPath())) {
-          throw alreadySetException;
-        }
-      }
-      for (PartialPath path : IoTDB.schemaProcessor.getMeasurementPaths(devicePath, true)) {
-        existSeriesSet.add(path.getMeasurement());
-        existSeriesSet.add(path.getMeasurementAlias());
-      }
-      for (ChunkMetadata chunkMetadata : chunkGroupMetadata.getChunkMetadataList()) {
-        if ("".equals(chunkMetadata.getMeasurementUid())) {
-          isAligned = true;
-          continue;
-        }
-        PartialPath series =
-            new PartialPath(
-                chunkGroupMetadata.getDevice()
-                    + TsFileConstant.PATH_SEPARATOR
-                    + chunkMetadata.getMeasurementUid());
-        if (!registeredSeries.contains(series)) {
-          registeredSeries.add(series);
-          IMeasurementSchema schema =
-              knownSchemas.get(new Path(series.getDevice(), series.getMeasurement(), true));
-          if (schema == null) {
-            throw new MetadataException(
-                String.format(
-                    "Can not get the schema of measurement [%s]",
-                    chunkMetadata.getMeasurementUid()));
-          }
-          if (!existSeriesSet.contains(chunkMetadata.getMeasurementUid())) {
-            needRegisterSeries.add(series);
-            needRegisterSchema.add(schema);
-          }
-        }
-      }
-
-      int size = needRegisterSeries.size();
-      if (size == 0) {
-        continue;
-      }
-      if (isAligned) {
-        List<String> measurements = new ArrayList<>();
-        List<TSDataType> dataTypes = new ArrayList<>();
-        List<TSEncoding> encodings = new ArrayList<>();
-        List<CompressionType> compressors = new ArrayList<>();
-
-        for (int i = 0; i < size; i++) {
-          IMeasurementSchema schema = needRegisterSchema.get(i);
-          measurements.add(needRegisterSeries.get(i).getMeasurement());
-          dataTypes.add(schema.getType());
-          encodings.add(schema.getEncodingType());
-          compressors.add(schema.getCompressor());
-        }
-
-        IoTDB.schemaProcessor.createAlignedTimeSeries(
-            devicePath, measurements, dataTypes, encodings, compressors);
-      } else {
-        for (int i = 0; i < size; i++) {
-          IMeasurementSchema schema = needRegisterSchema.get(i);
-          IoTDB.schemaProcessor.createTimeseries(
-              needRegisterSeries.get(i),
-              schema.getType(),
-              schema.getEncodingType(),
-              schema.getCompressor(),
-              Collections.emptyMap());
-        }
-      }
-    }
-  }
-
-  private void operateRemoveFile(OperateFilePlan plan) throws QueryProcessException {
-    try {
-      if (!plan.getFile().exists() || !StorageEngine.getInstance().deleteTsfile(plan.getFile())) {
-        throw new QueryProcessException(
-            String.format("File '%s' doesn't exist.", plan.getFile().getAbsolutePath()));
-      }
-    } catch (StorageEngineException | IllegalPathException e) {
-      throw new QueryProcessException(
-          String.format("Cannot remove file because %s", e.getMessage()));
-    }
-  }
-
-  private void operateUnloadFile(OperateFilePlan plan) throws QueryProcessException {
-    if (!plan.getTargetDir().exists() || !plan.getTargetDir().isDirectory()) {
-      throw new QueryProcessException(
-          String.format("Target dir '%s' is invalid.", plan.getTargetDir().getAbsolutePath()));
-    }
-    try {
-      if (!plan.getFile().exists()
-          || !StorageEngine.getInstance().unloadTsfile(plan.getFile(), plan.getTargetDir())) {
-        throw new QueryProcessException(
-            String.format("File '%s' doesn't exist.", plan.getFile().getAbsolutePath()));
-      }
-    } catch (StorageEngineException | IllegalPathException e) {
-      throw new QueryProcessException(
-          String.format(
-              "Cannot unload file '%s' to target directory '%s' because %s",
-              plan.getFile().getAbsolutePath(), plan.getTargetDir().getPath(), e.getMessage()));
-    }
-  }
-
-  private void operateTTL(SetTTLPlan plan) throws QueryProcessException {
-    try {
-      List<PartialPath> storageGroupPaths =
-          IoTDB.schemaProcessor.getMatchedStorageGroups(
-              plan.getStorageGroup(), plan.isPrefixMatch());
-      for (PartialPath storagePath : storageGroupPaths) {
-        IoTDB.schemaProcessor.setTTL(storagePath, plan.getDataTTL());
-        StorageEngine.getInstance().setTTL(storagePath, plan.getDataTTL());
-      }
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    } catch (IOException e) {
-      throw new QueryProcessException(e.getMessage());
-    }
-  }
-
-  @Override
-  public void update(PartialPath path, long startTime, long endTime, String value) {
-    throw new UnsupportedOperationException("update is not supported now");
-  }
-
-  @Override
-  public void delete(
-      PartialPath path,
-      long startTime,
-      long endTime,
-      long planIndex,
-      TimePartitionFilter timePartitionFilter)
-      throws QueryProcessException {
-    try {
-      StorageEngine.getInstance().delete(path, startTime, endTime, planIndex, timePartitionFilter);
-    } catch (StorageEngineException e) {
-      throw new QueryProcessException(e);
-    }
-  }
-
-  private void checkFailedMeasurments(InsertPlan plan)
-      throws PathNotExistException, StorageEngineException {
-    // check if all path not exist exceptions
-    List<String> failedPaths = plan.getFailedMeasurements();
-    List<Exception> exceptions = plan.getFailedExceptions();
-    boolean isPathNotExistException = true;
-    for (Exception e : exceptions) {
-      Throwable curException = e;
-      while (curException.getCause() != null) {
-        curException = curException.getCause();
-      }
-      if (!(curException instanceof PathNotExistException)) {
-        isPathNotExistException = false;
-        break;
-      }
-    }
-    if (isPathNotExistException) {
-      throw new PathNotExistException(failedPaths);
-    } else {
-      throw new StorageEngineException(
-          INSERT_MEASUREMENTS_FAILED_MESSAGE
-              + plan.getFailedMeasurements()
-              + (!exceptions.isEmpty() ? (" caused by " + exceptions.get(0).getMessage()) : ""));
-    }
-  }
-
-  @Override
-  public void insert(InsertRowsOfOneDevicePlan insertRowsOfOneDevicePlan)
-      throws QueryProcessException {
-    if (insertRowsOfOneDevicePlan.getRowPlans().length == 0) {
-      return;
-    }
-    try {
-      // insert to storage engine
-      StorageEngine.getInstance().insert(insertRowsOfOneDevicePlan);
-
-      List<String> notExistedPaths = null;
-      List<String> failedMeasurements = null;
-
-      // If there are some exceptions, we assume they caused by the same reason.
-      Exception exception = null;
-      for (InsertRowPlan plan : insertRowsOfOneDevicePlan.getRowPlans()) {
-        if (plan.getFailedMeasurements() != null) {
-          if (notExistedPaths == null) {
-            notExistedPaths = new ArrayList<>();
-            failedMeasurements = new ArrayList<>();
-          }
-          // check if all path not exist exceptions
-          List<String> failedPaths = plan.getFailedMeasurements();
-          List<Exception> exceptions = plan.getFailedExceptions();
-          boolean isPathNotExistException = true;
-          for (Exception e : exceptions) {
-            exception = e;
-            Throwable curException = e;
-            while (curException.getCause() != null) {
-              curException = curException.getCause();
-            }
-            if (!(curException instanceof PathNotExistException)) {
-              isPathNotExistException = false;
-              break;
-            }
-          }
-          if (isPathNotExistException) {
-            notExistedPaths.addAll(failedPaths);
-          } else {
-            failedMeasurements.addAll(plan.getFailedMeasurements());
-          }
-        }
-      }
-      if (notExistedPaths != null && !notExistedPaths.isEmpty()) {
-        throw new PathNotExistException(notExistedPaths);
-      } else if (notExistedPaths != null && !failedMeasurements.isEmpty()) {
-        throw new StorageEngineException(
-            "failed to insert points "
-                + failedMeasurements
-                + (exception != null ? (" caused by " + exception.getMessage()) : ""));
-      }
-
-    } catch (StorageEngineException | MetadataException e) {
-      throw new QueryProcessException(e);
-    }
-  }
-
-  @Override
-  public void insert(InsertRowsPlan plan) throws QueryProcessException {
-    for (int i = 0; i < plan.getInsertRowPlanList().size(); i++) {
-      if (plan.getResults().containsKey(i) || plan.isExecuted(i)) {
-        continue;
-      }
-      try {
-        insert(plan.getInsertRowPlanList().get(i));
-      } catch (QueryProcessException e) {
-        plan.getResults().put(i, RpcUtils.getStatus(e.getErrorCode(), e.getMessage()));
-      }
-    }
-    if (!plan.getResults().isEmpty()) {
-      throw new BatchProcessException(plan.getFailingStatus());
-    }
-  }
-
-  @Override
-  public void insert(InsertRowPlan insertRowPlan) throws QueryProcessException {
-    try {
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-      // When insert data with sql statement, the data types will be null here.
-      // We need to predicted the data types first
-      if (insertRowPlan.getDataTypes()[0] == null) {
-        for (int i = 0; i < insertRowPlan.getDataTypes().length; i++) {
-          insertRowPlan.getDataTypes()[i] =
-              TypeInferenceUtils.getPredictedDataType(
-                  insertRowPlan.getValues()[i], insertRowPlan.isNeedInferType());
-        }
-      }
-
-      StorageEngine.getInstance().insert(insertRowPlan);
-
-      if (insertRowPlan.getFailedMeasurements() != null) {
-        checkFailedMeasurments(insertRowPlan);
-      }
-    } catch (StorageEngineException | MetadataException e) {
-      throw new QueryProcessException(e);
-    } catch (Exception e) {
-      throw e;
-    }
-  }
-
-  @Override
-  public void insertTablet(InsertMultiTabletsPlan insertMultiTabletsPlan)
-      throws QueryProcessException {
-    if (insertMultiTabletsPlan.isEnableMultiThreading()) {
-      insertTabletParallel(insertMultiTabletsPlan);
-    } else {
-      insertTabletSerial(insertMultiTabletsPlan);
-    }
-  }
-
-  private void insertTabletSerial(InsertMultiTabletsPlan insertMultiTabletsPlan)
-      throws BatchProcessException {
-    for (int i = 0; i < insertMultiTabletsPlan.getInsertTabletPlanList().size(); i++) {
-      if (insertMultiTabletsPlan.getResults().containsKey(i)
-          || insertMultiTabletsPlan.isExecuted(i)) {
-        continue;
-      }
-      try {
-        insertTablet(insertMultiTabletsPlan.getInsertTabletPlanList().get(i));
-      } catch (QueryProcessException e) {
-        insertMultiTabletsPlan
-            .getResults()
-            .put(i, RpcUtils.getStatus(e.getErrorCode(), e.getMessage()));
-      }
-    }
-    if (!insertMultiTabletsPlan.getResults().isEmpty()) {
-      throw new BatchProcessException(insertMultiTabletsPlan.getFailingStatus());
-    }
-  }
-
-  private void insertTabletParallel(InsertMultiTabletsPlan insertMultiTabletsPlan)
-      throws BatchProcessException {
-    updateInsertTabletsPool(insertMultiTabletsPlan.getDifferentStorageGroupsCount());
-
-    List<InsertTabletPlan> planList = insertMultiTabletsPlan.getInsertTabletPlanList();
-    List<Future<?>> futureList = new ArrayList<>();
-
-    Map<Integer, TSStatus> results = insertMultiTabletsPlan.getResults();
-
-    List<InsertTabletPlan> runPlanList = new ArrayList<>();
-    Map<Integer, Integer> runIndexToRealIndex = new HashMap<>();
-    for (int i = 0; i < planList.size(); i++) {
-      if (!(results.containsKey(i) || insertMultiTabletsPlan.isExecuted(i))) {
-        runPlanList.add(planList.get(i));
-        runIndexToRealIndex.put(runPlanList.size() - 1, i);
-      }
-    }
-    for (InsertTabletPlan plan : runPlanList) {
-      Future<?> f =
-          insertionPool.submit(
-              () -> {
-                insertTablet(plan);
-                return null;
-              });
-      futureList.add(f);
-    }
-    for (int i = 0; i < futureList.size(); i++) {
-      try {
-        futureList.get(i).get();
-      } catch (Exception e) {
-        if (e.getCause() instanceof QueryProcessException) {
-          QueryProcessException qe = (QueryProcessException) e.getCause();
-          results.put(
-              runIndexToRealIndex.get(i), RpcUtils.getStatus(qe.getErrorCode(), qe.getMessage()));
-        } else {
-          results.put(
-              runIndexToRealIndex.get(i),
-              RpcUtils.getStatus(INTERNAL_SERVER_ERROR, e.getMessage()));
-        }
-      }
-    }
-
-    if (!results.isEmpty()) {
-      throw new BatchProcessException(insertMultiTabletsPlan.getFailingStatus());
-    }
-  }
-
-  private void updateInsertTabletsPool(int sgSize) {
-    int updateCoreSize = Math.min(sgSize, Runtime.getRuntime().availableProcessors() / 2);
-    if (insertionPool == null || insertionPool.isTerminated()) {
-      insertionPool =
-          (ThreadPoolExecutor)
-              IoTDBThreadPoolFactory.newFixedThreadPool(
-                  updateCoreSize, ThreadName.INSERTION_SERVICE.getName());
-    } else if (insertionPool.getCorePoolSize() > updateCoreSize) {
-      insertionPool.setCorePoolSize(updateCoreSize);
-      insertionPool.setMaximumPoolSize(updateCoreSize);
-    } else if (insertionPool.getCorePoolSize() < updateCoreSize) {
-      insertionPool.setMaximumPoolSize(updateCoreSize);
-      insertionPool.setCorePoolSize(updateCoreSize);
-    }
-  }
-
-  @Override
-  public void insertTablet(InsertTabletPlan insertTabletPlan) throws QueryProcessException {
-
-    if (insertTabletPlan.getRowCount() == 0) {
-      return;
-    }
-    try {
-      insertTabletPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertTabletPlan.getMeasurements().length]);
-
-      StorageEngine.getInstance().insertTablet(insertTabletPlan);
-
-      if (insertTabletPlan.getFailedMeasurements() != null) {
-        checkFailedMeasurments(insertTabletPlan);
-      }
-    } catch (StorageEngineException | MetadataException e) {
-      throw new QueryProcessException(e);
-    } catch (Exception e) {
-      throw e;
-    }
-  }
-
-  private boolean operateAuthor(AuthorPlan author) throws QueryProcessException {
-    AuthorOperator.AuthorType authorType = author.getAuthorType();
-    String userName = author.getUserName();
-    String roleName = author.getRoleName();
-    String password = author.getPassword();
-    String newPassword = author.getNewPassword();
-    Set<Integer> permissions = author.getPermissions();
-    List<PartialPath> nodeNameList = author.getNodeNameList();
-    try {
-      switch (authorType) {
-        case UPDATE_USER:
-          authorizerManager.updateUserPassword(userName, newPassword);
-          break;
-        case CREATE_USER:
-          authorizerManager.createUser(userName, password);
-          break;
-        case CREATE_ROLE:
-          authorizerManager.createRole(roleName);
-          break;
-        case DROP_USER:
-          authorizerManager.deleteUser(userName);
-          break;
-        case DROP_ROLE:
-          authorizerManager.deleteRole(roleName);
-          break;
-        case GRANT_ROLE:
-          for (int i : permissions) {
-            for (PartialPath path : nodeNameList) {
-              authorizerManager.grantPrivilegeToRole(roleName, path.getFullPath(), i);
-            }
-          }
-          break;
-        case GRANT_USER:
-          for (int i : permissions) {
-            for (PartialPath path : nodeNameList) {
-              authorizerManager.grantPrivilegeToUser(userName, path.getFullPath(), i);
-            }
-          }
-          break;
-        case GRANT_USER_ROLE:
-          authorizerManager.grantRoleToUser(roleName, userName);
-          break;
-        case REVOKE_USER:
-          for (int i : permissions) {
-            for (PartialPath path : nodeNameList) {
-              authorizerManager.revokePrivilegeFromUser(userName, path.getFullPath(), i);
-            }
-          }
-          break;
-        case REVOKE_ROLE:
-          for (int i : permissions) {
-            for (PartialPath path : nodeNameList) {
-              authorizerManager.revokePrivilegeFromRole(roleName, path.getFullPath(), i);
-            }
-          }
-          break;
-        case REVOKE_USER_ROLE:
-          authorizerManager.revokeRoleFromUser(roleName, userName);
-          break;
-        default:
-          throw new QueryProcessException("Unsupported operation " + authorType);
-      }
-    } catch (AuthException e) {
-      throw new QueryProcessException(e.getMessage(), true);
-    }
-    return true;
-  }
-
-  private boolean operateWatermarkEmbedding(List<String> users, boolean useWatermark)
-      throws QueryProcessException {
-    try {
-      for (String user : users) {
-        authorizerManager.setUserUseWaterMark(user, useWatermark);
-      }
-    } catch (AuthException e) {
-      throw new QueryProcessException(e.getMessage());
-    }
-    return true;
-  }
-
-  private boolean createTimeSeries(CreateTimeSeriesPlan createTimeSeriesPlan)
-      throws QueryProcessException {
-    try {
-      IoTDB.schemaProcessor.createTimeseries(createTimeSeriesPlan);
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    }
-    return true;
-  }
-
-  private boolean createAlignedTimeSeries(CreateAlignedTimeSeriesPlan createAlignedTimeSeriesPlan)
-      throws QueryProcessException {
-    try {
-      IoTDB.schemaProcessor.createAlignedTimeSeries(createAlignedTimeSeriesPlan);
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    }
-    return true;
-  }
-
   // high Cognitive Complexity
-  private boolean createMultiTimeSeries(CreateMultiTimeSeriesPlan multiPlan)
-      throws BatchProcessException {
-    int dataTypeIdx = 0;
-    for (int i = 0; i < multiPlan.getPaths().size(); i++) {
-      if (multiPlan.getResults().containsKey(i) || multiPlan.isExecuted(i)) {
-        continue;
-      }
-      PartialPath path = multiPlan.getPaths().get(i);
-      String measurement = path.getMeasurement();
-      CreateTimeSeriesPlan plan =
-          new CreateTimeSeriesPlan(
-              multiPlan.getPaths().get(i),
-              multiPlan.getDataTypes().get(i),
-              multiPlan.getEncodings().get(i),
-              multiPlan.getCompressors().get(i),
-              multiPlan.getProps() == null ? null : multiPlan.getProps().get(i),
-              multiPlan.getTags() == null ? null : multiPlan.getTags().get(i),
-              multiPlan.getAttributes() == null ? null : multiPlan.getAttributes().get(i),
-              multiPlan.getAlias() == null ? null : multiPlan.getAlias().get(i));
-      dataTypeIdx++;
-      try {
-        createTimeSeries(plan);
-      } catch (QueryProcessException e) {
-        multiPlan.getResults().put(i, RpcUtils.getStatus(e.getErrorCode(), e.getMessage()));
-      }
-    }
-    if (!multiPlan.getResults().isEmpty()) {
-      throw new BatchProcessException(multiPlan.getFailingStatus());
-    }
-    return true;
-  }
-
-  protected boolean deleteTimeSeries(DeleteTimeSeriesPlan deleteTimeSeriesPlan)
-      throws QueryProcessException {
-    AUDIT_LOGGER.info("delete timeseries {}", deleteTimeSeriesPlan.getPaths());
-    List<PartialPath> deletePathList = deleteTimeSeriesPlan.getPaths();
-    for (int i = 0; i < deletePathList.size(); i++) {
-      PartialPath path = deletePathList.get(i);
-      try {
-        StorageEngine.getInstance()
-            .deleteTimeseries(
-                path, deleteTimeSeriesPlan.getIndex(), deleteTimeSeriesPlan.getPartitionFilter());
-        if (deleteTimeSeriesPlan.isPrefixMatch()) {
-          // adapt to prefix match of 0.12
-          StorageEngine.getInstance()
-              .deleteTimeseries(
-                  path.concatNode(MULTI_LEVEL_PATH_WILDCARD),
-                  deleteTimeSeriesPlan.getIndex(),
-                  deleteTimeSeriesPlan.getPartitionFilter());
-        }
-        String failed =
-            IoTDB.schemaProcessor.deleteTimeseries(path, deleteTimeSeriesPlan.isPrefixMatch());
-        if (failed != null) {
-          deleteTimeSeriesPlan
-              .getResults()
-              .put(i, RpcUtils.getStatus(TSStatusCode.NODE_DELETE_FAILED_ERROR, failed));
-        }
-      } catch (StorageEngineException | MetadataException e) {
-        deleteTimeSeriesPlan
-            .getResults()
-            .put(i, RpcUtils.getStatus(e.getErrorCode(), e.getMessage()));
-      }
-    }
-    if (!deleteTimeSeriesPlan.getResults().isEmpty()) {
-      throw new BatchProcessException(deleteTimeSeriesPlan.getFailingStatus());
-    }
-    return true;
-  }
-
-  private boolean alterTimeSeries(AlterTimeSeriesPlan alterTimeSeriesPlan)
-      throws QueryProcessException {
-    PartialPath path = alterTimeSeriesPlan.getPath();
-    Map<String, String> alterMap = alterTimeSeriesPlan.getAlterMap();
-    try {
-      switch (alterTimeSeriesPlan.getAlterType()) {
-        case RENAME:
-          String beforeName = alterMap.keySet().iterator().next();
-          String currentName = alterMap.get(beforeName);
-          IoTDB.schemaProcessor.renameTagOrAttributeKey(beforeName, currentName, path);
-          break;
-        case SET:
-          IoTDB.schemaProcessor.setTagsOrAttributesValue(alterMap, path);
-          break;
-        case DROP:
-          IoTDB.schemaProcessor.dropTagsOrAttributes(alterMap.keySet(), path);
-          break;
-        case ADD_TAGS:
-          IoTDB.schemaProcessor.addTags(alterMap, path);
-          break;
-        case ADD_ATTRIBUTES:
-          IoTDB.schemaProcessor.addAttributes(alterMap, path);
-          break;
-        case UPSERT:
-          IoTDB.schemaProcessor.upsertTagsAndAttributes(
-              alterTimeSeriesPlan.getAlias(),
-              alterTimeSeriesPlan.getTagsMap(),
-              alterTimeSeriesPlan.getAttributesMap(),
-              path);
-          break;
-      }
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    } catch (IOException e) {
-      throw new QueryProcessException(
-          String.format(
-              "Something went wrong while read/write the [%s]'s tag/attribute info.",
-              path.getFullPath()));
-    }
-    return true;
-  }
-
-  public boolean setStorageGroup(SetStorageGroupPlan setStorageGroupPlan)
-      throws QueryProcessException {
-    AUDIT_LOGGER.info("set storage group to {}", setStorageGroupPlan.getPaths());
-    PartialPath path = setStorageGroupPlan.getPath();
-    try {
-      IoTDB.schemaProcessor.setStorageGroup(path);
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    }
-    return true;
-  }
-
-  protected boolean deleteStorageGroups(DeleteStorageGroupPlan deleteStorageGroupPlan)
-      throws QueryProcessException {
-    AUDIT_LOGGER.info("delete storage group {}", deleteStorageGroupPlan.getPaths());
-    List<PartialPath> deletePathList = new ArrayList<>();
-    try {
-      for (PartialPath storageGroupPath : deleteStorageGroupPlan.getPaths()) {
-        List<PartialPath> allRelatedStorageGroupPath =
-            IoTDB.schemaProcessor.getMatchedStorageGroups(
-                storageGroupPath, deleteStorageGroupPlan.isPrefixMatch());
-        if (allRelatedStorageGroupPath.isEmpty()) {
-          throw new PathNotExistException(storageGroupPath.getFullPath(), true);
-        }
-        for (PartialPath path : allRelatedStorageGroupPath) {
-          StorageEngine.getInstance().deleteStorageGroup(path);
-          deletePathList.add(path);
-        }
-      }
-      IoTDB.schemaProcessor.deleteStorageGroups(deletePathList);
-      operateClearCache();
-    } catch (MetadataException e) {
-      throw new QueryProcessException(e);
-    }
-    return true;
-  }
 
   protected QueryDataSet processAuthorQuery(AuthorPlan plan) throws QueryProcessException {
     AuthorType authorType = plan.getAuthorType();
@@ -2353,11 +1086,6 @@ public class PlanExecutor implements IPlanExecutor {
     }
   }
 
-  // for the distributed version
-  protected void loadConfiguration(LoadConfigurationPlan plan) throws QueryProcessException {
-    IoTDBDescriptor.getInstance().loadHotModifiedProps();
-  }
-
   private boolean processShowQueryResource() {
     DEBUG_LOGGER.info(String.format("**********%s**********\n\n", new Date()));
     FileReaderManager.getInstance().writeFileReferenceInfo();
@@ -2380,106 +1108,4 @@ public class PlanExecutor implements IPlanExecutor {
     }
     return listDataSet;
   }
-
-  /**
-   * @param storageGroups the storage groups to check
-   * @return List of PartialPath the storage groups that not exist
-   */
-  public static List<PartialPath> checkStorageGroupExist(List<PartialPath> storageGroups) {
-    List<PartialPath> noExistSg = new ArrayList<>();
-    if (storageGroups == null) {
-      return noExistSg;
-    }
-    for (PartialPath storageGroup : storageGroups) {
-      if (!IoTDB.schemaProcessor.isStorageGroup(storageGroup)) {
-        noExistSg.add(storageGroup);
-      }
-    }
-    return noExistSg;
-  }
-
-  private void settle(SettlePlan plan) throws StorageEngineException {
-    if (CommonDescriptor.getInstance().getConfig().isReadOnly()) {
-      throw new StorageEngineException(
-          "Current system mode is read only, does not support file settle");
-    }
-    if (!SettleService.getINSTANCE().isRecoverFinish()) {
-      throw new StorageEngineException("Existing sg that is not ready, please try later.");
-    }
-    PartialPath sgPath = null;
-    try {
-      List<TsFileResource> seqResourcesToBeSettled = new ArrayList<>();
-      List<TsFileResource> unseqResourcesToBeSettled = new ArrayList<>();
-      List<String> tsFilePaths = new ArrayList<>();
-      if (plan.isSgPath()) {
-        sgPath = plan.getSgPath();
-      } else {
-        String tsFilePath = plan.getTsFilePath();
-        if (new File(tsFilePath).isDirectory()) {
-          throw new WriteProcessException("The file should not be a directory.");
-        } else if (!new File(tsFilePath).exists()) {
-          throw new WriteProcessException("The tsFile " + tsFilePath + " is not existed.");
-        }
-        sgPath = SettleService.getINSTANCE().getSGByFilePath(tsFilePath);
-        tsFilePaths.add(tsFilePath);
-      }
-      StorageEngine.getInstance()
-          .getResourcesToBeSettled(
-              sgPath, seqResourcesToBeSettled, unseqResourcesToBeSettled, tsFilePaths);
-      SettleService.getINSTANCE().startSettling(seqResourcesToBeSettled, unseqResourcesToBeSettled);
-      StorageEngine.getInstance().setSettling(sgPath, false);
-    } catch (WriteProcessException e) {
-      if (sgPath != null) {
-        StorageEngine.getInstance().setSettling(sgPath, false);
-      }
-      throw new StorageEngineException(e.getMessage());
-    }
-  }
-
-  private void createPipeSink(CreatePipeSinkPlan plan) throws QueryProcessException {
-    try {
-      SyncService.getInstance().addPipeSink(plan);
-    } catch (PipeSinkException e) {
-      throw new QueryProcessException("Create pipeSink error.", e); // e will override the message
-    } catch (IllegalArgumentException e) {
-      throw new QueryProcessException(
-          "Do not support pipeSink type " + plan.getPipeSinkType() + ".");
-    }
-  }
-
-  private void dropPipeSink(DropPipeSinkPlan plan) throws QueryProcessException {
-    try {
-      SyncService.getInstance().dropPipeSink(plan.getPipeSinkName());
-    } catch (PipeSinkException e) {
-      throw new QueryProcessException("Can not drop pipeSink.", e);
-    }
-  }
-
-  private void createPipe(CreatePipePlan plan) throws QueryProcessException {
-    try {
-      SyncService.getInstance()
-          .addPipe(SyncPipeUtil.parseCreatePipePlanAsPipeInfo(plan, System.currentTimeMillis()));
-    } catch (PipeException e) {
-      throw new QueryProcessException("Create pipe error.", e);
-    }
-  }
-
-  private void operatePipe(OperatePipePlan plan) throws QueryProcessException {
-    try {
-      if (Operator.OperatorType.STOP_PIPE.equals(plan.getOperatorType())) {
-        SyncService.getInstance().stopPipe(plan.getPipeName());
-      } else if (Operator.OperatorType.START_PIPE.equals(plan.getOperatorType())) {
-        SyncService.getInstance().startPipe(plan.getPipeName());
-      } else if (Operator.OperatorType.DROP_PIPE.equals(plan.getOperatorType())) {
-        SyncService.getInstance().dropPipe(plan.getPipeName());
-      } else {
-        throw new QueryProcessException(
-            String.format("Error operator type %s.", plan.getOperatorType()),
-            INTERNAL_SERVER_ERROR.getStatusCode());
-      }
-    } catch (PipeException e) {
-      throw new QueryProcessException(
-          String.format("%s pipe error.", plan.getOperatorType().name()), e);
-    }
-  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java b/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java
index f3fdd057c9..5ad9663405 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java
@@ -171,16 +171,6 @@ public class IoTDB implements IoTDBMBean {
     logger.info(
         "IoTDB is setting up, some storage groups may not be ready now, please wait several seconds...");
 
-    while (!StorageEngine.getInstance().isAllSgReady()) {
-      try {
-        Thread.sleep(1000);
-      } catch (InterruptedException e) {
-        logger.warn("IoTDB failed to set up.", e);
-        Thread.currentThread().interrupt();
-        return;
-      }
-    }
-
     registerManager.register(UpgradeSevice.getINSTANCE());
     registerManager.register(SettleService.getINSTANCE());
     registerManager.register(TriggerRegistrationService.getInstance());
diff --git a/server/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java b/server/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java
index b6d55393f5..492b87a671 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java
@@ -21,7 +21,6 @@ package org.apache.iotdb.db.service;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.consensus.DataRegionConsensusImpl;
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.StorageEngineV2;
 import org.apache.iotdb.db.metadata.schemaregion.SchemaEngineMode;
 import org.apache.iotdb.db.utils.MemUtils;
@@ -48,12 +47,8 @@ public class IoTDBShutdownHook extends Thread {
     WALManager.getInstance().waitAllWALFlushed();
 
     // flush data to Tsfile and remove WAL log files
-    if (IoTDBDescriptor.getInstance().getConfig().isMppMode()) {
-      if (!IoTDBDescriptor.getInstance().getConfig().isClusterMode()) {
-        StorageEngineV2.getInstance().syncCloseAllProcessor();
-      }
-    } else {
-      StorageEngine.getInstance().syncCloseAllProcessor();
+    if (!IoTDBDescriptor.getInstance().getConfig().isClusterMode()) {
+      StorageEngineV2.getInstance().syncCloseAllProcessor();
     }
     WALManager.getInstance().deleteOutdatedWALFiles();
 
diff --git a/server/src/main/java/org/apache/iotdb/db/service/MQTTService.java b/server/src/main/java/org/apache/iotdb/db/service/MQTTService.java
index 564019254b..f572a4fd28 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/MQTTService.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/MQTTService.java
@@ -59,11 +59,7 @@ public class MQTTService implements IService {
     IoTDBConfig iotDBConfig = IoTDBDescriptor.getInstance().getConfig();
     IConfig config = createBrokerConfig(iotDBConfig);
     List<InterceptHandler> handlers = new ArrayList<>(1);
-    if (iotDBConfig.isMppMode()) {
-      handlers.add(new MPPPublishHandler(iotDBConfig));
-    } else {
-      throw new UnsupportedOperationException();
-    }
+    handlers.add(new MPPPublishHandler(iotDBConfig));
     IAuthenticator authenticator = new BrokerAuthenticator();
 
     server.startServer(config, handlers, null, authenticator, null);
diff --git a/server/src/main/java/org/apache/iotdb/db/service/SettleService.java b/server/src/main/java/org/apache/iotdb/db/service/SettleService.java
index e7afcf92f1..4df2a49069 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/SettleService.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/SettleService.java
@@ -26,11 +26,9 @@ import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.service.IService;
 import org.apache.iotdb.commons.service.ServiceType;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.settle.SettleLog;
 import org.apache.iotdb.db.engine.settle.SettleTask;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.WriteProcessException;
 import org.apache.iotdb.db.tools.settle.TsFileAndModSettleTool;
 
@@ -89,25 +87,25 @@ public class SettleService implements IService {
           tmpSgResourcesMap.put(sgPath, tsFilePaths);
         }
       }
-      while (!StorageEngine.getInstance().isAllSgReady()) {
-        // wait for all sg ready
-      }
+      //      while (!StorageEngine.getInstance().isAllSgReady()) {
+      //        // wait for all sg ready
+      //      }
       List<TsFileResource> seqResourcesToBeSettled = new ArrayList<>();
       List<TsFileResource> unseqResourcesToBeSettled = new ArrayList<>();
-      for (Map.Entry<PartialPath, List<String>> entry : tmpSgResourcesMap.entrySet()) {
-        try {
-          StorageEngine.getInstance()
-              .getResourcesToBeSettled(
-                  entry.getKey(),
-                  seqResourcesToBeSettled,
-                  unseqResourcesToBeSettled,
-                  entry.getValue());
-        } catch (StorageEngineException e) {
-          e.printStackTrace();
-        } finally {
-          StorageEngine.getInstance().setSettling(entry.getKey(), false);
-        }
-      }
+      //      for (Map.Entry<PartialPath, List<String>> entry : tmpSgResourcesMap.entrySet()) {
+      //        try {
+      //          StorageEngine.getInstance()
+      //              .getResourcesToBeSettled(
+      //                  entry.getKey(),
+      //                  seqResourcesToBeSettled,
+      //                  unseqResourcesToBeSettled,
+      //                  entry.getValue());
+      //        } catch (StorageEngineException e) {
+      //          e.printStackTrace();
+      //        } finally {
+      //          StorageEngine.getInstance().setSettling(entry.getKey(), false);
+      //        }
+      //      }
       startSettling(seqResourcesToBeSettled, unseqResourcesToBeSettled);
       setRecoverFinish(true);
     } catch (WriteProcessException e) {
diff --git a/server/src/main/java/org/apache/iotdb/db/service/UpgradeSevice.java b/server/src/main/java/org/apache/iotdb/db/service/UpgradeSevice.java
index c32cd5db70..4838baba34 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/UpgradeSevice.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/UpgradeSevice.java
@@ -22,10 +22,8 @@ import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.commons.service.IService;
 import org.apache.iotdb.commons.service.ServiceType;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.upgrade.UpgradeLog;
 import org.apache.iotdb.db.engine.upgrade.UpgradeTask;
-import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.utils.UpgradeUtils;
 
 import org.slf4j.Logger;
@@ -95,15 +93,15 @@ public class UpgradeSevice implements IService {
   }
 
   private static void countUpgradeFiles() {
-    cntUpgradeFileNum.addAndGet(StorageEngine.getInstance().countUpgradeFiles());
-    logger.info("finish counting upgrading files, total num:{}", cntUpgradeFileNum);
+    //    cntUpgradeFileNum.addAndGet(StorageEngine.getInstance().countUpgradeFiles());
+    //    logger.info("finish counting upgrading files, total num:{}", cntUpgradeFileNum);
   }
 
   private static void upgradeAll() {
-    try {
-      StorageEngine.getInstance().upgradeAll();
-    } catch (StorageEngineException e) {
-      logger.error("Cannot perform a global upgrade because", e);
-    }
+    //    try {
+    //      StorageEngine.getInstance().upgradeAll();
+    //    } catch (StorageEngineException e) {
+    //      logger.error("Cannot perform a global upgrade because", e);
+    //    }
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/DeletionLoader.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/DeletionLoader.java
index 400d8b629c..ed79fdcf1a 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/DeletionLoader.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/DeletionLoader.java
@@ -21,7 +21,6 @@ package org.apache.iotdb.db.sync.receiver.load;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.exception.sync.PipeDataLoadException;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.modification.Deletion;
 import org.apache.iotdb.db.exception.LoadFileException;
 import org.apache.iotdb.db.mpp.plan.Coordinator;
@@ -53,12 +52,6 @@ public class DeletionLoader implements ILoader {
       throw new PipeDataLoadException("storage engine readonly");
     }
     try {
-      if (!config.isMppMode()) {
-        StorageEngine.getInstance()
-            .delete(deletion.getPath(), deletion.getStartTime(), deletion.getEndTime(), 0, null);
-        return;
-      }
-
       Statement statement = generateStatement();
       long queryId = SessionManager.getInstance().requestQueryId();
       ExecutionResult result =
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/TsFileLoader.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/TsFileLoader.java
index 7b85835f32..f377bb1f5a 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/TsFileLoader.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/TsFileLoader.java
@@ -28,9 +28,6 @@ import org.apache.iotdb.db.mpp.plan.Coordinator;
 import org.apache.iotdb.db.mpp.plan.execution.ExecutionResult;
 import org.apache.iotdb.db.mpp.plan.statement.crud.LoadTsFileStatement;
 import org.apache.iotdb.db.qp.executor.PlanExecutor;
-import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.sys.OperateFilePlan;
 import org.apache.iotdb.db.query.control.SessionManager;
 import org.apache.iotdb.rpc.TSStatusCode;
 
@@ -64,18 +61,6 @@ public class TsFileLoader implements ILoader {
   @Override
   public void load() throws PipeDataLoadException {
     try {
-      if (!config.isMppMode()) {
-        PhysicalPlan plan =
-            new OperateFilePlan(
-                tsFile,
-                Operator.OperatorType.LOAD_FILES,
-                true,
-                IoTDBDescriptor.getInstance().getConfig().getDefaultStorageGroupLevel(),
-                true,
-                true);
-        planExecutor.processNonQuery(plan);
-        return;
-      }
 
       LoadTsFileStatement statement = new LoadTsFileStatement(tsFile.getAbsolutePath());
       statement.setDeleteAfterLoad(true);
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/TsFileSplitByPartitionTool.java b/server/src/main/java/org/apache/iotdb/db/tools/TsFileSplitByPartitionTool.java
index cee707e0ac..a2f1762e1f 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/TsFileSplitByPartitionTool.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/TsFileSplitByPartitionTool.java
@@ -21,7 +21,7 @@ package org.apache.iotdb.db.tools;
 
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.StorageEngineV2;
 import org.apache.iotdb.db.engine.modification.Deletion;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.modification.ModificationFile;
@@ -289,8 +289,8 @@ public class TsFileSplitByPartitionTool implements AutoCloseable {
         }
       }
     }
-    return StorageEngine.getTimePartition(pageHeader.getStartTime())
-        != StorageEngine.getTimePartition(pageHeader.getEndTime());
+    return StorageEngineV2.getTimePartition(pageHeader.getStartTime())
+        != StorageEngineV2.getTimePartition(pageHeader.getEndTime());
   }
 
   /**
@@ -376,7 +376,7 @@ public class TsFileSplitByPartitionTool implements AutoCloseable {
       ByteBuffer pageData,
       Map<Long, ChunkWriterImpl> partitionChunkWriterMap)
       throws PageException {
-    long partitionId = StorageEngine.getTimePartition(pageHeader.getStartTime());
+    long partitionId = StorageEngineV2.getTimePartition(pageHeader.getStartTime());
     getOrDefaultTsFileIOWriter(oldTsFile, partitionId);
     ChunkWriterImpl chunkWriter =
         partitionChunkWriterMap.computeIfAbsent(partitionId, v -> new ChunkWriterImpl(schema));
@@ -431,7 +431,7 @@ public class TsFileSplitByPartitionTool implements AutoCloseable {
     while (batchData.hasCurrent()) {
       long time = batchData.currentTime();
       Object value = batchData.currentValue();
-      long partitionId = StorageEngine.getTimePartition(time);
+      long partitionId = StorageEngineV2.getTimePartition(time);
 
       ChunkWriterImpl chunkWriter =
           partitionChunkWriterMap.computeIfAbsent(partitionId, v -> new ChunkWriterImpl(schema));
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/dataregion/DeviceMappingViewer.java b/server/src/main/java/org/apache/iotdb/db/tools/dataregion/DeviceMappingViewer.java
deleted file mode 100644
index 4a9bde9407..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/tools/dataregion/DeviceMappingViewer.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.tools.dataregion;
-
-import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.storagegroup.dataregion.HashVirtualPartitioner;
-import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
-import org.apache.iotdb.db.service.IoTDB;
-
-import java.util.Set;
-
-/**
- * for DBA to view the mapping from device to virtual storage group ID usage: run this class with
- * arguments [system_schema_dir], if args are not given, use default in config
- */
-public class DeviceMappingViewer {
-
-  public static void main(String[] args) throws MetadataException {
-    // has schema log dir
-    if (args.length == 1) {
-      IoTDBDescriptor.getInstance().getConfig().setSchemaDir(args[0]);
-    }
-
-    HashVirtualPartitioner partitioner = HashVirtualPartitioner.getInstance();
-    IoTDB.configManager.init();
-    LocalSchemaProcessor schemaProcessor = LocalSchemaProcessor.getInstance();
-
-    Set<PartialPath> partialPathSet =
-        schemaProcessor.getMatchedDevices(new PartialPath("root.**"), false);
-
-    if (partialPathSet.isEmpty() && args.length == 1) {
-      System.out.println("no mlog in given system schema dir: " + args[0] + " please have a check");
-    } else {
-      System.out.println();
-      System.out.println(
-          "--------------------- mapping from device to virtual storage group ID ---------------------");
-      System.out.println("Format is: device name -> virtual storage group ID");
-      for (PartialPath partialPath : partialPathSet) {
-        System.out.println(partialPath + " -> " + partitioner.deviceToDataRegionId(partialPath));
-      }
-    }
-
-    IoTDB.configManager.clear();
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java b/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
index 976b5d178a..99bf57b8af 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/upgrade/TsFileOnlineUpgradeTool.java
@@ -19,7 +19,7 @@
 package org.apache.iotdb.db.tools.upgrade;
 
 import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.StorageEngineV2;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.tools.TsFileSplitByPartitionTool;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
@@ -250,8 +250,8 @@ public class TsFileOnlineUpgradeTool extends TsFileSplitByPartitionTool {
     return dataType == TSDataType.BOOLEAN
         || dataType == TSDataType.TEXT
         || (dataType == TSDataType.INT32 && encoding == TSEncoding.PLAIN)
-        || StorageEngine.getTimePartition(pageHeader.getStartTime())
-            != StorageEngine.getTimePartition(pageHeader.getEndTime())
+        || StorageEngineV2.getTimePartition(pageHeader.getStartTime())
+            != StorageEngineV2.getTimePartition(pageHeader.getEndTime())
         || super.checkIfNeedToDecode(schema, deviceId, pageHeader, chunkHeaderOffset);
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntry.java b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntry.java
index 4286c9f386..60fa745858 100644
--- a/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntry.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntry.java
@@ -26,10 +26,6 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeType;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.DeleteDataNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.utils.SerializedSize;
 import org.apache.iotdb.db.wal.utils.listener.WALFlushListener;
 
@@ -63,13 +59,7 @@ public abstract class WALEntry implements SerializedSize {
   public WALEntry(long memTableId, WALEntryValue value, boolean wait) {
     this.memTableId = memTableId;
     this.value = value;
-    if (value instanceof InsertRowPlan) {
-      this.type = WALEntryType.INSERT_ROW_PLAN;
-    } else if (value instanceof InsertTabletPlan) {
-      this.type = WALEntryType.INSERT_TABLET_PLAN;
-    } else if (value instanceof DeletePlan) {
-      this.type = WALEntryType.DELETE_PLAN;
-    } else if (value instanceof IMemTable) {
+    if (value instanceof IMemTable) {
       this.type = WALEntryType.MEMORY_TABLE_SNAPSHOT;
     } else if (value instanceof InsertRowNode) {
       this.type = WALEntryType.INSERT_ROW_NODE;
@@ -109,15 +99,6 @@ public abstract class WALEntry implements SerializedSize {
     long memTableId = stream.readLong();
     WALEntryValue value = null;
     switch (type) {
-      case INSERT_ROW_PLAN:
-        value = (InsertRowPlan) PhysicalPlan.Factory.create(stream);
-        break;
-      case INSERT_TABLET_PLAN:
-        value = (InsertTabletPlan) PhysicalPlan.Factory.create(stream);
-        break;
-      case DELETE_PLAN:
-        value = (DeletePlan) PhysicalPlan.Factory.create(stream);
-        break;
       case MEMORY_TABLE_SNAPSHOT:
         value = AbstractMemTable.Factory.create(stream);
         break;
@@ -130,6 +111,8 @@ public abstract class WALEntry implements SerializedSize {
       case DELETE_DATA_NODE:
         value = (DeleteDataNode) PlanNodeType.deserializeFromWAL(stream);
         break;
+      default:
+        throw new RuntimeException("Unknown WALEntry type " + type);
     }
     return new WALInfoEntry(type, memTableId, value);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryType.java b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryType.java
index be32b339d7..ff1da0c37a 100644
--- a/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryType.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALEntryType.java
@@ -22,10 +22,13 @@ package org.apache.iotdb.db.wal.buffer;
 public enum WALEntryType {
   // region info entry type
   /** {@link org.apache.iotdb.db.qp.physical.crud.InsertRowPlan} */
+  @Deprecated
   INSERT_ROW_PLAN((byte) 0),
   /** {@link org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan} */
+  @Deprecated
   INSERT_TABLET_PLAN((byte) 1),
   /** {@link org.apache.iotdb.db.qp.physical.crud.DeletePlan} */
+  @Deprecated
   DELETE_PLAN((byte) 2),
   /** snapshot of {@link org.apache.iotdb.db.engine.memtable.IMemTable} */
   MEMORY_TABLE_SNAPSHOT((byte) 3),
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALInfoEntry.java b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALInfoEntry.java
index a47eff9a54..d4246616c7 100644
--- a/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALInfoEntry.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/buffer/WALInfoEntry.java
@@ -21,7 +21,6 @@ package org.apache.iotdb.db.wal.buffer;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.wal.utils.WALMode;
 
 /** This entry class stores info for persistence */
@@ -40,18 +39,11 @@ public class WALInfoEntry extends WALEntry {
 
   public WALInfoEntry(long memTableId, WALEntryValue value) {
     this(memTableId, value, config.getWalMode() == WALMode.SYNC);
-    if (value instanceof InsertTabletPlan) {
-      tabletInfo = new TabletInfo(0, ((InsertTabletPlan) value).getRowCount());
-    } else if (value instanceof InsertTabletNode) {
+    if (value instanceof InsertTabletNode) {
       tabletInfo = new TabletInfo(0, ((InsertTabletNode) value).getRowCount());
     }
   }
 
-  public WALInfoEntry(long memTableId, InsertTabletPlan value, int tabletStart, int tabletEnd) {
-    this(memTableId, value, config.getWalMode() == WALMode.SYNC);
-    tabletInfo = new TabletInfo(tabletStart, tabletEnd);
-  }
-
   public WALInfoEntry(long memTableId, InsertTabletNode value, int tabletStart, int tabletEnd) {
     this(memTableId, value, config.getWalMode() == WALMode.SYNC);
     tabletInfo = new TabletInfo(tabletStart, tabletEnd);
@@ -71,21 +63,17 @@ public class WALInfoEntry extends WALEntry {
     buffer.put(type.getCode());
     buffer.putLong(memTableId);
     switch (type) {
-      case INSERT_TABLET_PLAN:
-        ((InsertTabletPlan) value)
-            .serializeToWAL(buffer, tabletInfo.tabletStart, tabletInfo.tabletEnd);
-        break;
       case INSERT_TABLET_NODE:
         ((InsertTabletNode) value)
             .serializeToWAL(buffer, tabletInfo.tabletStart, tabletInfo.tabletEnd);
         break;
-      case INSERT_ROW_PLAN:
       case INSERT_ROW_NODE:
-      case DELETE_PLAN:
       case DELETE_DATA_NODE:
       case MEMORY_TABLE_SNAPSHOT:
         value.serializeToWAL(buffer);
         break;
+      default:
+        throw new RuntimeException("Unsupported wal entry type " + type);
     }
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/node/IWALNode.java b/server/src/main/java/org/apache/iotdb/db/wal/node/IWALNode.java
index 38b12a8a85..c0995ad584 100644
--- a/server/src/main/java/org/apache/iotdb/db/wal/node/IWALNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/node/IWALNode.java
@@ -25,28 +25,17 @@ import org.apache.iotdb.db.engine.memtable.IMemTable;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.DeleteDataNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.wal.utils.listener.WALFlushListener;
 
 /** This interface provides uniform interface for writing wal and making checkpoints. */
 public interface IWALNode extends FlushListener, AutoCloseable, ConsensusReqReader, DataSet {
-  /** Log InsertRowPlan */
-  WALFlushListener log(long memTableId, InsertRowPlan insertRowPlan);
 
   /** Log InsertRowNode */
   WALFlushListener log(long memTableId, InsertRowNode insertRowNode);
 
-  /** Log InsertTabletPlan */
-  WALFlushListener log(long memTableId, InsertTabletPlan insertTabletPlan, int start, int end);
-
   /** Log InsertTabletNode */
   WALFlushListener log(long memTableId, InsertTabletNode insertTabletNode, int start, int end);
 
-  /** Log DeletePlan */
-  WALFlushListener log(long memTableId, DeletePlan deletePlan);
-
   /** Log DeleteDataNode */
   WALFlushListener log(long memTableId, DeleteDataNode deleteDataNode);
 
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/node/WALFakeNode.java b/server/src/main/java/org/apache/iotdb/db/wal/node/WALFakeNode.java
index e9cfb6d435..bfe20bd03d 100644
--- a/server/src/main/java/org/apache/iotdb/db/wal/node/WALFakeNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/node/WALFakeNode.java
@@ -22,9 +22,6 @@ import org.apache.iotdb.db.engine.memtable.IMemTable;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.DeleteDataNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.wal.exception.WALException;
 import org.apache.iotdb.db.wal.utils.listener.WALFlushListener;
 
@@ -42,33 +39,17 @@ public class WALFakeNode implements IWALNode {
     this.cause = cause;
   }
 
-  @Override
-  public WALFlushListener log(long memTableId, InsertRowPlan insertRowPlan) {
-    return getResult();
-  }
-
   @Override
   public WALFlushListener log(long memTableId, InsertRowNode insertRowNode) {
     return getResult();
   }
 
-  @Override
-  public WALFlushListener log(
-      long memTableId, InsertTabletPlan insertTabletPlan, int start, int end) {
-    return getResult();
-  }
-
   @Override
   public WALFlushListener log(
       long memTableId, InsertTabletNode insertTabletNode, int start, int end) {
     return getResult();
   }
 
-  @Override
-  public WALFlushListener log(long memTableId, DeletePlan deletePlan) {
-    return getResult();
-  }
-
   @Override
   public WALFlushListener log(long memTableId, DeleteDataNode deleteDataNode) {
     return getResult();
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/node/WALNode.java b/server/src/main/java/org/apache/iotdb/db/wal/node/WALNode.java
index 941a1cb571..8abd338d34 100644
--- a/server/src/main/java/org/apache/iotdb/db/wal/node/WALNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/node/WALNode.java
@@ -20,28 +20,21 @@ package org.apache.iotdb.db.wal.node;
 
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.consensus.DataRegionId;
-import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.file.SystemFileFactory;
-import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.consensus.common.request.IConsensusRequest;
 import org.apache.iotdb.consensus.common.request.IndexedConsensusRequest;
 import org.apache.iotdb.consensus.common.request.MultiLeaderConsensusRequest;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.StorageEngine;
 import org.apache.iotdb.db.engine.StorageEngineV2;
 import org.apache.iotdb.db.engine.flush.FlushStatus;
 import org.apache.iotdb.db.engine.memtable.IMemTable;
 import org.apache.iotdb.db.engine.storagegroup.DataRegion;
-import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeType;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.DeleteDataNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.wal.WALManager;
 import org.apache.iotdb.db.wal.buffer.IWALBuffer;
 import org.apache.iotdb.db.wal.buffer.WALBuffer;
@@ -131,25 +124,12 @@ public class WALNode implements IWALNode {
     this.checkpointManager = new CheckpointManager(identifier, logDirectory);
   }
 
-  @Override
-  public WALFlushListener log(long memTableId, InsertRowPlan insertRowPlan) {
-    WALEntry walEntry = new WALInfoEntry(memTableId, insertRowPlan);
-    return log(walEntry);
-  }
-
   @Override
   public WALFlushListener log(long memTableId, InsertRowNode insertRowNode) {
     WALEntry walEntry = new WALInfoEntry(memTableId, insertRowNode);
     return log(walEntry);
   }
 
-  @Override
-  public WALFlushListener log(
-      long memTableId, InsertTabletPlan insertTabletPlan, int start, int end) {
-    WALEntry walEntry = new WALInfoEntry(memTableId, insertTabletPlan, start, end);
-    return log(walEntry);
-  }
-
   @Override
   public WALFlushListener log(
       long memTableId, InsertTabletNode insertTabletNode, int start, int end) {
@@ -157,12 +137,6 @@ public class WALNode implements IWALNode {
     return log(walEntry);
   }
 
-  @Override
-  public WALFlushListener log(long memTableId, DeletePlan deletePlan) {
-    WALEntry walEntry = new WALInfoEntry(memTableId, deletePlan);
-    return log(walEntry);
-  }
-
   @Override
   public WALFlushListener log(long memTableId, DeleteDataNode deleteDataNode) {
     WALEntry walEntry = new WALInfoEntry(memTableId, deleteDataNode);
@@ -362,18 +336,10 @@ public class WALNode implements IWALNode {
           FSFactoryProducer.getFSFactory().getFile(oldestMemTableInfo.getTsFilePath());
       DataRegion dataRegion;
       try {
-        if (config.isMppMode()) {
-          dataRegion =
-              StorageEngineV2.getInstance()
-                  .getDataRegion(new DataRegionId(TsFileUtils.getDataRegionId(oldestTsFile)));
-        } else {
-          dataRegion =
-              StorageEngine.getInstance()
-                  .getProcessorByDataRegionId(
-                      new PartialPath(TsFileUtils.getStorageGroup(oldestTsFile)),
-                      TsFileUtils.getDataRegionId(oldestTsFile));
-        }
-      } catch (IllegalPathException | StorageEngineException e) {
+        dataRegion =
+            StorageEngineV2.getInstance()
+                .getDataRegion(new DataRegionId(TsFileUtils.getDataRegionId(oldestTsFile)));
+      } catch (Exception e) {
         logger.error("Fail to get virtual storage group processor for {}", oldestTsFile, e);
         return false;
       }
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoer.java b/server/src/main/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoer.java
index eef04fe04e..9f992a9442 100644
--- a/server/src/main/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoer.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoer.java
@@ -19,7 +19,6 @@
 package org.apache.iotdb.db.wal.recover.file;
 
 import org.apache.iotdb.commons.conf.IoTDBConstant;
-import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.memtable.IMemTable;
@@ -29,7 +28,6 @@ import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.WriteProcessException;
 import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.idtable.IDTable;
 import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
 import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
@@ -38,11 +36,7 @@ import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.DeleteDataNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
-import org.apache.iotdb.db.service.IoTDB;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -72,24 +66,6 @@ public class TsFilePlanRedoer {
     this.idTable = idTable;
   }
 
-  void redoDelete(DeletePlan deletePlan) throws IOException, MetadataException {
-    List<PartialPath> paths = deletePlan.getPaths();
-    for (PartialPath path : paths) {
-      for (PartialPath device : IoTDB.schemaProcessor.getBelongedDevices(path)) {
-        recoveryMemTable.delete(
-            path, device, deletePlan.getDeleteStartTime(), deletePlan.getDeleteEndTime());
-      }
-      tsFileResource
-          .getModFile()
-          .write(
-              new Deletion(
-                  path,
-                  tsFileResource.getTsFileSize(),
-                  deletePlan.getDeleteStartTime(),
-                  deletePlan.getDeleteEndTime()));
-    }
-  }
-
   void redoDelete(DeleteDataNode deleteDataNode) throws IOException {
     List<PartialPath> paths = deleteDataNode.getPathList();
     for (PartialPath path : paths) {
@@ -110,55 +86,6 @@ public class TsFilePlanRedoer {
     }
   }
 
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  void redoInsert(InsertPlan plan) throws WriteProcessException, QueryProcessException {
-    if (!plan.hasValidMeasurements()) {
-      return;
-    }
-    if (tsFileResource != null) {
-      String deviceId =
-          plan.isAligned()
-              ? plan.getDevicePath().getDevicePath().getFullPath()
-              : plan.getDevicePath().getFullPath();
-      // orders of insert plan is guaranteed by storage engine, just check time in the file
-      // the last chunk group may contain the same data with the logs, ignore such logs in seq file
-      long lastEndTime = tsFileResource.getEndTime(deviceId);
-      if (lastEndTime != Long.MIN_VALUE && lastEndTime >= plan.getMinTime() && sequence) {
-        return;
-      }
-    }
-
-    plan.setMeasurementMNodes(new IMeasurementMNode[plan.getMeasurements().length]);
-    try {
-      if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()) {
-        idTable.getSeriesSchemas(plan);
-      } else {
-        IoTDB.schemaProcessor.getSeriesSchemasAndReadLockDevice(plan);
-        plan.setDeviceID(DeviceIDFactory.getInstance().getDeviceID(plan.getDevicePath()));
-      }
-    } catch (IOException | MetadataException e) {
-      throw new QueryProcessException("can't replay insert logs, ", e);
-    }
-
-    // mark failed plan manually
-    checkDataTypeAndMarkFailed(plan.getMeasurementMNodes(), plan);
-    if (plan instanceof InsertRowPlan) {
-      if (plan.isAligned()) {
-        recoveryMemTable.insertAlignedRow((InsertRowPlan) plan);
-      } else {
-        recoveryMemTable.insert((InsertRowPlan) plan);
-      }
-    } else {
-      if (plan.isAligned()) {
-        recoveryMemTable.insertAlignedTablet(
-            (InsertTabletPlan) plan, 0, ((InsertTabletPlan) plan).getRowCount());
-      } else {
-        recoveryMemTable.insertTablet(
-            (InsertTabletPlan) plan, 0, ((InsertTabletPlan) plan).getRowCount());
-      }
-    }
-  }
-
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   void redoInsert(InsertNode node) throws WriteProcessException {
     if (!node.hasValidMeasurements()) {
diff --git a/server/src/main/java/org/apache/iotdb/db/wal/recover/file/UnsealedTsFileRecoverPerformer.java b/server/src/main/java/org/apache/iotdb/db/wal/recover/file/UnsealedTsFileRecoverPerformer.java
index ab2e96ccd9..5fc5715362 100644
--- a/server/src/main/java/org/apache/iotdb/db/wal/recover/file/UnsealedTsFileRecoverPerformer.java
+++ b/server/src/main/java/org/apache/iotdb/db/wal/recover/file/UnsealedTsFileRecoverPerformer.java
@@ -31,8 +31,6 @@ import org.apache.iotdb.db.metadata.idtable.IDTable;
 import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.DeleteDataNode;
 import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertNode;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.wal.buffer.WALEntry;
 import org.apache.iotdb.db.wal.exception.WALRecoverException;
 import org.apache.iotdb.db.wal.utils.listener.WALRecoverListener;
@@ -187,13 +185,6 @@ public class UnsealedTsFileRecoverPerformer extends AbstractTsFileRecoverPerform
     }
     try {
       switch (walEntry.getType()) {
-        case INSERT_ROW_PLAN:
-        case INSERT_TABLET_PLAN:
-          walRedoer.redoInsert((InsertPlan) walEntry.getValue());
-          break;
-        case DELETE_PLAN:
-          walRedoer.redoDelete((DeletePlan) walEntry.getValue());
-          break;
         case MEMORY_TABLE_SNAPSHOT:
           IMemTable memTable = (IMemTable) walEntry.getValue();
           if (!memTable.isSignalMemTable()) {
@@ -207,6 +198,8 @@ public class UnsealedTsFileRecoverPerformer extends AbstractTsFileRecoverPerform
         case DELETE_DATA_NODE:
           walRedoer.redoDelete((DeleteDataNode) walEntry.getValue());
           break;
+        default:
+          throw new RuntimeException("Unsupported type " + walEntry.getType());
       }
     } catch (Exception e) {
       logger.warn("meet error when redo wal of {}", tsFileResource.getTsFile(), e);
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSeqCompactionTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSeqCompactionTest.java
index 095a6d260f..b8ae8b52b2 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSeqCompactionTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerSeqCompactionTest.java
@@ -1012,7 +1012,7 @@ public class InnerSeqCompactionTest {
             COMPACTION_TEST_SG);
     vsgp.getTsFileResourceManager().addAll(sourceResources, true);
     // delete data before compaction
-    vsgp.delete(new PartialPath(fullPaths[0]), 0, 1000, 0, null);
+    vsgp.deleteByDevice(new PartialPath(fullPaths[0]), 0, 1000, 0, null);
 
     InnerSpaceCompactionTask task =
         new InnerSpaceCompactionTask(
@@ -1026,8 +1026,8 @@ public class InnerSeqCompactionTest {
     task.setSourceFilesToCompactionCandidate();
     task.checkValidAndSetMerging();
     // delete data during compaction
-    vsgp.delete(new PartialPath(fullPaths[0]), 0, 1200, 0, null);
-    vsgp.delete(new PartialPath(fullPaths[0]), 0, 1800, 0, null);
+    vsgp.deleteByDevice(new PartialPath(fullPaths[0]), 0, 1200, 0, null);
+    vsgp.deleteByDevice(new PartialPath(fullPaths[0]), 0, 1800, 0, null);
     for (int i = 0; i < sourceResources.size() - 1; i++) {
       TsFileResource resource = sourceResources.get(i);
       resource.resetModFile();
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableFlushTaskTest.java b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableFlushTaskTest.java
index 8e86a5b70a..ca50c72466 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableFlushTaskTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableFlushTaskTest.java
@@ -22,6 +22,7 @@ import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.MetadataManagerHelper;
 import org.apache.iotdb.db.engine.flush.MemTableFlushTask;
+import org.apache.iotdb.db.exception.WriteProcessException;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -32,7 +33,6 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.IOException;
 import java.util.concurrent.ExecutionException;
 
 import static org.junit.Assert.assertEquals;
@@ -107,7 +107,7 @@ public class MemTableFlushTaskTest {
 
   @Test
   public void testFlushVectorMemTable()
-      throws ExecutionException, InterruptedException, IllegalPathException, IOException {
+      throws ExecutionException, InterruptedException, IllegalPathException, WriteProcessException {
     MemTableTestUtils.produceVectorData(memTable);
     MemTableFlushTask memTableFlushTask = new MemTableFlushTask(memTable, writer, storageGroup);
     assertTrue(
@@ -134,7 +134,7 @@ public class MemTableFlushTaskTest {
 
   @Test
   public void testFlushNullableVectorMemTable()
-      throws ExecutionException, InterruptedException, IllegalPathException, IOException {
+      throws ExecutionException, InterruptedException, IllegalPathException, WriteProcessException {
     MemTableTestUtils.produceNullableVectorData(memTable);
     MemTableFlushTask memTableFlushTask = new MemTableFlushTask(memTable, writer, storageGroup);
     assertTrue(
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableTestUtils.java b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableTestUtils.java
index 7b0a2085a8..0c88249821 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableTestUtils.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/memtable/MemTableTestUtils.java
@@ -20,21 +20,18 @@ package org.apache.iotdb.db.engine.memtable;
 
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.exception.WriteProcessException;
 import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.utils.BitMap;
-import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 import org.apache.iotdb.tsfile.write.schema.Schema;
 
-import java.util.ArrayList;
 import java.util.Collections;
-import java.util.List;
 
 public class MemTableTestUtils {
 
@@ -71,37 +68,18 @@ public class MemTableTestUtils {
     }
   }
 
-  public static void produceVectorData(IMemTable iMemTable) throws IllegalPathException {
-    iMemTable.write(genInsertTablePlan(), 1, 101);
+  public static void produceVectorData(IMemTable iMemTable)
+      throws IllegalPathException, WriteProcessException {
+    iMemTable.insertTablet(genInsertTableNode(), 1, 101);
   }
 
-  private static InsertTabletPlan genInsertTablePlan() throws IllegalPathException {
+  private static InsertTabletNode genInsertTableNode() throws IllegalPathException {
     String[] measurements = new String[2];
     measurements[0] = "sensor0";
     measurements[1] = "sensor1";
-
-    List<Integer> dataTypesList = new ArrayList<>();
     TSDataType[] dataTypes = new TSDataType[2];
-    dataTypesList.add(TSDataType.BOOLEAN.ordinal());
-    dataTypesList.add(TSDataType.INT64.ordinal());
     dataTypes[0] = TSDataType.BOOLEAN;
     dataTypes[1] = TSDataType.INT64;
-
-    TSEncoding[] encodings = new TSEncoding[2];
-    encodings[0] = TSEncoding.PLAIN;
-    encodings[1] = TSEncoding.GORILLA;
-
-    IMeasurementMNode[] mNodes = new IMeasurementMNode[2];
-    IMeasurementSchema schema0 = new MeasurementSchema(measurements[0], dataTypes[0], encodings[0]);
-    IMeasurementSchema schema1 = new MeasurementSchema(measurements[1], dataTypes[1], encodings[1]);
-    mNodes[0] = MeasurementMNode.getMeasurementMNode(null, "sensor0", schema0, null);
-    mNodes[1] = MeasurementMNode.getMeasurementMNode(null, "sensor1", schema1, null);
-
-    InsertTabletPlan insertTabletPlan =
-        new InsertTabletPlan(new PartialPath(deviceId0), measurements, dataTypesList);
-
-    insertTabletPlan.setAligned(true);
-
     long[] times = new long[101];
     Object[] columns = new Object[2];
     columns[0] = new boolean[101];
@@ -112,16 +90,31 @@ public class MemTableTestUtils {
       ((boolean[]) columns[0])[(int) r] = false;
       ((long[]) columns[1])[(int) r] = r;
     }
-    insertTabletPlan.setTimes(times);
-    insertTabletPlan.setColumns(columns);
-    insertTabletPlan.setRowCount(times.length);
-    insertTabletPlan.setMeasurementMNodes(mNodes);
+    TSEncoding[] encodings = new TSEncoding[2];
+    encodings[0] = TSEncoding.PLAIN;
+    encodings[1] = TSEncoding.GORILLA;
 
-    return insertTabletPlan;
+    MeasurementSchema[] schemas = new MeasurementSchema[2];
+    schemas[0] = new MeasurementSchema(measurements[0], dataTypes[0], encodings[0]);
+    schemas[1] = new MeasurementSchema(measurements[1], dataTypes[1], encodings[1]);
+    InsertTabletNode node =
+        new InsertTabletNode(
+            new PlanNodeId("0"),
+            new PartialPath(deviceId0),
+            true,
+            measurements,
+            dataTypes,
+            times,
+            null,
+            columns,
+            times.length);
+    node.setMeasurementSchemas(schemas);
+    return node;
   }
 
-  public static void produceNullableVectorData(IMemTable iMemTable) throws IllegalPathException {
-    InsertTabletPlan plan = genInsertTablePlan();
+  public static void produceNullableVectorData(IMemTable iMemTable)
+      throws IllegalPathException, WriteProcessException {
+    InsertTabletNode node = genInsertTableNode();
     BitMap[] bitMaps = new BitMap[2];
     bitMaps[1] = new BitMap(101);
     for (int r = 0; r < 101; r++) {
@@ -129,8 +122,8 @@ public class MemTableTestUtils {
         bitMaps[1].mark(r);
       }
     }
-    plan.setBitMaps(bitMaps);
-    iMemTable.write(plan, 1, 101);
+    node.setBitMaps(bitMaps);
+    iMemTable.insertTablet(node, 1, 101);
   }
 
   public static Schema getSchema() {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTableTest.java b/server/src/test/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTableTest.java
index aee61075b8..59a4a4046c 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTableTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTableTest.java
@@ -26,11 +26,11 @@ import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.engine.modification.Deletion;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
+import org.apache.iotdb.db.exception.WriteProcessException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode;
 import org.apache.iotdb.db.utils.MathUtils;
 import org.apache.iotdb.db.wal.utils.WALByteBufferForTest;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
@@ -377,8 +377,8 @@ public class PrimitiveMemTableTest {
   }
 
   private void writeVector(IMemTable memTable)
-      throws IOException, QueryProcessException, MetadataException {
-    memTable.writeAlignedTablet(genInsertTablePlan(), 0, 100);
+      throws IOException, QueryProcessException, MetadataException, WriteProcessException {
+    memTable.insertAlignedTablet(genInsertTableNode(), 0, 100);
 
     AlignedPath fullPath =
         new AlignedPath(
@@ -435,7 +435,8 @@ public class PrimitiveMemTableTest {
   }
 
   @Test
-  public void testAllType() throws IOException, QueryProcessException, MetadataException {
+  public void testAllType()
+      throws IOException, QueryProcessException, MetadataException, WriteProcessException {
     IMemTable memTable = new PrimitiveMemTable();
     int count = 10;
     String deviceId = "d1";
@@ -496,33 +497,14 @@ public class PrimitiveMemTableTest {
     return ret;
   }
 
-  private InsertTabletPlan genInsertTablePlan() throws IllegalPathException {
+  private static InsertTabletNode genInsertTableNode() throws IllegalPathException {
     String[] measurements = new String[2];
     measurements[0] = "sensor0";
     measurements[1] = "sensor1";
-
-    List<Integer> dataTypesList = new ArrayList<>();
+    String deviceId = "root.sg.device5";
     TSDataType[] dataTypes = new TSDataType[2];
-    dataTypesList.add(TSDataType.BOOLEAN.ordinal());
-    dataTypesList.add(TSDataType.INT64.ordinal());
     dataTypes[0] = TSDataType.BOOLEAN;
     dataTypes[1] = TSDataType.INT64;
-
-    TSEncoding[] encodings = new TSEncoding[2];
-    encodings[0] = TSEncoding.PLAIN;
-    encodings[1] = TSEncoding.GORILLA;
-
-    String deviceId = "root.sg.device5";
-
-    IMeasurementMNode[] mNodes = new IMeasurementMNode[2];
-    IMeasurementSchema schema0 = new MeasurementSchema(measurements[0], dataTypes[0], encodings[0]);
-    IMeasurementSchema schema1 = new MeasurementSchema(measurements[1], dataTypes[1], encodings[1]);
-    mNodes[0] = MeasurementMNode.getMeasurementMNode(null, "sensor0", schema0, null);
-    mNodes[1] = MeasurementMNode.getMeasurementMNode(null, "sensor1", schema1, null);
-
-    InsertTabletPlan insertTabletPlan =
-        new InsertTabletPlan(new PartialPath(deviceId), measurements, dataTypesList);
-
     long[] times = new long[100];
     Object[] columns = new Object[2];
     columns[0] = new boolean[100];
@@ -533,17 +515,32 @@ public class PrimitiveMemTableTest {
       ((boolean[]) columns[0])[(int) r] = false;
       ((long[]) columns[1])[(int) r] = r;
     }
-    insertTabletPlan.setTimes(times);
-    insertTabletPlan.setColumns(columns);
-    insertTabletPlan.setRowCount(times.length);
-    insertTabletPlan.setMeasurementMNodes(mNodes);
-    insertTabletPlan.setAligned(true);
+    TSEncoding[] encodings = new TSEncoding[2];
+    encodings[0] = TSEncoding.PLAIN;
+    encodings[1] = TSEncoding.GORILLA;
 
-    return insertTabletPlan;
+    MeasurementSchema[] schemas = new MeasurementSchema[2];
+    schemas[0] = new MeasurementSchema(measurements[0], dataTypes[0], encodings[0]);
+    schemas[1] = new MeasurementSchema(measurements[1], dataTypes[1], encodings[1]);
+
+    InsertTabletNode node =
+        new InsertTabletNode(
+            new PlanNodeId("0"),
+            new PartialPath(deviceId),
+            true,
+            measurements,
+            dataTypes,
+            times,
+            null,
+            columns,
+            times.length);
+    node.setMeasurementSchemas(schemas);
+    return node;
   }
 
   @Test
-  public void testSerializeSize() throws IOException, QueryProcessException, MetadataException {
+  public void testSerializeSize()
+      throws IOException, QueryProcessException, MetadataException, WriteProcessException {
     IMemTable memTable = new PrimitiveMemTable();
     int count = 10;
     String deviceId = "d1";
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/DataRegionTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/DataRegionTest.java
index 953faa31b3..c19fad496c 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/DataRegionTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/DataRegionTest.java
@@ -20,6 +20,7 @@
 package org.apache.iotdb.db.engine.storagegroup;
 
 import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.consensus.DataRegionId;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.exception.ShutdownException;
@@ -29,7 +30,7 @@ import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.constant.TestConstant;
 import org.apache.iotdb.db.engine.MetadataManagerHelper;
-import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.StorageEngineV2;
 import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
 import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTask;
 import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
@@ -61,6 +62,7 @@ import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -88,12 +90,16 @@ public class DataRegionTest {
     MetadataManagerHelper.initMetadata();
     EnvironmentUtils.envSetUp();
     dataRegion = new DummyDataRegion(systemDir, storageGroup);
+    StorageEngineV2.getInstance().setDataRegion(new DataRegionId(0), dataRegion);
     CompactionTaskManager.getInstance().start();
   }
 
   @After
   public void tearDown() throws Exception {
-    dataRegion.syncDeleteDataFiles();
+    if (dataRegion != null) {
+      dataRegion.syncDeleteDataFiles();
+      StorageEngineV2.getInstance().deleteDataRegion(new DataRegionId(0));
+    }
     EnvironmentUtils.cleanEnv();
     EnvironmentUtils.cleanDir(TestConstant.OUTPUT_DATA_DIR);
     CompactionTaskManager.getInstance().stop();
@@ -167,7 +173,7 @@ public class DataRegionTest {
                 CompressionType.UNCOMPRESSED,
                 Collections.emptyMap()));
 
-    dataRegion.delete(new PartialPath(deviceId, measurementId), 0, 15L, -1, null);
+    dataRegion.deleteByDevice(new PartialPath(deviceId, measurementId), 0, 15L, -1, null);
 
     List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
     for (TsFileProcessor tsfileProcessor : dataRegion.getWorkUnsequenceTsFileProcessors()) {
@@ -808,6 +814,7 @@ public class DataRegionTest {
         .setEnableUnseqSpaceCompaction(originEnableUnseqSpaceCompaction);
   }
 
+  @Ignore
   @Test
   public void testDeleteStorageGroupWhenCompacting() throws Exception {
     IoTDBDescriptor.getInstance().getConfig().setMaxInnerCompactionCandidateFileNum(10);
@@ -830,7 +837,17 @@ public class DataRegionTest {
               0);
       CompactionTaskManager.getInstance().addTaskToWaitingQueue(task);
       Thread.sleep(20);
-      StorageEngine.getInstance().deleteStorageGroup(new PartialPath(storageGroup));
+      List<DataRegion> dataRegions = StorageEngineV2.getInstance().getAllDataRegions();
+      List<DataRegion> regionsToBeDeleted = new ArrayList<>();
+      for (DataRegion region : dataRegions) {
+        if (region.getStorageGroupName().equals(storageGroup)) {
+          regionsToBeDeleted.add(region);
+        }
+      }
+      for (DataRegion region : regionsToBeDeleted) {
+        StorageEngineV2.getInstance()
+            .deleteDataRegion(new DataRegionId(Integer.parseInt(region.getDataRegionId())));
+      }
       Thread.sleep(500);
 
       for (TsFileResource resource : dataRegion.getSequenceFileList()) {
@@ -870,7 +887,7 @@ public class DataRegionTest {
     long preFLushInterval = config.getSeqMemtableFlushInterval();
     config.setEnableTimedFlushSeqMemtable(true);
     config.setSeqMemtableFlushInterval(5);
-    StorageEngine.getInstance().rebootTimedService();
+    StorageEngineV2.getInstance().rebootTimedService();
 
     Thread.sleep(500);
 
@@ -926,7 +943,7 @@ public class DataRegionTest {
     long preFLushInterval = config.getUnseqMemtableFlushInterval();
     config.setEnableTimedFlushUnseqMemtable(true);
     config.setUnseqMemtableFlushInterval(5);
-    StorageEngine.getInstance().rebootTimedService();
+    StorageEngineV2.getInstance().rebootTimedService();
 
     Thread.sleep(500);
 
@@ -993,10 +1010,10 @@ public class DataRegionTest {
     }
 
     // delete root.vehicle.d2.s0 data in the second file
-    dataRegion.delete(new PartialPath("root.vehicle.d2.s0"), 50, 150, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d2.s0"), 50, 150, 0, null);
 
     // delete root.vehicle.d2.s0 data in the third file
-    dataRegion.delete(new PartialPath("root.vehicle.d2.s0"), 150, 450, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d2.s0"), 150, 450, 0, null);
 
     for (int i = 0; i < dataRegion.getSequenceFileList().size(); i++) {
       TsFileResource resource = dataRegion.getSequenceFileList().get(i);
@@ -1011,7 +1028,17 @@ public class DataRegionTest {
       }
     }
 
-    StorageEngine.getInstance().deleteStorageGroup(new PartialPath(storageGroup));
+    List<DataRegion> dataRegions = StorageEngineV2.getInstance().getAllDataRegions();
+    List<DataRegion> regionsToBeDeleted = new ArrayList<>();
+    for (DataRegion region : dataRegions) {
+      if (region.getStorageGroupName().equals(storageGroup)) {
+        regionsToBeDeleted.add(region);
+      }
+    }
+    for (DataRegion region : regionsToBeDeleted) {
+      StorageEngineV2.getInstance()
+          .deleteDataRegion(new DataRegionId(Integer.parseInt(region.getDataRegionId())));
+    }
     Thread.sleep(500);
 
     for (TsFileResource resource : dataRegion.getSequenceFileList()) {
@@ -1032,10 +1059,10 @@ public class DataRegionTest {
     tsFileProcessor.getFlushingMemTable().addLast(tsFileProcessor.getWorkMemTable());
 
     // delete data which is in memtable
-    dataRegion.delete(new PartialPath("root.vehicle.d2.s0"), 50, 70, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d2.s0"), 50, 70, 0, null);
 
     // delete data which is not in memtable
-    dataRegion.delete(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0, null);
 
     dataRegion.syncCloseAllWorkingTsFileProcessors();
     Assert.assertFalse(tsFileResource.getModFile().exists());
@@ -1054,13 +1081,13 @@ public class DataRegionTest {
     tsFileProcessor.getFlushingMemTable().addLast(tsFileProcessor.getWorkMemTable());
 
     // delete data which is not in flushing memtable
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 50, 99, 0, null);
-    dataRegion.delete(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 99, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0, null);
 
     // delete data which is in flushing memtable
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 50, 100, 0, null);
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 50, 150, 0, null);
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 100, 300, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 100, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 150, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 100, 300, 0, null);
 
     dataRegion.syncCloseAllWorkingTsFileProcessors();
     Assert.assertTrue(tsFileResource.getModFile().exists());
@@ -1078,13 +1105,13 @@ public class DataRegionTest {
     TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0);
 
     // delete data which is not in work memtable
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 50, 99, 0, null);
-    dataRegion.delete(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 99, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0, null);
 
     // delete data which is in work memtable
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 50, 100, 0, null);
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 50, 150, 0, null);
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 100, 300, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 100, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 150, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 100, 300, 0, null);
 
     dataRegion.syncCloseAllWorkingTsFileProcessors();
     Assert.assertFalse(tsFileResource.getModFile().exists());
@@ -1096,11 +1123,11 @@ public class DataRegionTest {
       dataRegion.insert(buildInsertRowNodeByTSRecord(record));
     }
     // delete data which is not in work memtable
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 200, 299, 0, null);
-    dataRegion.delete(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 200, 299, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0, null);
 
     // delete data which is in work memtable
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 80, 85, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 80, 85, 0, null);
 
     Assert.assertFalse(tsFileResource.getModFile().exists());
 
@@ -1109,14 +1136,14 @@ public class DataRegionTest {
     tsFileProcessor.getFlushingMemTable().addLast(tsFileProcessor.getWorkMemTable());
 
     // delete data which is not in flushing memtable
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 0, 49, 0, null);
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 100, 200, 0, null);
-    dataRegion.delete(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 0, 49, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 100, 200, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0, null);
 
     // delete data which is in flushing memtable
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 25, 50, 0, null);
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 50, 80, 0, null);
-    dataRegion.delete(new PartialPath("root.vehicle.d0.s0"), 99, 150, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 25, 50, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 80, 0, null);
+    dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 99, 150, 0, null);
 
     dataRegion.syncCloseAllWorkingTsFileProcessors();
     Assert.assertTrue(tsFileResource.getModFile().exists());
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java
deleted file mode 100644
index 527a30be24..0000000000
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.engine.storagegroup;
-
-import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.db.utils.RandomNum;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.write.record.TSRecord;
-import org.apache.iotdb.tsfile.write.record.datapoint.LongDataPoint;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicLong;
-
-/** Bench The storage group manager with mul-thread and get its performance. */
-public class FileNodeManagerBenchmark {
-
-  private static int numOfWorker = 10;
-  private static int numOfDevice = 10;
-  private static int numOfMeasurement = 10;
-  private static long numOfTotalLine = 10000000;
-  private static CountDownLatch latch = new CountDownLatch(numOfWorker);
-  private static AtomicLong atomicLong = new AtomicLong();
-
-  private static String[] devices = new String[numOfDevice];
-  private static String prefix = "root.bench";
-  private static String[] measurements = new String[numOfMeasurement];
-
-  static {
-    for (int i = 0; i < numOfDevice; i++) {
-      devices[i] = prefix + TsFileConstant.PATH_SEPARATOR + "device_" + i;
-    }
-  }
-
-  static {
-    for (int i = 0; i < numOfMeasurement; i++) {
-      measurements[i] = "measurement_" + i;
-    }
-  }
-
-  private static void prepare() throws MetadataException {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-    schemaProcessor.setStorageGroup(new PartialPath(prefix));
-    for (String device : devices) {
-      for (String measurement : measurements) {
-        schemaProcessor.createTimeseries(
-            new PartialPath(device + "." + measurement),
-            TSDataType.INT64,
-            TSEncoding.PLAIN,
-            TSFileDescriptor.getInstance().getConfig().getCompressor(),
-            Collections.emptyMap());
-      }
-    }
-  }
-
-  private static void tearDown() throws IOException, StorageEngineException {
-    EnvironmentUtils.cleanEnv();
-  }
-
-  public static void main(String[] args)
-      throws InterruptedException, IOException, MetadataException, StorageEngineException {
-    tearDown();
-    prepare();
-    long startTime = System.currentTimeMillis();
-    for (int i = 0; i < numOfWorker; i++) {
-      Worker worker = new Worker();
-      worker.start();
-    }
-    latch.await();
-    long endTime = System.currentTimeMillis();
-    System.out.println("Elapsed time: " + (endTime - startTime) + "ms");
-    tearDown();
-  }
-
-  private static TSRecord getRecord(String deltaObjectId, long timestamp) {
-    TSRecord tsRecord = new TSRecord(timestamp, deltaObjectId);
-    for (String measurement : measurements) {
-      tsRecord.addTuple(new LongDataPoint(measurement, timestamp));
-    }
-    return tsRecord;
-  }
-
-  private static class Worker extends Thread {
-
-    @Override
-    public void run() {
-      try {
-        while (true) {
-          long seed = atomicLong.addAndGet(1);
-          if (seed > numOfTotalLine) {
-            break;
-          }
-          long time = RandomNum.getRandomLong(1, seed);
-          String deltaObject = devices[(int) (time % numOfDevice)];
-          TSRecord tsRecord = getRecord(deltaObject, time);
-          StorageEngine.getInstance().insert(new InsertRowPlan(tsRecord));
-        }
-      } catch (StorageEngineException | MetadataException e) {
-        e.printStackTrace();
-      } finally {
-        latch.countDown();
-      }
-    }
-  }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
deleted file mode 100644
index faf2508aa1..0000000000
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessorTest.java
+++ /dev/null
@@ -1,871 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.engine.storagegroup;
-
-import org.apache.iotdb.commons.conf.CommonDescriptor;
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.commons.exception.ShutdownException;
-import org.apache.iotdb.commons.path.MeasurementPath;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.conf.IoTDBConfig;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.MetadataManagerHelper;
-import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.engine.compaction.CompactionTaskManager;
-import org.apache.iotdb.db.engine.compaction.inner.InnerSpaceCompactionTask;
-import org.apache.iotdb.db.engine.compaction.log.CompactionLogger;
-import org.apache.iotdb.db.engine.compaction.performer.impl.ReadChunkCompactionPerformer;
-import org.apache.iotdb.db.engine.compaction.utils.CompactionConfigRestorer;
-import org.apache.iotdb.db.engine.flush.FlushManager;
-import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy;
-import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
-import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
-import org.apache.iotdb.db.exception.DataRegionException;
-import org.apache.iotdb.db.exception.TriggerExecutionException;
-import org.apache.iotdb.db.exception.WriteProcessException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
-import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.rescon.MemTableManager;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.read.TimeValuePair;
-import org.apache.iotdb.tsfile.read.reader.IPointReader;
-import org.apache.iotdb.tsfile.write.record.TSRecord;
-import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class StorageGroupProcessorTest {
-  private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
-  private static Logger logger = LoggerFactory.getLogger(StorageGroupProcessorTest.class);
-
-  private String storageGroup = "root.vehicle.d0";
-  private String systemDir = TestConstant.OUTPUT_DATA_DIR.concat("info");
-  private String deviceId = "root.vehicle.d0";
-  private String measurementId = "s0";
-  private DataRegion processor;
-  private QueryContext context = EnvironmentUtils.TEST_QUERY_CONTEXT;
-
-  @Before
-  public void setUp() throws Exception {
-    MetadataManagerHelper.initMetadata();
-    EnvironmentUtils.envSetUp();
-    processor = new DummySGP(systemDir, storageGroup);
-    CompactionTaskManager.getInstance().start();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    processor.syncDeleteDataFiles();
-    EnvironmentUtils.cleanEnv();
-    EnvironmentUtils.cleanDir(TestConstant.OUTPUT_DATA_DIR);
-    CompactionTaskManager.getInstance().stop();
-    EnvironmentUtils.cleanEnv();
-  }
-
-  private void insertToStorageGroupProcessor(TSRecord record)
-      throws WriteProcessException, IllegalPathException, TriggerExecutionException {
-    InsertRowPlan insertRowPlan = new InsertRowPlan(record);
-    processor.insert(insertRowPlan);
-  }
-
-  @Test
-  public void testUnseqUnsealedDelete()
-      throws WriteProcessException, IOException, MetadataException, TriggerExecutionException {
-    TSRecord record = new TSRecord(10000, deviceId);
-    record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(1000)));
-    processor.insert(new InsertRowPlan(record));
-    processor.syncCloseAllWorkingTsFileProcessors();
-
-    for (int j = 1; j <= 10; j++) {
-      record = new TSRecord(j, deviceId);
-      record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j)));
-      processor.insert(new InsertRowPlan(record));
-    }
-
-    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessors()) {
-      tsfileProcessor.syncFlush();
-    }
-
-    for (int j = 11; j <= 20; j++) {
-      record = new TSRecord(j, deviceId);
-      record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j)));
-      processor.insert(new InsertRowPlan(record));
-    }
-
-    PartialPath fullPath =
-        new MeasurementPath(
-            deviceId,
-            measurementId,
-            new MeasurementSchema(
-                measurementId,
-                TSDataType.INT32,
-                TSEncoding.RLE,
-                CompressionType.UNCOMPRESSED,
-                Collections.emptyMap()));
-
-    processor.delete(new PartialPath(deviceId, measurementId), 0, 15L, -1, null);
-
-    List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
-    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessors()) {
-      tsfileProcessor.query(
-          Collections.singletonList(fullPath),
-          EnvironmentUtils.TEST_QUERY_CONTEXT,
-          tsfileResourcesForQuery);
-    }
-
-    Assert.assertEquals(1, tsfileResourcesForQuery.size());
-    List<ReadOnlyMemChunk> memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath);
-    long time = 16;
-    for (ReadOnlyMemChunk memChunk : memChunks) {
-      IPointReader iterator = memChunk.getPointReader();
-      while (iterator.hasNextTimeValuePair()) {
-        TimeValuePair timeValuePair = iterator.nextTimeValuePair();
-        Assert.assertEquals(time++, timeValuePair.getTimestamp());
-      }
-    }
-  }
-
-  @Test
-  public void testSequenceSyncClose()
-      throws WriteProcessException, QueryProcessException, IllegalPathException,
-          TriggerExecutionException {
-    for (int j = 1; j <= 10; j++) {
-      TSRecord record = new TSRecord(j, deviceId);
-      record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j)));
-      processor.insert(new InsertRowPlan(record));
-      processor.asyncCloseAllWorkingTsFileProcessors();
-    }
-
-    try {
-      Thread.sleep(1000);
-    } catch (InterruptedException e) {
-      e.printStackTrace();
-    }
-    processor.syncCloseAllWorkingTsFileProcessors();
-    QueryDataSource queryDataSource =
-        processor.query(
-            Collections.singletonList(new PartialPath(deviceId, measurementId)),
-            deviceId,
-            context,
-            null,
-            null);
-    Assert.assertEquals(10, queryDataSource.getSeqResources().size());
-    for (TsFileResource resource : queryDataSource.getSeqResources()) {
-      Assert.assertTrue(resource.isClosed());
-    }
-  }
-
-  @Test
-  public void testInsertDataAndRemovePartitionAndInsert()
-      throws WriteProcessException, QueryProcessException, IllegalPathException,
-          TriggerExecutionException {
-    for (int j = 0; j < 10; j++) {
-      TSRecord record = new TSRecord(j, deviceId);
-      record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j)));
-      processor.insert(new InsertRowPlan(record));
-      processor.asyncCloseAllWorkingTsFileProcessors();
-    }
-    processor.syncCloseAllWorkingTsFileProcessors();
-
-    processor.removePartitions((storageGroupName, timePartitionId) -> true);
-
-    for (int j = 0; j < 10; j++) {
-      TSRecord record = new TSRecord(j, deviceId);
-      record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j)));
-      processor.insert(new InsertRowPlan(record));
-      processor.asyncCloseAllWorkingTsFileProcessors();
-    }
-    processor.syncCloseAllWorkingTsFileProcessors();
-
-    QueryDataSource queryDataSource =
-        processor.query(
-            Collections.singletonList(new PartialPath(deviceId, measurementId)),
-            deviceId,
-            context,
-            null,
-            null);
-    Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
-  }
-
-  @Test
-  public void testIoTDBTabletWriteAndSyncClose()
-      throws QueryProcessException, IllegalPathException, TriggerExecutionException {
-    String[] measurements = new String[2];
-    measurements[0] = "s0";
-    measurements[1] = "s1";
-    List<Integer> dataTypes = new ArrayList<>();
-    dataTypes.add(TSDataType.INT32.ordinal());
-    dataTypes.add(TSDataType.INT64.ordinal());
-
-    IMeasurementMNode[] measurementMNodes = new IMeasurementMNode[2];
-    measurementMNodes[0] =
-        MeasurementMNode.getMeasurementMNode(
-            null, "s0", new MeasurementSchema("s0", TSDataType.INT32, TSEncoding.PLAIN), null);
-    measurementMNodes[1] =
-        MeasurementMNode.getMeasurementMNode(
-            null, "s1", new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.PLAIN), null);
-
-    InsertTabletPlan insertTabletPlan1 =
-        new InsertTabletPlan(new PartialPath("root.vehicle.d0"), measurements, dataTypes);
-    insertTabletPlan1.setMeasurementMNodes(measurementMNodes);
-
-    long[] times = new long[100];
-    Object[] columns = new Object[2];
-    columns[0] = new int[100];
-    columns[1] = new long[100];
-
-    for (int r = 0; r < 100; r++) {
-      times[r] = r;
-      ((int[]) columns[0])[r] = 1;
-      ((long[]) columns[1])[r] = 1;
-    }
-    insertTabletPlan1.setTimes(times);
-    insertTabletPlan1.setColumns(columns);
-    insertTabletPlan1.setRowCount(times.length);
-
-    processor.insertTablet(insertTabletPlan1);
-    processor.asyncCloseAllWorkingTsFileProcessors();
-
-    InsertTabletPlan insertTabletPlan2 =
-        new InsertTabletPlan(new PartialPath("root.vehicle.d0"), measurements, dataTypes);
-    insertTabletPlan2.setMeasurementMNodes(measurementMNodes);
-
-    for (int r = 50; r < 149; r++) {
-      times[r - 50] = r;
-      ((int[]) columns[0])[r - 50] = 1;
-      ((long[]) columns[1])[r - 50] = 1;
-    }
-    insertTabletPlan2.setTimes(times);
-    insertTabletPlan2.setColumns(columns);
-    insertTabletPlan2.setRowCount(times.length);
-
-    processor.insertTablet(insertTabletPlan2);
-    processor.asyncCloseAllWorkingTsFileProcessors();
-    processor.syncCloseAllWorkingTsFileProcessors();
-
-    QueryDataSource queryDataSource =
-        processor.query(
-            Collections.singletonList(new PartialPath(deviceId, measurementId)),
-            deviceId,
-            context,
-            null,
-            null);
-
-    Assert.assertEquals(2, queryDataSource.getSeqResources().size());
-    Assert.assertEquals(1, queryDataSource.getUnseqResources().size());
-    for (TsFileResource resource : queryDataSource.getSeqResources()) {
-      Assert.assertTrue(resource.isClosed());
-    }
-  }
-
-  @Test
-  public void testSeqAndUnSeqSyncClose()
-      throws WriteProcessException, QueryProcessException, IllegalPathException,
-          TriggerExecutionException {
-    for (int j = 21; j <= 30; j++) {
-      TSRecord record = new TSRecord(j, deviceId);
-      record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j)));
-      processor.insert(new InsertRowPlan(record));
-      processor.asyncCloseAllWorkingTsFileProcessors();
-    }
-    processor.syncCloseAllWorkingTsFileProcessors();
-
-    for (int j = 10; j >= 1; j--) {
-      TSRecord record = new TSRecord(j, deviceId);
-      record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j)));
-      processor.insert(new InsertRowPlan(record));
-      processor.asyncCloseAllWorkingTsFileProcessors();
-    }
-
-    processor.syncCloseAllWorkingTsFileProcessors();
-
-    QueryDataSource queryDataSource =
-        processor.query(
-            Collections.singletonList(new PartialPath(deviceId, measurementId)),
-            deviceId,
-            context,
-            null,
-            null);
-    Assert.assertEquals(10, queryDataSource.getSeqResources().size());
-    Assert.assertEquals(10, queryDataSource.getUnseqResources().size());
-    for (TsFileResource resource : queryDataSource.getSeqResources()) {
-      Assert.assertTrue(resource.isClosed());
-    }
-    for (TsFileResource resource : queryDataSource.getUnseqResources()) {
-      Assert.assertTrue(resource.isClosed());
-    }
-  }
-
-  @Test
-  public void testEnableDiscardOutOfOrderDataForInsertRowPlan()
-      throws WriteProcessException, QueryProcessException, IllegalPathException, IOException,
-          TriggerExecutionException {
-    boolean defaultValue = config.isEnableDiscardOutOfOrderData();
-    config.setEnableDiscardOutOfOrderData(true);
-
-    for (int j = 21; j <= 30; j++) {
-      TSRecord record = new TSRecord(j, deviceId);
-      record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j)));
-      insertToStorageGroupProcessor(record);
-      processor.asyncCloseAllWorkingTsFileProcessors();
-    }
-    processor.syncCloseAllWorkingTsFileProcessors();
-
-    for (int j = 10; j >= 1; j--) {
-      TSRecord record = new TSRecord(j, deviceId);
-      record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j)));
-      insertToStorageGroupProcessor(record);
-      processor.asyncCloseAllWorkingTsFileProcessors();
-    }
-
-    processor.syncCloseAllWorkingTsFileProcessors();
-
-    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessors()) {
-      tsfileProcessor.syncFlush();
-    }
-
-    QueryDataSource queryDataSource =
-        processor.query(
-            Collections.singletonList(new PartialPath(deviceId, measurementId)),
-            deviceId,
-            context,
-            null,
-            null);
-    Assert.assertEquals(10, queryDataSource.getSeqResources().size());
-    Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
-    for (TsFileResource resource : queryDataSource.getSeqResources()) {
-      Assert.assertTrue(resource.isClosed());
-    }
-    for (TsFileResource resource : queryDataSource.getUnseqResources()) {
-      Assert.assertTrue(resource.isClosed());
-    }
-
-    config.setEnableDiscardOutOfOrderData(defaultValue);
-  }
-
-  @Test
-  public void testEnableDiscardOutOfOrderDataForInsertTablet1()
-      throws QueryProcessException, IllegalPathException, IOException, TriggerExecutionException {
-    boolean defaultEnableDiscard = config.isEnableDiscardOutOfOrderData();
-    long defaultTimePartition = config.getTimePartitionIntervalForStorage();
-    boolean defaultEnablePartition = config.isEnablePartition();
-    config.setEnableDiscardOutOfOrderData(true);
-    config.setEnablePartition(true);
-    config.setTimePartitionIntervalForStorage(100000);
-
-    String[] measurements = new String[2];
-    measurements[0] = "s0";
-    measurements[1] = "s1";
-    List<Integer> dataTypes = new ArrayList<>();
-    dataTypes.add(TSDataType.INT32.ordinal());
-    dataTypes.add(TSDataType.INT64.ordinal());
-
-    IMeasurementMNode[] measurementMNodes = new IMeasurementMNode[2];
-    measurementMNodes[0] =
-        MeasurementMNode.getMeasurementMNode(
-            null, "s0", new MeasurementSchema("s0", TSDataType.INT32, TSEncoding.PLAIN), null);
-    measurementMNodes[1] =
-        MeasurementMNode.getMeasurementMNode(
-            null, "s1", new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.PLAIN), null);
-
-    InsertTabletPlan insertTabletPlan1 =
-        new InsertTabletPlan(new PartialPath("root.vehicle.d0"), measurements, dataTypes);
-
-    long[] times = new long[100];
-    Object[] columns = new Object[2];
-    columns[0] = new int[100];
-    columns[1] = new long[100];
-
-    for (int r = 0; r < 100; r++) {
-      times[r] = r;
-      ((int[]) columns[0])[r] = 1;
-      ((long[]) columns[1])[r] = 1;
-    }
-    insertTabletPlan1.setTimes(times);
-    insertTabletPlan1.setColumns(columns);
-    insertTabletPlan1.setRowCount(times.length);
-    insertTabletPlan1.setMeasurementMNodes(measurementMNodes);
-
-    processor.insertTablet(insertTabletPlan1);
-    processor.asyncCloseAllWorkingTsFileProcessors();
-
-    InsertTabletPlan insertTabletPlan2 =
-        new InsertTabletPlan(new PartialPath("root.vehicle.d0"), measurements, dataTypes);
-
-    for (int r = 149; r >= 50; r--) {
-      times[r - 50] = r;
-      ((int[]) columns[0])[r - 50] = 1;
-      ((long[]) columns[1])[r - 50] = 1;
-    }
-    insertTabletPlan2.setTimes(times);
-    insertTabletPlan2.setColumns(columns);
-    insertTabletPlan2.setRowCount(times.length);
-    insertTabletPlan2.setMeasurementMNodes(measurementMNodes);
-
-    processor.insertTablet(insertTabletPlan2);
-    processor.asyncCloseAllWorkingTsFileProcessors();
-    processor.syncCloseAllWorkingTsFileProcessors();
-
-    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessors()) {
-      tsfileProcessor.syncFlush();
-    }
-
-    QueryDataSource queryDataSource =
-        processor.query(
-            Collections.singletonList(new PartialPath(deviceId, measurementId)),
-            deviceId,
-            context,
-            null,
-            null);
-
-    Assert.assertEquals(2, queryDataSource.getSeqResources().size());
-    Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
-    for (TsFileResource resource : queryDataSource.getSeqResources()) {
-      Assert.assertTrue(resource.isClosed());
-    }
-
-    config.setEnableDiscardOutOfOrderData(defaultEnableDiscard);
-    config.setTimePartitionIntervalForStorage(defaultTimePartition);
-    config.setEnablePartition(defaultEnablePartition);
-  }
-
-  @Test
-  public void testEnableDiscardOutOfOrderDataForInsertTablet2()
-      throws QueryProcessException, IllegalPathException, IOException, TriggerExecutionException {
-    boolean defaultEnableDiscard = config.isEnableDiscardOutOfOrderData();
-    long defaultTimePartition = config.getTimePartitionIntervalForStorage();
-    boolean defaultEnablePartition = config.isEnablePartition();
-    config.setEnableDiscardOutOfOrderData(true);
-    config.setEnablePartition(true);
-    config.setTimePartitionIntervalForStorage(1200000);
-
-    String[] measurements = new String[2];
-    measurements[0] = "s0";
-    measurements[1] = "s1";
-    List<Integer> dataTypes = new ArrayList<>();
-    dataTypes.add(TSDataType.INT32.ordinal());
-    dataTypes.add(TSDataType.INT64.ordinal());
-
-    IMeasurementMNode[] measurementMNodes = new IMeasurementMNode[2];
-    measurementMNodes[0] =
-        MeasurementMNode.getMeasurementMNode(
-            null, "s0", new MeasurementSchema("s0", TSDataType.INT32, TSEncoding.PLAIN), null);
-    measurementMNodes[1] =
-        MeasurementMNode.getMeasurementMNode(
-            null, "s1", new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.PLAIN), null);
-
-    InsertTabletPlan insertTabletPlan1 =
-        new InsertTabletPlan(new PartialPath("root.vehicle.d0"), measurements, dataTypes);
-
-    long[] times = new long[1200];
-    Object[] columns = new Object[2];
-    columns[0] = new int[1200];
-    columns[1] = new long[1200];
-
-    for (int r = 0; r < 1200; r++) {
-      times[r] = r;
-      ((int[]) columns[0])[r] = 1;
-      ((long[]) columns[1])[r] = 1;
-    }
-    insertTabletPlan1.setTimes(times);
-    insertTabletPlan1.setColumns(columns);
-    insertTabletPlan1.setRowCount(times.length);
-    insertTabletPlan1.setMeasurementMNodes(measurementMNodes);
-
-    processor.insertTablet(insertTabletPlan1);
-    processor.asyncCloseAllWorkingTsFileProcessors();
-
-    InsertTabletPlan insertTabletPlan2 =
-        new InsertTabletPlan(new PartialPath("root.vehicle.d0"), measurements, dataTypes);
-
-    for (int r = 1249; r >= 50; r--) {
-      times[r - 50] = r;
-      ((int[]) columns[0])[r - 50] = 1;
-      ((long[]) columns[1])[r - 50] = 1;
-    }
-    insertTabletPlan2.setTimes(times);
-    insertTabletPlan2.setColumns(columns);
-    insertTabletPlan2.setRowCount(times.length);
-    insertTabletPlan2.setMeasurementMNodes(measurementMNodes);
-
-    processor.insertTablet(insertTabletPlan2);
-    processor.asyncCloseAllWorkingTsFileProcessors();
-    processor.syncCloseAllWorkingTsFileProcessors();
-
-    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessors()) {
-      tsfileProcessor.syncFlush();
-    }
-
-    QueryDataSource queryDataSource =
-        processor.query(
-            Collections.singletonList(new PartialPath(deviceId, measurementId)),
-            deviceId,
-            context,
-            null,
-            null);
-
-    Assert.assertEquals(2, queryDataSource.getSeqResources().size());
-    Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
-    for (TsFileResource resource : queryDataSource.getSeqResources()) {
-      Assert.assertTrue(resource.isClosed());
-    }
-
-    config.setEnableDiscardOutOfOrderData(defaultEnableDiscard);
-    config.setTimePartitionIntervalForStorage(defaultTimePartition);
-    config.setEnablePartition(defaultEnablePartition);
-  }
-
-  @Test
-  public void testEnableDiscardOutOfOrderDataForInsertTablet3()
-      throws QueryProcessException, IllegalPathException, IOException, TriggerExecutionException {
-    boolean defaultEnableDiscard = config.isEnableDiscardOutOfOrderData();
-    long defaultTimePartition = config.getTimePartitionIntervalForStorage();
-    boolean defaultEnablePartition = config.isEnablePartition();
-    config.setEnableDiscardOutOfOrderData(true);
-    config.setEnablePartition(true);
-    config.setTimePartitionIntervalForStorage(1000000);
-
-    String[] measurements = new String[2];
-    measurements[0] = "s0";
-    measurements[1] = "s1";
-    List<Integer> dataTypes = new ArrayList<>();
-    dataTypes.add(TSDataType.INT32.ordinal());
-    dataTypes.add(TSDataType.INT64.ordinal());
-
-    IMeasurementMNode[] measurementMNodes = new IMeasurementMNode[2];
-    measurementMNodes[0] =
-        MeasurementMNode.getMeasurementMNode(
-            null, "s0", new MeasurementSchema("s0", TSDataType.INT32, TSEncoding.PLAIN), null);
-    measurementMNodes[1] =
-        MeasurementMNode.getMeasurementMNode(
-            null, "s1", new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.PLAIN), null);
-
-    InsertTabletPlan insertTabletPlan1 =
-        new InsertTabletPlan(new PartialPath("root.vehicle.d0"), measurements, dataTypes);
-
-    long[] times = new long[1200];
-    Object[] columns = new Object[2];
-    columns[0] = new int[1200];
-    columns[1] = new long[1200];
-
-    for (int r = 0; r < 1200; r++) {
-      times[r] = r;
-      ((int[]) columns[0])[r] = 1;
-      ((long[]) columns[1])[r] = 1;
-    }
-    insertTabletPlan1.setTimes(times);
-    insertTabletPlan1.setColumns(columns);
-    insertTabletPlan1.setRowCount(times.length);
-    insertTabletPlan1.setMeasurementMNodes(measurementMNodes);
-
-    processor.insertTablet(insertTabletPlan1);
-    processor.asyncCloseAllWorkingTsFileProcessors();
-
-    InsertTabletPlan insertTabletPlan2 =
-        new InsertTabletPlan(new PartialPath("root.vehicle.d0"), measurements, dataTypes);
-
-    for (int r = 1249; r >= 50; r--) {
-      times[r - 50] = r;
-      ((int[]) columns[0])[r - 50] = 1;
-      ((long[]) columns[1])[r - 50] = 1;
-    }
-    insertTabletPlan2.setTimes(times);
-    insertTabletPlan2.setColumns(columns);
-    insertTabletPlan2.setRowCount(times.length);
-    insertTabletPlan2.setMeasurementMNodes(measurementMNodes);
-
-    processor.insertTablet(insertTabletPlan2);
-    processor.asyncCloseAllWorkingTsFileProcessors();
-    processor.syncCloseAllWorkingTsFileProcessors();
-
-    for (TsFileProcessor tsfileProcessor : processor.getWorkUnsequenceTsFileProcessors()) {
-      tsfileProcessor.syncFlush();
-    }
-
-    QueryDataSource queryDataSource =
-        processor.query(
-            Collections.singletonList(new PartialPath(deviceId, measurementId)),
-            deviceId,
-            context,
-            null,
-            null);
-
-    Assert.assertEquals(2, queryDataSource.getSeqResources().size());
-    Assert.assertEquals(0, queryDataSource.getUnseqResources().size());
-    for (TsFileResource resource : queryDataSource.getSeqResources()) {
-      Assert.assertTrue(resource.isClosed());
-    }
-
-    config.setEnableDiscardOutOfOrderData(defaultEnableDiscard);
-    config.setTimePartitionIntervalForStorage(defaultTimePartition);
-    config.setEnablePartition(defaultEnablePartition);
-  }
-
-  @Test
-  public void testMerge()
-      throws WriteProcessException, QueryProcessException, IllegalPathException,
-          TriggerExecutionException {
-    int originCandidateFileNum =
-        IoTDBDescriptor.getInstance().getConfig().getMaxInnerCompactionCandidateFileNum();
-    IoTDBDescriptor.getInstance().getConfig().setMaxInnerCompactionCandidateFileNum(9);
-    boolean originEnableSeqSpaceCompaction =
-        IoTDBDescriptor.getInstance().getConfig().isEnableSeqSpaceCompaction();
-    boolean originEnableUnseqSpaceCompaction =
-        IoTDBDescriptor.getInstance().getConfig().isEnableUnseqSpaceCompaction();
-    IoTDBDescriptor.getInstance().getConfig().setEnableSeqSpaceCompaction(true);
-    IoTDBDescriptor.getInstance().getConfig().setEnableUnseqSpaceCompaction(true);
-    for (int j = 21; j <= 30; j++) {
-      TSRecord record = new TSRecord(j, deviceId);
-      record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j)));
-      processor.insert(new InsertRowPlan(record));
-      processor.asyncCloseAllWorkingTsFileProcessors();
-    }
-    processor.syncCloseAllWorkingTsFileProcessors();
-
-    for (int j = 10; j >= 1; j--) {
-      TSRecord record = new TSRecord(j, deviceId);
-      record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j)));
-      processor.insert(new InsertRowPlan(record));
-      processor.asyncCloseAllWorkingTsFileProcessors();
-    }
-
-    processor.syncCloseAllWorkingTsFileProcessors();
-    processor.compact();
-    long totalWaitingTime = 0;
-    do {
-      // wait
-      try {
-        Thread.sleep(100);
-      } catch (InterruptedException e) {
-        e.printStackTrace();
-      }
-      totalWaitingTime += 100;
-      if (totalWaitingTime % 1000 == 0) {
-        logger.warn("has waited for {} seconds", totalWaitingTime / 1000);
-      }
-      if (totalWaitingTime > 120_000) {
-        Assert.fail();
-        break;
-      }
-    } while (CompactionTaskManager.getInstance().getExecutingTaskCount() > 0);
-
-    QueryDataSource queryDataSource =
-        processor.query(
-            Collections.singletonList(new PartialPath(deviceId, measurementId)),
-            deviceId,
-            context,
-            null,
-            null);
-    Assert.assertEquals(2, queryDataSource.getSeqResources().size());
-    for (TsFileResource resource : queryDataSource.getSeqResources()) {
-      Assert.assertTrue(resource.isClosed());
-    }
-    for (TsFileResource resource : queryDataSource.getUnseqResources()) {
-      Assert.assertTrue(resource.isClosed());
-    }
-    IoTDBDescriptor.getInstance()
-        .getConfig()
-        .setMaxInnerCompactionCandidateFileNum(originCandidateFileNum);
-    IoTDBDescriptor.getInstance()
-        .getConfig()
-        .setEnableSeqSpaceCompaction(originEnableSeqSpaceCompaction);
-    IoTDBDescriptor.getInstance()
-        .getConfig()
-        .setEnableUnseqSpaceCompaction(originEnableUnseqSpaceCompaction);
-  }
-
-  @Test
-  public void testDeleteStorageGroupWhenCompacting() throws Exception {
-    IoTDBDescriptor.getInstance().getConfig().setMaxInnerCompactionCandidateFileNum(10);
-    try {
-      for (int j = 0; j < 10; j++) {
-        TSRecord record = new TSRecord(j, deviceId);
-        record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j)));
-        processor.insert(new InsertRowPlan(record));
-        processor.asyncCloseAllWorkingTsFileProcessors();
-      }
-      processor.syncCloseAllWorkingTsFileProcessors();
-      InnerSpaceCompactionTask task =
-          new InnerSpaceCompactionTask(
-              0,
-              processor.getTsFileManager(),
-              processor.getSequenceFileList(),
-              true,
-              new ReadChunkCompactionPerformer(processor.getSequenceFileList()),
-              new AtomicInteger(0),
-              0);
-      CompactionTaskManager.getInstance().addTaskToWaitingQueue(task);
-      Thread.sleep(20);
-      StorageEngine.getInstance().deleteStorageGroup(new PartialPath(storageGroup));
-      Thread.sleep(500);
-
-      for (TsFileResource resource : processor.getSequenceFileList()) {
-        Assert.assertFalse(resource.getTsFile().exists());
-      }
-      TsFileResource targetTsFileResource =
-          TsFileNameGenerator.getInnerCompactionTargetFileResource(
-              processor.getSequenceFileList(), true);
-      Assert.assertFalse(targetTsFileResource.getTsFile().exists());
-      String dataDirectory = targetTsFileResource.getTsFile().getParent();
-      File logFile =
-          new File(
-              dataDirectory
-                  + File.separator
-                  + targetTsFileResource.getTsFile().getName()
-                  + CompactionLogger.INNER_COMPACTION_LOG_NAME_SUFFIX);
-      Assert.assertFalse(logFile.exists());
-      Assert.assertFalse(CommonDescriptor.getInstance().getConfig().isReadOnly());
-      Assert.assertTrue(processor.getTsFileManager().isAllowCompaction());
-    } finally {
-      new CompactionConfigRestorer().restoreCompactionConfig();
-    }
-  }
-
-  @Test
-  public void testTimedFlushSeqMemTable()
-      throws IllegalPathException, InterruptedException, WriteProcessException,
-          TriggerExecutionException, ShutdownException {
-    // create one sequence memtable
-    TSRecord record = new TSRecord(10000, deviceId);
-    record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(1000)));
-    processor.insert(new InsertRowPlan(record));
-    Assert.assertEquals(1, MemTableManager.getInstance().getCurrentMemtableNumber());
-
-    // change config & reboot timed service
-    boolean prevEnableTimedFlushSeqMemtable = config.isEnableTimedFlushSeqMemtable();
-    long preFLushInterval = config.getSeqMemtableFlushInterval();
-    config.setEnableTimedFlushSeqMemtable(true);
-    config.setSeqMemtableFlushInterval(5);
-    StorageEngine.getInstance().rebootTimedService();
-
-    Thread.sleep(500);
-
-    Assert.assertEquals(1, processor.getWorkSequenceTsFileProcessors().size());
-    TsFileProcessor tsFileProcessor = processor.getWorkSequenceTsFileProcessors().iterator().next();
-    FlushManager flushManager = FlushManager.getInstance();
-
-    // flush the sequence memtable
-    processor.timedFlushSeqMemTable();
-
-    // wait until memtable flush task is done
-    int waitCnt = 0;
-    while (tsFileProcessor.getFlushingMemTableSize() != 0
-        || tsFileProcessor.isManagedByFlushManager()
-        || flushManager.getNumberOfPendingTasks() != 0
-        || flushManager.getNumberOfPendingSubTasks() != 0
-        || flushManager.getNumberOfWorkingTasks() != 0
-        || flushManager.getNumberOfWorkingSubTasks() != 0) {
-      Thread.sleep(500);
-      ++waitCnt;
-      if (waitCnt % 10 == 0) {
-        logger.info("already wait {} s", waitCnt / 2);
-      }
-    }
-
-    Assert.assertEquals(0, MemTableManager.getInstance().getCurrentMemtableNumber());
-
-    config.setEnableTimedFlushSeqMemtable(prevEnableTimedFlushSeqMemtable);
-    config.setSeqMemtableFlushInterval(preFLushInterval);
-  }
-
-  @Test
-  public void testTimedFlushUnseqMemTable()
-      throws IllegalPathException, InterruptedException, WriteProcessException,
-          TriggerExecutionException, ShutdownException {
-    // create one sequence memtable & close
-    TSRecord record = new TSRecord(10000, deviceId);
-    record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(1000)));
-    processor.insert(new InsertRowPlan(record));
-    Assert.assertEquals(1, MemTableManager.getInstance().getCurrentMemtableNumber());
-    processor.syncCloseAllWorkingTsFileProcessors();
-    Assert.assertEquals(0, MemTableManager.getInstance().getCurrentMemtableNumber());
-
-    // create one unsequence memtable
-    record = new TSRecord(1, deviceId);
-    record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(1000)));
-    processor.insert(new InsertRowPlan(record));
-    Assert.assertEquals(1, MemTableManager.getInstance().getCurrentMemtableNumber());
-
-    // change config & reboot timed service
-    boolean prevEnableTimedFlushUnseqMemtable = config.isEnableTimedFlushUnseqMemtable();
-    long preFLushInterval = config.getUnseqMemtableFlushInterval();
-    config.setEnableTimedFlushUnseqMemtable(true);
-    config.setUnseqMemtableFlushInterval(5);
-    StorageEngine.getInstance().rebootTimedService();
-
-    Thread.sleep(500);
-
-    Assert.assertEquals(1, processor.getWorkUnsequenceTsFileProcessors().size());
-    TsFileProcessor tsFileProcessor =
-        processor.getWorkUnsequenceTsFileProcessors().iterator().next();
-    FlushManager flushManager = FlushManager.getInstance();
-
-    // flush the unsequence memtable
-    processor.timedFlushUnseqMemTable();
-
-    // wait until memtable flush task is done
-    int waitCnt = 0;
-    while (tsFileProcessor.getFlushingMemTableSize() != 0
-        || tsFileProcessor.isManagedByFlushManager()
-        || flushManager.getNumberOfPendingTasks() != 0
-        || flushManager.getNumberOfPendingSubTasks() != 0
-        || flushManager.getNumberOfWorkingTasks() != 0
-        || flushManager.getNumberOfWorkingSubTasks() != 0) {
-      Thread.sleep(500);
-      ++waitCnt;
-      if (waitCnt % 10 == 0) {
-        logger.info("already wait {} s", waitCnt / 2);
-      }
-    }
-
-    Assert.assertEquals(0, MemTableManager.getInstance().getCurrentMemtableNumber());
-
-    config.setEnableTimedFlushUnseqMemtable(prevEnableTimedFlushUnseqMemtable);
-    config.setUnseqMemtableFlushInterval(preFLushInterval);
-  }
-
-  class DummySGP extends DataRegion {
-
-    DummySGP(String systemInfoDir, String storageGroupName) throws DataRegionException {
-      super(systemInfoDir, "0", new TsFileFlushPolicy.DirectFlushPolicy(), storageGroupName);
-    }
-  }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java
index 229865a9df..8f96c9223d 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java
@@ -35,26 +35,22 @@ import org.apache.iotdb.db.exception.TriggerExecutionException;
 import org.apache.iotdb.db.exception.WriteProcessException;
 import org.apache.iotdb.db.exception.query.OutOfTTLException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
-import org.apache.iotdb.db.qp.Planner;
-import org.apache.iotdb.db.qp.executor.PlanExecutor;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
-import org.apache.iotdb.db.qp.physical.sys.ShowTTLPlan;
+import org.apache.iotdb.db.mpp.plan.parser.StatementGenerator;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNodeId;
+import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.SetTTLStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowTTLStatement;
+import org.apache.iotdb.db.mpp.plan.statement.metadata.UnSetTTLStatement;
 import org.apache.iotdb.db.query.reader.series.SeriesRawDataBatchReader;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.db.utils.SchemaTestUtils;
 import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
-import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.common.BatchData;
-import org.apache.iotdb.tsfile.read.common.RowRecord;
-import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 import org.apache.iotdb.tsfile.read.reader.IBatchReader;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
@@ -64,6 +60,7 @@ import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
+import java.time.ZoneId;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
@@ -72,9 +69,7 @@ import java.util.Set;
 import java.util.stream.Collectors;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 public class TTLTest {
 
@@ -148,65 +143,64 @@ public class TTLTest {
   public void testTTLWrite()
       throws WriteProcessException, QueryProcessException, IllegalPathException,
           TriggerExecutionException {
-    InsertRowPlan plan = new InsertRowPlan();
-    plan.setDevicePath(new PartialPath(sg1));
-    plan.setTime(System.currentTimeMillis());
-    plan.setMeasurements(new String[] {"s1"});
-    plan.setDataTypes(new TSDataType[] {TSDataType.INT64});
-    plan.setValues(new Object[] {1L});
-    plan.setMeasurementMNodes(
-        new IMeasurementMNode[] {
-          MeasurementMNode.getMeasurementMNode(
-              null, "s1", new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.PLAIN), null)
-        });
-    plan.transferType();
+    InsertRowNode node =
+        new InsertRowNode(
+            new PlanNodeId("0"),
+            new PartialPath(sg1),
+            false,
+            new String[] {"s1"},
+            new TSDataType[] {TSDataType.INT64},
+            System.currentTimeMillis(),
+            new Object[] {1L},
+            false);
+    node.setMeasurementSchemas(
+        new MeasurementSchema[] {new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.PLAIN)});
 
     // ok without ttl
-    dataRegion.insert(plan);
+    dataRegion.insert(node);
 
     dataRegion.setDataTTL(1000);
     // with ttl
-    plan.setTime(System.currentTimeMillis() - 1001);
+    node.setTime(System.currentTimeMillis() - 1001);
     boolean caught = false;
     try {
-      dataRegion.insert(plan);
+      dataRegion.insert(node);
     } catch (OutOfTTLException e) {
       caught = true;
     }
     assertTrue(caught);
-    plan.setTime(System.currentTimeMillis() - 900);
-    dataRegion.insert(plan);
+    node.setTime(System.currentTimeMillis() - 900);
+    dataRegion.insert(node);
   }
 
   private void prepareData()
-      throws WriteProcessException, QueryProcessException, IllegalPathException,
-          TriggerExecutionException {
-    InsertRowPlan plan = new InsertRowPlan();
-    plan.setDevicePath(new PartialPath(sg1));
-    plan.setTime(System.currentTimeMillis());
-    plan.setMeasurements(new String[] {"s1"});
-    plan.setDataTypes(new TSDataType[] {TSDataType.INT64});
-    plan.setValues(new Object[] {1L});
-    plan.setMeasurementMNodes(
-        new IMeasurementMNode[] {
-          MeasurementMNode.getMeasurementMNode(
-              null, "s1", new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.PLAIN), null)
-        });
-    plan.transferType();
+      throws WriteProcessException, IllegalPathException, TriggerExecutionException {
+    InsertRowNode node =
+        new InsertRowNode(
+            new PlanNodeId("0"),
+            new PartialPath(sg1),
+            false,
+            new String[] {"s1"},
+            new TSDataType[] {TSDataType.INT64},
+            System.currentTimeMillis(),
+            new Object[] {1L},
+            false);
+    node.setMeasurementSchemas(
+        new MeasurementSchema[] {new MeasurementSchema("s1", TSDataType.INT64, TSEncoding.PLAIN)});
 
     long initTime = System.currentTimeMillis();
     // sequence data
     for (int i = 1000; i < 2000; i++) {
-      plan.setTime(initTime - 2000 + i);
-      dataRegion.insert(plan);
+      node.setTime(initTime - 2000 + i);
+      dataRegion.insert(node);
       if ((i + 1) % 300 == 0) {
         dataRegion.syncCloseAllWorkingTsFileProcessors();
       }
     }
     // unsequence data
     for (int i = 0; i < 1000; i++) {
-      plan.setTime(initTime - 2000 + i);
-      dataRegion.insert(plan);
+      node.setTime(initTime - 2000 + i);
+      dataRegion.insert(node);
       if ((i + 1) % 300 == 0) {
         dataRegion.syncCloseAllWorkingTsFileProcessors();
       }
@@ -379,56 +373,39 @@ public class TTLTest {
   }
 
   @Test
-  public void testParseSetTTL() throws QueryProcessException {
-    Planner planner = new Planner();
-    SetTTLPlan plan = (SetTTLPlan) planner.parseSQLToPhysicalPlan("SET TTL TO " + sg1 + " 10000");
-    assertEquals(sg1, plan.getStorageGroup().getFullPath());
-    assertEquals(10000, plan.getDataTTL());
-
-    plan = (SetTTLPlan) planner.parseSQLToPhysicalPlan("UNSET TTL TO " + sg2);
-    assertEquals(sg2, plan.getStorageGroup().getFullPath());
-    assertEquals(Long.MAX_VALUE, plan.getDataTTL());
+  public void testParseSetTTL() {
+    SetTTLStatement statement1 =
+        (SetTTLStatement)
+            StatementGenerator.createStatement(
+                "SET TTL TO " + sg1 + " 10000", ZoneId.systemDefault());
+    assertEquals(sg1, statement1.getStorageGroupPath().getFullPath());
+    assertEquals(10000, statement1.getTTL());
+
+    UnSetTTLStatement statement2 =
+        (UnSetTTLStatement)
+            StatementGenerator.createStatement("UNSET TTL TO " + sg2, ZoneId.systemDefault());
+    assertEquals(sg2, statement2.getStorageGroupPath().getFullPath());
+    assertEquals(Long.MAX_VALUE, statement2.getTTL());
   }
 
   @Test
-  public void testParseShowTTL() throws QueryProcessException {
-    Planner planner = new Planner();
-    ShowTTLPlan plan = (ShowTTLPlan) planner.parseSQLToPhysicalPlan("SHOW ALL TTL");
-    assertTrue(plan.getStorageGroups().isEmpty());
+  public void testParseShowTTL() {
+    ShowTTLStatement statement1 =
+        (ShowTTLStatement)
+            StatementGenerator.createStatement("SHOW ALL TTL", ZoneId.systemDefault());
+    assertTrue(statement1.getPaths().isEmpty());
 
     List<String> sgs = new ArrayList<>();
     sgs.add("root.sg1");
     sgs.add("root.sg2");
     sgs.add("root.sg3");
-    plan = (ShowTTLPlan) planner.parseSQLToPhysicalPlan("SHOW TTL ON root.sg1,root.sg2,root.sg3");
+    ShowTTLStatement statement2 =
+        (ShowTTLStatement)
+            StatementGenerator.createStatement(
+                "SHOW TTL ON root.sg1,root.sg2,root.sg3", ZoneId.systemDefault());
     assertEquals(
         sgs,
-        plan.getStorageGroups().stream()
-            .map(PartialPath::getFullPath)
-            .collect(Collectors.toList()));
-  }
-
-  @Test
-  public void testShowTTL()
-      throws IOException, QueryProcessException, QueryFilterOptimizationException,
-          StorageEngineException, MetadataException, InterruptedException {
-    IoTDB.schemaProcessor.setTTL(new PartialPath(sg1), ttl);
-
-    ShowTTLPlan plan = new ShowTTLPlan(Collections.emptyList());
-    PlanExecutor executor = new PlanExecutor();
-    QueryDataSet queryDataSet = executor.processQuery(plan, EnvironmentUtils.TEST_QUERY_CONTEXT);
-
-    while (queryDataSet.hasNext()) {
-      RowRecord rowRecord = queryDataSet.next();
-      String sg = rowRecord.getFields().get(0).getStringValue();
-      if (sg.equals(sg1)) {
-        assertEquals(ttl, rowRecord.getFields().get(1).getLongV());
-      } else if (sg.equals(sg2)) {
-        assertNull(rowRecord.getFields().get(1));
-      } else {
-        fail();
-      }
-    }
+        statement2.getPaths().stream().map(PartialPath::getFullPath).collect(Collectors.toList()));
   }
 
   @Test
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
deleted file mode 100644
index 73c9010fd2..0000000000
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java
+++ /dev/null
@@ -1,476 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.engine.storagegroup;
-
-import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.commons.file.SystemFileFactory;
-import org.apache.iotdb.commons.path.MeasurementPath;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.constant.TestConstant;
-import org.apache.iotdb.db.engine.MetadataManagerHelper;
-import org.apache.iotdb.db.engine.memtable.IMemTable;
-import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
-import org.apache.iotdb.db.exception.TsFileProcessorException;
-import org.apache.iotdb.db.exception.WriteProcessException;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
-import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.rescon.SystemInfo;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
-import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.read.TimeValuePair;
-import org.apache.iotdb.tsfile.read.reader.IPointReader;
-import org.apache.iotdb.tsfile.write.record.TSRecord;
-import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
-import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
-
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import static junit.framework.TestCase.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-public class TsFileProcessorTest {
-
-  private TsFileProcessor processor;
-  private String storageGroup = "root.vehicle";
-  private final String systemDir = TestConstant.OUTPUT_DATA_DIR.concat("info");
-  private DataRegionInfo sgInfo;
-  private String filePath = TestConstant.getTestTsFilePath("root.vehicle", 0, 0, 0);
-  private String deviceId = "root.vehicle.d0";
-  private String measurementId = "s0";
-  private TSDataType dataType = TSDataType.INT32;
-  private TSEncoding encoding = TSEncoding.RLE;
-  private Map<String, String> props = Collections.emptyMap();
-  private QueryContext context;
-  private static Logger logger = LoggerFactory.getLogger(TsFileProcessorTest.class);
-
-  @Before
-  public void setUp() throws Exception {
-    File file = new File(filePath);
-    if (!file.getParentFile().exists()) {
-      Assert.assertTrue(file.getParentFile().mkdirs());
-    }
-    EnvironmentUtils.envSetUp();
-    sgInfo = new DataRegionInfo(new DataRegionTest.DummyDataRegion(systemDir, storageGroup));
-    MetadataManagerHelper.initMetadata();
-    context = EnvironmentUtils.TEST_QUERY_CONTEXT;
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    EnvironmentUtils.cleanEnv();
-    EnvironmentUtils.cleanDir(TestConstant.OUTPUT_DATA_DIR);
-  }
-
-  @Test
-  public void testWriteAndFlush() throws IOException, WriteProcessException, MetadataException {
-    logger.info("testWriteAndFlush begin..");
-    processor =
-        new TsFileProcessor(
-            storageGroup,
-            SystemFileFactory.INSTANCE.getFile(filePath),
-            sgInfo,
-            this::closeTsFileProcessor,
-            (tsFileProcessor) -> true,
-            true);
-
-    TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo);
-    processor.setTsFileProcessorInfo(tsFileProcessorInfo);
-    this.sgInfo.initTsFileProcessorInfo(processor);
-    SystemInfo.getInstance().reportStorageGroupStatus(sgInfo, processor);
-    List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
-    MeasurementPath fullPath =
-        new MeasurementPath(
-            deviceId,
-            measurementId,
-            new MeasurementSchema(
-                measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props));
-    processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery);
-    assertTrue(tsfileResourcesForQuery.isEmpty());
-
-    for (int i = 1; i <= 100; i++) {
-      TSRecord record = new TSRecord(i, deviceId);
-      record.addTuple(DataPoint.getDataPoint(dataType, measurementId, String.valueOf(i)));
-      processor.insert(new InsertRowPlan(record));
-    }
-
-    // query data in memory
-    tsfileResourcesForQuery.clear();
-    processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery);
-
-    TsFileResource tsFileResource = tsfileResourcesForQuery.get(0);
-    assertFalse(tsFileResource.getReadOnlyMemChunk(fullPath).isEmpty());
-    List<ReadOnlyMemChunk> memChunks = tsFileResource.getReadOnlyMemChunk(fullPath);
-    for (ReadOnlyMemChunk chunk : memChunks) {
-      IPointReader iterator = chunk.getPointReader();
-      for (int num = 1; num <= 100; num++) {
-        iterator.hasNextTimeValuePair();
-        TimeValuePair timeValuePair = iterator.nextTimeValuePair();
-        assertEquals(num, timeValuePair.getTimestamp());
-        assertEquals(num, timeValuePair.getValue().getInt());
-      }
-    }
-
-    // flush synchronously
-    processor.syncFlush();
-
-    tsfileResourcesForQuery.clear();
-    processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery);
-    assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath).isEmpty());
-    processor.syncClose();
-  }
-
-  @Test
-  public void testWriteAndRestoreMetadata()
-      throws IOException, WriteProcessException, MetadataException {
-    logger.info("testWriteAndRestoreMetadata begin..");
-    processor =
-        new TsFileProcessor(
-            storageGroup,
-            SystemFileFactory.INSTANCE.getFile(filePath),
-            sgInfo,
-            this::closeTsFileProcessor,
-            (tsFileProcessor) -> true,
-            true);
-
-    TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo);
-    processor.setTsFileProcessorInfo(tsFileProcessorInfo);
-    this.sgInfo.initTsFileProcessorInfo(processor);
-    SystemInfo.getInstance().reportStorageGroupStatus(sgInfo, processor);
-    List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
-    MeasurementPath fullPath =
-        new MeasurementPath(
-            deviceId,
-            measurementId,
-            new MeasurementSchema(
-                measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props));
-    processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery);
-    assertTrue(tsfileResourcesForQuery.isEmpty());
-
-    for (int i = 1; i <= 100; i++) {
-      TSRecord record = new TSRecord(i, deviceId);
-      record.addTuple(DataPoint.getDataPoint(dataType, measurementId, String.valueOf(i)));
-      processor.insert(new InsertRowPlan(record));
-    }
-
-    // query data in memory
-    tsfileResourcesForQuery.clear();
-    processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery);
-    assertFalse(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath).isEmpty());
-    int num = 1;
-    List<ReadOnlyMemChunk> memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath);
-    for (ReadOnlyMemChunk chunk : memChunks) {
-      IPointReader iterator = chunk.getPointReader();
-      for (; num <= 100; num++) {
-        iterator.hasNextTimeValuePair();
-        TimeValuePair timeValuePair = iterator.nextTimeValuePair();
-        assertEquals(num, timeValuePair.getTimestamp());
-        assertEquals(num, timeValuePair.getValue().getInt());
-      }
-    }
-    logger.info("syncFlush..");
-    // flush synchronously
-    processor.syncFlush();
-
-    tsfileResourcesForQuery.clear();
-    processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery);
-    assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath).isEmpty());
-
-    RestorableTsFileIOWriter tsFileIOWriter = processor.getWriter();
-    Map<String, List<ChunkMetadata>> chunkMetaDataListInChunkGroups =
-        tsFileIOWriter.getDeviceChunkMetadataMap();
-    RestorableTsFileIOWriter restorableTsFileIOWriter =
-        new RestorableTsFileIOWriter(SystemFileFactory.INSTANCE.getFile(filePath));
-    Map<String, List<ChunkMetadata>> restoredChunkMetaDataListInChunkGroups =
-        restorableTsFileIOWriter.getDeviceChunkMetadataMap();
-    assertEquals(
-        chunkMetaDataListInChunkGroups.size(), restoredChunkMetaDataListInChunkGroups.size());
-    for (Map.Entry<String, List<ChunkMetadata>> entry1 :
-        chunkMetaDataListInChunkGroups.entrySet()) {
-      for (Map.Entry<String, List<ChunkMetadata>> entry2 :
-          restoredChunkMetaDataListInChunkGroups.entrySet()) {
-        assertEquals(entry1.getKey(), entry2.getKey());
-        assertEquals(entry1.getValue().size(), entry2.getValue().size());
-        for (int i = 0; i < entry1.getValue().size(); i++) {
-          ChunkMetadata chunkMetaData = entry1.getValue().get(i);
-          chunkMetaData.setFilePath(filePath);
-
-          ChunkMetadata chunkMetadataRestore = entry2.getValue().get(i);
-          chunkMetadataRestore.setFilePath(filePath);
-
-          assertEquals(chunkMetaData, chunkMetadataRestore);
-        }
-      }
-    }
-    restorableTsFileIOWriter.close();
-    logger.info("syncClose..");
-    processor.syncClose();
-    // we need to close the tsfile writer first and then reopen it.
-  }
-
-  @Test
-  public void testMultiFlush() throws IOException, WriteProcessException, MetadataException {
-    logger.info("testWriteAndRestoreMetadata begin..");
-    processor =
-        new TsFileProcessor(
-            storageGroup,
-            SystemFileFactory.INSTANCE.getFile(filePath),
-            sgInfo,
-            this::closeTsFileProcessor,
-            (tsFileProcessor) -> true,
-            true);
-
-    TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo);
-    processor.setTsFileProcessorInfo(tsFileProcessorInfo);
-    this.sgInfo.initTsFileProcessorInfo(processor);
-    SystemInfo.getInstance().reportStorageGroupStatus(sgInfo, processor);
-    List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
-    MeasurementPath fullPath =
-        new MeasurementPath(
-            deviceId,
-            measurementId,
-            new MeasurementSchema(
-                measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props));
-    processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery);
-    assertTrue(tsfileResourcesForQuery.isEmpty());
-
-    for (int flushId = 0; flushId < 10; flushId++) {
-      for (int i = 1; i <= 10; i++) {
-        TSRecord record = new TSRecord(i, deviceId);
-        record.addTuple(DataPoint.getDataPoint(dataType, measurementId, String.valueOf(i)));
-        processor.insert(new InsertRowPlan(record));
-      }
-      processor.asyncFlush();
-    }
-    processor.syncFlush();
-
-    tsfileResourcesForQuery.clear();
-    processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery);
-    assertFalse(tsfileResourcesForQuery.isEmpty());
-    assertTrue(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath).isEmpty());
-    processor.syncClose();
-  }
-
-  @Test
-  public void alignedTvListRamCostTest()
-      throws MetadataException, WriteProcessException, IOException {
-    processor =
-        new TsFileProcessor(
-            storageGroup,
-            SystemFileFactory.INSTANCE.getFile(filePath),
-            sgInfo,
-            this::closeTsFileProcessor,
-            (tsFileProcessor) -> true,
-            true);
-    TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo);
-    processor.setTsFileProcessorInfo(tsFileProcessorInfo);
-    this.sgInfo.initTsFileProcessorInfo(processor);
-    SystemInfo.getInstance().reportStorageGroupStatus(sgInfo, processor);
-    processor.insertTablet(genInsertTablePlan(0, true), 0, 10, new TSStatus[10]);
-    IMemTable memTable = processor.getWorkMemTable();
-    Assert.assertEquals(828424, memTable.getTVListsRamCost());
-    processor.insertTablet(genInsertTablePlan(100, true), 0, 10, new TSStatus[10]);
-    Assert.assertEquals(828424, memTable.getTVListsRamCost());
-    processor.insertTablet(genInsertTablePlan(200, true), 0, 10, new TSStatus[10]);
-    Assert.assertEquals(828424, memTable.getTVListsRamCost());
-    Assert.assertEquals(90000, memTable.getTotalPointsNum());
-    Assert.assertEquals(720360, memTable.memSize());
-    // Test records
-    for (int i = 1; i <= 100; i++) {
-      TSRecord record = new TSRecord(i, deviceId);
-      record.addTuple(DataPoint.getDataPoint(dataType, measurementId, String.valueOf(i)));
-      processor.insert(new InsertRowPlan(record));
-    }
-    Assert.assertEquals(830120, memTable.getTVListsRamCost());
-    Assert.assertEquals(90100, memTable.getTotalPointsNum());
-    Assert.assertEquals(721560, memTable.memSize());
-  }
-
-  @Test
-  public void nonAlignedTvListRamCostTest()
-      throws MetadataException, WriteProcessException, IOException {
-    processor =
-        new TsFileProcessor(
-            storageGroup,
-            SystemFileFactory.INSTANCE.getFile(filePath),
-            sgInfo,
-            this::closeTsFileProcessor,
-            (tsFileProcessor) -> true,
-            true);
-    TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo);
-    processor.setTsFileProcessorInfo(tsFileProcessorInfo);
-    this.sgInfo.initTsFileProcessorInfo(processor);
-    SystemInfo.getInstance().reportStorageGroupStatus(sgInfo, processor);
-    processor.insertTablet(genInsertTablePlan(0, false), 0, 10, new TSStatus[10]);
-    IMemTable memTable = processor.getWorkMemTable();
-    Assert.assertEquals(1656000, memTable.getTVListsRamCost());
-    processor.insertTablet(genInsertTablePlan(100, false), 0, 10, new TSStatus[10]);
-    Assert.assertEquals(1656000, memTable.getTVListsRamCost());
-    processor.insertTablet(genInsertTablePlan(200, false), 0, 10, new TSStatus[10]);
-    Assert.assertEquals(1656000, memTable.getTVListsRamCost());
-    Assert.assertEquals(90000, memTable.getTotalPointsNum());
-    Assert.assertEquals(1440000, memTable.memSize());
-    for (int i = 1; i <= 100; i++) {
-      TSRecord record = new TSRecord(i, deviceId);
-      record.addTuple(DataPoint.getDataPoint(dataType, measurementId, String.valueOf(i)));
-      processor.insert(new InsertRowPlan(record));
-    }
-    Assert.assertEquals(1657696, memTable.getTVListsRamCost());
-    Assert.assertEquals(90100, memTable.getTotalPointsNum());
-    Assert.assertEquals(1441200, memTable.memSize());
-  }
-
-  @Test
-  public void testWriteAndClose() throws IOException, WriteProcessException, MetadataException {
-    logger.info("testWriteAndRestoreMetadata begin..");
-    processor =
-        new TsFileProcessor(
-            storageGroup,
-            SystemFileFactory.INSTANCE.getFile(filePath),
-            sgInfo,
-            this::closeTsFileProcessor,
-            (tsFileProcessor) -> true,
-            true);
-
-    TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo);
-    processor.setTsFileProcessorInfo(tsFileProcessorInfo);
-    this.sgInfo.initTsFileProcessorInfo(processor);
-    SystemInfo.getInstance().reportStorageGroupStatus(sgInfo, processor);
-    List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
-
-    MeasurementPath fullPath =
-        new MeasurementPath(
-            deviceId,
-            measurementId,
-            new MeasurementSchema(
-                measurementId, dataType, encoding, CompressionType.UNCOMPRESSED, props));
-    processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery);
-    assertTrue(tsfileResourcesForQuery.isEmpty());
-
-    for (int i = 1; i <= 100; i++) {
-      TSRecord record = new TSRecord(i, deviceId);
-      record.addTuple(DataPoint.getDataPoint(dataType, measurementId, String.valueOf(i)));
-      processor.insert(new InsertRowPlan(record));
-    }
-
-    // query data in memory
-    tsfileResourcesForQuery.clear();
-    processor.query(Collections.singletonList(fullPath), context, tsfileResourcesForQuery);
-    assertFalse(tsfileResourcesForQuery.isEmpty());
-    assertFalse(tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath).isEmpty());
-    List<ReadOnlyMemChunk> memChunks = tsfileResourcesForQuery.get(0).getReadOnlyMemChunk(fullPath);
-    for (ReadOnlyMemChunk chunk : memChunks) {
-      IPointReader iterator = chunk.getPointReader();
-      for (int num = 1; num <= 100; num++) {
-        iterator.hasNextTimeValuePair();
-        TimeValuePair timeValuePair = iterator.nextTimeValuePair();
-        assertEquals(num, timeValuePair.getTimestamp());
-        assertEquals(num, timeValuePair.getValue().getInt());
-      }
-    }
-
-    // close synchronously
-    processor.syncClose();
-    try {
-      Thread.sleep(1000);
-    } catch (InterruptedException e) {
-      e.printStackTrace();
-    }
-
-    assertTrue(processor.getTsFileResource().isClosed());
-  }
-
-  private void closeTsFileProcessor(TsFileProcessor unsealedTsFileProcessor)
-      throws TsFileProcessorException {
-    TsFileResource resource = unsealedTsFileProcessor.getTsFileResource();
-    synchronized (resource) {
-      for (String deviceId : resource.getDevices()) {
-        resource.updateEndTime(deviceId, resource.getStartTime(deviceId));
-      }
-      try {
-        resource.close();
-      } catch (IOException e) {
-        throw new TsFileProcessorException(e);
-      }
-    }
-  }
-
-  private InsertTabletPlan genInsertTablePlan(long startTime, boolean isAligned)
-      throws IllegalPathException {
-    String deviceId = "root.sg.device5";
-    String[] measurements = new String[3000];
-    List<Integer> dataTypesList = new ArrayList<>();
-    TSDataType[] dataTypes = new TSDataType[3000];
-    TSEncoding[] encodings = new TSEncoding[3000];
-    IMeasurementMNode[] mNodes = new IMeasurementMNode[3000];
-    for (int i = 0; i < 3000; i++) {
-      measurements[i] = "s" + i;
-      dataTypesList.add(TSDataType.INT64.ordinal());
-      dataTypes[i] = TSDataType.INT64;
-      encodings[i] = TSEncoding.PLAIN;
-      IMeasurementSchema schema =
-          new MeasurementSchema(measurements[i], dataTypes[i], encodings[i]);
-      mNodes[i] = MeasurementMNode.getMeasurementMNode(null, measurements[i], schema, null);
-    }
-    InsertTabletPlan insertTabletPlan =
-        new InsertTabletPlan(new PartialPath(deviceId), measurements, dataTypesList);
-
-    long[] times = new long[10];
-    Object[] columns = new Object[3000];
-    for (int i = 0; i < 3000; i++) {
-      columns[i] = new long[10];
-    }
-
-    for (long r = 0; r < 10; r++) {
-      times[(int) r] = r + startTime;
-      for (int i = 0; i < 3000; i++) {
-        ((long[]) columns[i])[(int) r] = r;
-      }
-    }
-    insertTabletPlan.setTimes(times);
-    insertTabletPlan.setColumns(columns);
-    insertTabletPlan.setRowCount(times.length);
-    insertTabletPlan.setMeasurementMNodes(mNodes);
-    insertTabletPlan.setAligned(isAligned);
-
-    return insertTabletPlan;
-  }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/dataregion/HashVirtualPartitionerTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/dataregion/HashVirtualPartitionerTest.java
deleted file mode 100644
index 71a9b944f9..0000000000
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/dataregion/HashVirtualPartitionerTest.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.engine.storagegroup.dataregion;
-
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-
-public class HashVirtualPartitionerTest {
-  @Before
-  public void setUp() {
-    EnvironmentUtils.envSetUp();
-    // init file dir
-    StorageEngine.getInstance();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    EnvironmentUtils.cleanEnv();
-  }
-
-  @Test
-  public void basicTest() throws IllegalPathException {
-    HashVirtualPartitioner hashVirtualPartitioner = HashVirtualPartitioner.getInstance();
-
-    // sg -> deviceId
-    HashMap<PartialPath, Set<PartialPath>> realMap = new HashMap<>();
-    PartialPath d1 = new PartialPath("root.sg1.d1");
-    PartialPath d2 = new PartialPath("root.sg1.d2");
-
-    int sg1 = hashVirtualPartitioner.deviceToDataRegionId(d1);
-    int sg2 = hashVirtualPartitioner.deviceToDataRegionId(d2);
-
-    assertEquals(sg1, Math.abs(d1.hashCode() % hashVirtualPartitioner.getPartitionCount()));
-    assertEquals(sg2, Math.abs(d2.hashCode() % hashVirtualPartitioner.getPartitionCount()));
-  }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/TemplateTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/TemplateTest.java
index c9730feab9..819d8cdedb 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/TemplateTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/TemplateTest.java
@@ -23,11 +23,9 @@ import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.MeasurementPath;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
 import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.metadata.template.Template;
-import org.apache.iotdb.db.qp.executor.PlanExecutor;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
 import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
@@ -301,73 +299,74 @@ public class TemplateTest {
    *
    * @throws MetadataException
    */
-  @Test
-  public void testShowTemplates() throws MetadataException, QueryProcessException {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-    assertEquals(0, schemaProcessor.getAllTemplates().size());
-    CreateTemplatePlan plan1 = getTreeTemplatePlan();
-    CreateTemplatePlan plan2 = getCreateTemplatePlan();
-    schemaProcessor.createSchemaTemplate(plan1);
-    schemaProcessor.createSchemaTemplate(plan2);
-
-    assertEquals("[template1, treeTemplate]", schemaProcessor.getAllTemplates().toString());
-
-    for (int i = 0; i < 3; i++) {
-      SetTemplatePlan setTemplatePlan =
-          new SetTemplatePlan("template1", String.format("root.sg%d.d%d", i, i + 1));
-      schemaProcessor.setSchemaTemplate(setTemplatePlan);
-    }
-
-    assertEquals(
-        new HashSet<>(Arrays.asList("root.sg1.d2", "root.sg0.d1", "root.sg2.d3")),
-        schemaProcessor.getPathsSetTemplate("*"));
-    assertEquals(
-        new HashSet<>(Arrays.asList()), schemaProcessor.getPathsSetTemplate("treeTemplate"));
-
-    for (int i = 0; i < 3; i++) {
-      SetTemplatePlan setTemplatePlan =
-          new SetTemplatePlan("treeTemplate", String.format("root.tsg%d.d%d", i + 9, i + 10));
-      schemaProcessor.setSchemaTemplate(setTemplatePlan);
-    }
-
-    assertEquals(
-        new HashSet<>(Arrays.asList("root.tsg10.d11", "root.tsg11.d12", "root.tsg9.d10")),
-        schemaProcessor.getPathsSetTemplate("treeTemplate"));
-    assertEquals(
-        new HashSet<>(
-            Arrays.asList(
-                "root.tsg10.d11",
-                "root.tsg11.d12",
-                "root.tsg9.d10",
-                "root.sg1.d2",
-                "root.sg0.d1",
-                "root.sg2.d3")),
-        schemaProcessor.getPathsSetTemplate("*"));
-
-    PlanExecutor exe1 = new PlanExecutor();
-    exe1.insert(getInsertRowPlan("root.sg0.d1", "s11"));
-    exe1.insert(getInsertRowPlan("root.sg1.d2", "s11"));
-    exe1.insert(getInsertRowPlan("root.tsg10.d11.d1", "s1"));
-
-    assertEquals(
-        new HashSet<>(Arrays.asList("root.tsg10.d11", "root.sg1.d2", "root.sg0.d1")),
-        schemaProcessor.getPathsUsingTemplate("*"));
-
-    try {
-      schemaProcessor.createSchemaTemplate(plan1);
-      fail();
-    } catch (MetadataException e) {
-      assertEquals("Duplicated template name: treeTemplate", e.getMessage());
-    }
-
-    try {
-      schemaProcessor.dropSchemaTemplate(new DropTemplatePlan("treeTemplate"));
-      fail();
-    } catch (MetadataException e) {
-      assertEquals(
-          "Template [treeTemplate] has been set on MTree, cannot be dropped now.", e.getMessage());
-    }
-  }
+  //  @Test
+  //  public void testShowTemplates() throws MetadataException, QueryProcessException {
+  //    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+  //    assertEquals(0, schemaProcessor.getAllTemplates().size());
+  //    CreateTemplatePlan plan1 = getTreeTemplatePlan();
+  //    CreateTemplatePlan plan2 = getCreateTemplatePlan();
+  //    schemaProcessor.createSchemaTemplate(plan1);
+  //    schemaProcessor.createSchemaTemplate(plan2);
+  //
+  //    assertEquals("[template1, treeTemplate]", schemaProcessor.getAllTemplates().toString());
+  //
+  //    for (int i = 0; i < 3; i++) {
+  //      SetTemplatePlan setTemplatePlan =
+  //          new SetTemplatePlan("template1", String.format("root.sg%d.d%d", i, i + 1));
+  //      schemaProcessor.setSchemaTemplate(setTemplatePlan);
+  //    }
+  //
+  //    assertEquals(
+  //        new HashSet<>(Arrays.asList("root.sg1.d2", "root.sg0.d1", "root.sg2.d3")),
+  //        schemaProcessor.getPathsSetTemplate("*"));
+  //    assertEquals(
+  //        new HashSet<>(Arrays.asList()), schemaProcessor.getPathsSetTemplate("treeTemplate"));
+  //
+  //    for (int i = 0; i < 3; i++) {
+  //      SetTemplatePlan setTemplatePlan =
+  //          new SetTemplatePlan("treeTemplate", String.format("root.tsg%d.d%d", i + 9, i + 10));
+  //      schemaProcessor.setSchemaTemplate(setTemplatePlan);
+  //    }
+  //
+  //    assertEquals(
+  //        new HashSet<>(Arrays.asList("root.tsg10.d11", "root.tsg11.d12", "root.tsg9.d10")),
+  //        schemaProcessor.getPathsSetTemplate("treeTemplate"));
+  //    assertEquals(
+  //        new HashSet<>(
+  //            Arrays.asList(
+  //                "root.tsg10.d11",
+  //                "root.tsg11.d12",
+  //                "root.tsg9.d10",
+  //                "root.sg1.d2",
+  //                "root.sg0.d1",
+  //                "root.sg2.d3")),
+  //        schemaProcessor.getPathsSetTemplate("*"));
+  //
+  //    PlanExecutor exe1 = new PlanExecutor();
+  //    exe1.insert(getInsertRowPlan("root.sg0.d1", "s11"));
+  //    exe1.insert(getInsertRowPlan("root.sg1.d2", "s11"));
+  //    exe1.insert(getInsertRowPlan("root.tsg10.d11.d1", "s1"));
+  //
+  //    assertEquals(
+  //        new HashSet<>(Arrays.asList("root.tsg10.d11", "root.sg1.d2", "root.sg0.d1")),
+  //        schemaProcessor.getPathsUsingTemplate("*"));
+  //
+  //    try {
+  //      schemaProcessor.createSchemaTemplate(plan1);
+  //      fail();
+  //    } catch (MetadataException e) {
+  //      assertEquals("Duplicated template name: treeTemplate", e.getMessage());
+  //    }
+  //
+  //    try {
+  //      schemaProcessor.dropSchemaTemplate(new DropTemplatePlan("treeTemplate"));
+  //      fail();
+  //    } catch (MetadataException e) {
+  //      assertEquals(
+  //          "Template [treeTemplate] has been set on MTree, cannot be dropped now.",
+  // e.getMessage());
+  //    }
+  //  }
 
   @Test
   public void testShowAllSchemas() throws MetadataException {
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableFlushTimeTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableFlushTimeTest.java
index cb48267bbf..4eab19cb67 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableFlushTimeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableFlushTimeTest.java
@@ -19,208 +19,181 @@
 
 package org.apache.iotdb.db.metadata.idtable;
 
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.engine.StorageEngine;
-import org.apache.iotdb.db.engine.storagegroup.DataRegion;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.Planner;
-import org.apache.iotdb.db.qp.executor.PlanExecutor;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePartitionPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.utils.Binary;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-
-public class IDTableFlushTimeTest {
-  private PlanExecutor executor = new PlanExecutor();
-
-  private final Planner processor = new Planner();
-
-  private boolean isEnableIDTable = false;
-
-  private String originalDeviceIDTransformationMethod = null;
-
-  private boolean isEnableIDTableLogFile = false;
-
-  public IDTableFlushTimeTest() throws QueryProcessException {}
-
-  @Before
-  public void before() {
-    IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(true);
-    isEnableIDTable = IoTDBDescriptor.getInstance().getConfig().isEnableIDTable();
-    originalDeviceIDTransformationMethod =
-        IoTDBDescriptor.getInstance().getConfig().getDeviceIDTransformationMethod();
-    isEnableIDTableLogFile = IoTDBDescriptor.getInstance().getConfig().isEnableIDTableLogFile();
-
-    IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(true);
-    IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("SHA256");
-    IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(true);
-    EnvironmentUtils.envSetUp();
-  }
-
-  @After
-  public void clean() throws IOException, StorageEngineException {
-    IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(isEnableIDTable);
-    IoTDBDescriptor.getInstance()
-        .getConfig()
-        .setDeviceIDTransformationMethod(originalDeviceIDTransformationMethod);
-    IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(isEnableIDTableLogFile);
-    EnvironmentUtils.cleanEnv();
-  }
-
-  @Test
-  public void testSequenceInsert()
-      throws MetadataException, QueryProcessException, StorageEngineException {
-    insertData(0);
-    insertData(10);
-    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
-    executor.processNonQuery(flushPlan);
-
-    insertData(20);
-
-    DataRegion storageGroupProcessor =
-        StorageEngine.getInstance().getProcessor(new PartialPath("root.isp.d1"));
-    assertEquals(2, storageGroupProcessor.getSequenceFileList().size());
-    assertEquals(0, storageGroupProcessor.getUnSequenceFileList().size());
-  }
-
-  @Test
-  public void testUnSequenceInsert()
-      throws MetadataException, QueryProcessException, StorageEngineException {
-    insertData(100);
-    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
-    executor.processNonQuery(flushPlan);
-
-    insertData(20);
-
-    DataRegion storageGroupProcessor =
-        StorageEngine.getInstance().getProcessor(new PartialPath("root.isp.d1"));
-    assertEquals(1, storageGroupProcessor.getSequenceFileList().size());
-    assertEquals(1, storageGroupProcessor.getUnSequenceFileList().size());
-  }
-
-  @Test
-  public void testSequenceAndUnSequenceInsert()
-      throws MetadataException, QueryProcessException, StorageEngineException {
-    // sequence
-    insertData(100);
-    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
-    executor.processNonQuery(flushPlan);
-
-    // sequence
-    insertData(120);
-    executor.processNonQuery(flushPlan);
-
-    // unsequence
-    insertData(20);
-    // sequence
-    insertData(130);
-    executor.processNonQuery(flushPlan);
-
-    // sequence
-    insertData(150);
-    // unsequence
-    insertData(90);
-
-    DataRegion storageGroupProcessor =
-        StorageEngine.getInstance().getProcessor(new PartialPath("root.isp.d1"));
-    assertEquals(4, storageGroupProcessor.getSequenceFileList().size());
-    assertEquals(2, storageGroupProcessor.getUnSequenceFileList().size());
-    assertEquals(1, storageGroupProcessor.getWorkSequenceTsFileProcessors().size());
-    assertEquals(1, storageGroupProcessor.getWorkUnsequenceTsFileProcessors().size());
-  }
-
-  @Test
-  public void testDeletePartition()
-      throws MetadataException, QueryProcessException, StorageEngineException {
-    insertData(100);
-    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
-    executor.processNonQuery(flushPlan);
-    insertData(20);
-    insertData(120);
-
-    DataRegion storageGroupProcessor =
-        StorageEngine.getInstance().getProcessor(new PartialPath("root.isp.d1"));
-
-    assertEquals(
-        103L, storageGroupProcessor.getLastFlushTimeManager().getFlushedTime(0L, "root.isp.d1"));
-    assertEquals(
-        123L, storageGroupProcessor.getLastFlushTimeManager().getLastTime(0L, "root.isp.d1"));
-    assertEquals(
-        103L, storageGroupProcessor.getLastFlushTimeManager().getGlobalFlushedTime("root.isp.d1"));
-
-    // delete time partition
-    Set<Long> deletedPartition = new HashSet<>();
-    deletedPartition.add(0L);
-    DeletePartitionPlan deletePartitionPlan =
-        new DeletePartitionPlan(new PartialPath("root.isp"), deletedPartition);
-    executor.processNonQuery(deletePartitionPlan);
-
-    assertEquals(
-        Long.MIN_VALUE,
-        storageGroupProcessor.getLastFlushTimeManager().getFlushedTime(0L, "root.isp.d1"));
-    assertEquals(
-        Long.MIN_VALUE,
-        storageGroupProcessor.getLastFlushTimeManager().getLastTime(0L, "root.isp.d1"));
-    assertEquals(
-        123L, storageGroupProcessor.getLastFlushTimeManager().getGlobalFlushedTime("root.isp.d1"));
-  }
-
-  private void insertData(long initTime) throws IllegalPathException, QueryProcessException {
-
-    long[] times = new long[] {initTime, initTime + 1, initTime + 2, initTime + 3};
-    List<Integer> dataTypes = new ArrayList<>();
-    dataTypes.add(TSDataType.DOUBLE.ordinal());
-    dataTypes.add(TSDataType.FLOAT.ordinal());
-    dataTypes.add(TSDataType.INT64.ordinal());
-    dataTypes.add(TSDataType.INT32.ordinal());
-    dataTypes.add(TSDataType.BOOLEAN.ordinal());
-    dataTypes.add(TSDataType.TEXT.ordinal());
-
-    Object[] columns = new Object[6];
-    columns[0] = new double[4];
-    columns[1] = new float[4];
-    columns[2] = new long[4];
-    columns[3] = new int[4];
-    columns[4] = new boolean[4];
-    columns[5] = new Binary[4];
-
-    for (int r = 0; r < 4; r++) {
-      ((double[]) columns[0])[r] = 10.0 + r;
-      ((float[]) columns[1])[r] = 20 + r;
-      ((long[]) columns[2])[r] = 100000 + r;
-      ((int[]) columns[3])[r] = 1000 + r;
-      ((boolean[]) columns[4])[r] = false;
-      ((Binary[]) columns[5])[r] = new Binary("mm" + r);
-    }
-
-    InsertTabletPlan tabletPlan =
-        new InsertTabletPlan(
-            new PartialPath("root.isp.d1"),
-            new String[] {"s1", "s2", "s3", "s4", "s5", "s6"},
-            dataTypes);
-    tabletPlan.setTimes(times);
-    tabletPlan.setColumns(columns);
-    tabletPlan.setRowCount(times.length);
-
-    PlanExecutor executor = new PlanExecutor();
-    executor.insertTablet(tabletPlan);
-  }
-}
+// public class IDTableFlushTimeTest {
+//  private PlanExecutor executor = new PlanExecutor();
+//
+//  private final Planner processor = new Planner();
+//
+//  private boolean isEnableIDTable = false;
+//
+//  private String originalDeviceIDTransformationMethod = null;
+//
+//  private boolean isEnableIDTableLogFile = false;
+//
+//  public IDTableFlushTimeTest() throws QueryProcessException {}
+//
+//  @Before
+//  public void before() {
+//    IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(true);
+//    isEnableIDTable = IoTDBDescriptor.getInstance().getConfig().isEnableIDTable();
+//    originalDeviceIDTransformationMethod =
+//        IoTDBDescriptor.getInstance().getConfig().getDeviceIDTransformationMethod();
+//    isEnableIDTableLogFile = IoTDBDescriptor.getInstance().getConfig().isEnableIDTableLogFile();
+//
+//    IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(true);
+//    IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("SHA256");
+//    IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(true);
+//    EnvironmentUtils.envSetUp();
+//  }
+//
+//  @After
+//  public void clean() throws IOException, StorageEngineException {
+//    IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(isEnableIDTable);
+//    IoTDBDescriptor.getInstance()
+//        .getConfig()
+//        .setDeviceIDTransformationMethod(originalDeviceIDTransformationMethod);
+//    IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(isEnableIDTableLogFile);
+//    EnvironmentUtils.cleanEnv();
+//  }
+//
+//  @Test
+//  public void testSequenceInsert()
+//      throws MetadataException, QueryProcessException, StorageEngineException {
+//    insertData(0);
+//    insertData(10);
+//    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
+//    executor.processNonQuery(flushPlan);
+//
+//    insertData(20);
+//
+//    DataRegion storageGroupProcessor =
+//        StorageEngine.getInstance().getProcessor(new PartialPath("root.isp.d1"));
+//    assertEquals(2, storageGroupProcessor.getSequenceFileList().size());
+//    assertEquals(0, storageGroupProcessor.getUnSequenceFileList().size());
+//  }
+//
+//  @Test
+//  public void testUnSequenceInsert()
+//      throws MetadataException, QueryProcessException, StorageEngineException {
+//    insertData(100);
+//    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
+//    executor.processNonQuery(flushPlan);
+//
+//    insertData(20);
+//
+//    DataRegion storageGroupProcessor =
+//        StorageEngine.getInstance().getProcessor(new PartialPath("root.isp.d1"));
+//    assertEquals(1, storageGroupProcessor.getSequenceFileList().size());
+//    assertEquals(1, storageGroupProcessor.getUnSequenceFileList().size());
+//  }
+//
+//  @Test
+//  public void testSequenceAndUnSequenceInsert()
+//      throws MetadataException, QueryProcessException, StorageEngineException {
+//    // sequence
+//    insertData(100);
+//    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
+//    executor.processNonQuery(flushPlan);
+//
+//    // sequence
+//    insertData(120);
+//    executor.processNonQuery(flushPlan);
+//
+//    // unsequence
+//    insertData(20);
+//    // sequence
+//    insertData(130);
+//    executor.processNonQuery(flushPlan);
+//
+//    // sequence
+//    insertData(150);
+//    // unsequence
+//    insertData(90);
+//
+//    DataRegion storageGroupProcessor =
+//        StorageEngine.getInstance().getProcessor(new PartialPath("root.isp.d1"));
+//    assertEquals(4, storageGroupProcessor.getSequenceFileList().size());
+//    assertEquals(2, storageGroupProcessor.getUnSequenceFileList().size());
+//    assertEquals(1, storageGroupProcessor.getWorkSequenceTsFileProcessors().size());
+//    assertEquals(1, storageGroupProcessor.getWorkUnsequenceTsFileProcessors().size());
+//  }
+//
+//  @Test
+//  public void testDeletePartition()
+//      throws MetadataException, QueryProcessException, StorageEngineException {
+//    insertData(100);
+//    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
+//    executor.processNonQuery(flushPlan);
+//    insertData(20);
+//    insertData(120);
+//
+//    DataRegion storageGroupProcessor =
+//        StorageEngine.getInstance().getProcessor(new PartialPath("root.isp.d1"));
+//
+//    assertEquals(
+//        103L, storageGroupProcessor.getLastFlushTimeManager().getFlushedTime(0L, "root.isp.d1"));
+//    assertEquals(
+//        123L, storageGroupProcessor.getLastFlushTimeManager().getLastTime(0L, "root.isp.d1"));
+//    assertEquals(
+//        103L,
+// storageGroupProcessor.getLastFlushTimeManager().getGlobalFlushedTime("root.isp.d1"));
+//
+//    // delete time partition
+//    Set<Long> deletedPartition = new HashSet<>();
+//    deletedPartition.add(0L);
+//    DeletePartitionPlan deletePartitionPlan =
+//        new DeletePartitionPlan(new PartialPath("root.isp"), deletedPartition);
+//    executor.processNonQuery(deletePartitionPlan);
+//
+//    assertEquals(
+//        Long.MIN_VALUE,
+//        storageGroupProcessor.getLastFlushTimeManager().getFlushedTime(0L, "root.isp.d1"));
+//    assertEquals(
+//        Long.MIN_VALUE,
+//        storageGroupProcessor.getLastFlushTimeManager().getLastTime(0L, "root.isp.d1"));
+//    assertEquals(
+//        123L,
+// storageGroupProcessor.getLastFlushTimeManager().getGlobalFlushedTime("root.isp.d1"));
+//  }
+//
+//  private void insertData(long initTime) throws IllegalPathException, QueryProcessException {
+//
+//    long[] times = new long[] {initTime, initTime + 1, initTime + 2, initTime + 3};
+//    List<Integer> dataTypes = new ArrayList<>();
+//    dataTypes.add(TSDataType.DOUBLE.ordinal());
+//    dataTypes.add(TSDataType.FLOAT.ordinal());
+//    dataTypes.add(TSDataType.INT64.ordinal());
+//    dataTypes.add(TSDataType.INT32.ordinal());
+//    dataTypes.add(TSDataType.BOOLEAN.ordinal());
+//    dataTypes.add(TSDataType.TEXT.ordinal());
+//
+//    Object[] columns = new Object[6];
+//    columns[0] = new double[4];
+//    columns[1] = new float[4];
+//    columns[2] = new long[4];
+//    columns[3] = new int[4];
+//    columns[4] = new boolean[4];
+//    columns[5] = new Binary[4];
+//
+//    for (int r = 0; r < 4; r++) {
+//      ((double[]) columns[0])[r] = 10.0 + r;
+//      ((float[]) columns[1])[r] = 20 + r;
+//      ((long[]) columns[2])[r] = 100000 + r;
+//      ((int[]) columns[3])[r] = 1000 + r;
+//      ((boolean[]) columns[4])[r] = false;
+//      ((Binary[]) columns[5])[r] = new Binary("mm" + r);
+//    }
+//
+//    InsertTabletPlan tabletPlan =
+//        new InsertTabletPlan(
+//            new PartialPath("root.isp.d1"),
+//            new String[] {"s1", "s2", "s3", "s4", "s5", "s6"},
+//            dataTypes);
+//    tabletPlan.setTimes(times);
+//    tabletPlan.setColumns(columns);
+//    tabletPlan.setRowCount(times.length);
+//
+//    PlanExecutor executor = new PlanExecutor();
+//    executor.insertTablet(tabletPlan);
+//  }
+// }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableRecoverTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableRecoverTest.java
index bcacb16073..2248843d34 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableRecoverTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableRecoverTest.java
@@ -19,193 +19,167 @@
 
 package org.apache.iotdb.db.metadata.idtable;
 
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.idtable.entry.DeviceEntry;
-import org.apache.iotdb.db.qp.Planner;
-import org.apache.iotdb.db.qp.executor.PlanExecutor;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.utils.Binary;
-
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
-public class IDTableRecoverTest {
-  private final Planner processor = new Planner();
-
-  private boolean isEnableIDTable = false;
-
-  private String originalDeviceIDTransformationMethod = null;
-
-  private boolean isEnableIDTableLogFile = false;
-
-  @Before
-  public void before() {
-    IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(true);
-    isEnableIDTable = IoTDBDescriptor.getInstance().getConfig().isEnableIDTable();
-    originalDeviceIDTransformationMethod =
-        IoTDBDescriptor.getInstance().getConfig().getDeviceIDTransformationMethod();
-    isEnableIDTableLogFile = IoTDBDescriptor.getInstance().getConfig().isEnableIDTableLogFile();
-
-    IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(true);
-    IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("SHA256");
-    IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(true);
-    EnvironmentUtils.envSetUp();
-  }
-
-  @After
-  public void clean() throws IOException, StorageEngineException {
-    IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(isEnableIDTable);
-    IoTDBDescriptor.getInstance()
-        .getConfig()
-        .setDeviceIDTransformationMethod(originalDeviceIDTransformationMethod);
-    IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(isEnableIDTableLogFile);
-
-    EnvironmentUtils.cleanEnv();
-  }
-
-  @Test
-  public void testRecover() throws Exception {
-    insertDataInMemoryWithTablet(false);
-    insertDataInMemoryWithRecord(false);
-
-    PlanExecutor executor = new PlanExecutor();
-    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
-    executor.processNonQuery(flushPlan);
-
-    IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.isp"));
-    List<DeviceEntry> memoryList = idTable.getAllDeviceEntry();
-
-    // restart
-    try {
-      EnvironmentUtils.restartDaemon();
-    } catch (Exception e) {
-      Assert.fail();
-    }
-
-    // check id table fields
-
-    idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.isp.d1"));
-    List<DeviceEntry> recoverList = idTable.getAllDeviceEntry();
-
-    assertEquals(memoryList, recoverList);
-  }
-
-  @Test
-  public void testRecoverAligned() throws Exception {
-    insertDataInMemoryWithTablet(true);
-    insertDataInMemoryWithRecord(false);
-
-    PlanExecutor executor = new PlanExecutor();
-    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
-    executor.processNonQuery(flushPlan);
-
-    IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.isp"));
-    List<DeviceEntry> memoryList = idTable.getAllDeviceEntry();
-
-    // restart
-    try {
-      EnvironmentUtils.restartDaemon();
-    } catch (Exception e) {
-      Assert.fail();
-    }
-
-    // check id table fields
-
-    idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.isp.d1"));
-    List<DeviceEntry> recoverList = idTable.getAllDeviceEntry();
-
-    assertEquals(memoryList, recoverList);
-  }
-
-  private void insertDataInMemoryWithRecord(boolean isAligned)
-      throws IllegalPathException, QueryProcessException {
-    long time = 100L;
-    TSDataType[] dataTypes =
-        new TSDataType[] {
-          TSDataType.DOUBLE,
-          TSDataType.FLOAT,
-          TSDataType.INT64,
-          TSDataType.INT32,
-          TSDataType.BOOLEAN,
-          TSDataType.TEXT
-        };
-
-    String[] columns = new String[6];
-    columns[0] = 1.0 + "";
-    columns[1] = 2 + "";
-    columns[2] = 10000 + "";
-    columns[3] = 100 + "";
-    columns[4] = false + "";
-    columns[5] = "hh" + 0;
-
-    InsertRowPlan insertRowPlan =
-        new InsertRowPlan(
-            new PartialPath("root.isp.d1"),
-            time,
-            new String[] {"s1", "s2", "s3", "s4", "s5", "s6"},
-            dataTypes,
-            columns);
-    insertRowPlan.setAligned(isAligned);
-
-    PlanExecutor executor = new PlanExecutor();
-    executor.insert(insertRowPlan);
-  }
-
-  private void insertDataInMemoryWithTablet(boolean isAligned)
-      throws IllegalPathException, QueryProcessException {
-    long[] times = new long[] {110L, 111L, 112L, 113L};
-    List<Integer> dataTypes = new ArrayList<>();
-    dataTypes.add(TSDataType.DOUBLE.ordinal());
-    dataTypes.add(TSDataType.FLOAT.ordinal());
-    dataTypes.add(TSDataType.INT64.ordinal());
-    dataTypes.add(TSDataType.INT32.ordinal());
-    dataTypes.add(TSDataType.BOOLEAN.ordinal());
-    dataTypes.add(TSDataType.TEXT.ordinal());
-
-    Object[] columns = new Object[6];
-    columns[0] = new double[4];
-    columns[1] = new float[4];
-    columns[2] = new long[4];
-    columns[3] = new int[4];
-    columns[4] = new boolean[4];
-    columns[5] = new Binary[4];
-
-    for (int r = 0; r < 4; r++) {
-      ((double[]) columns[0])[r] = 10.0 + r;
-      ((float[]) columns[1])[r] = 20 + r;
-      ((long[]) columns[2])[r] = 100000 + r;
-      ((int[]) columns[3])[r] = 1000 + r;
-      ((boolean[]) columns[4])[r] = false;
-      ((Binary[]) columns[5])[r] = new Binary("mm" + r);
-    }
-
-    InsertTabletPlan tabletPlan =
-        new InsertTabletPlan(
-            new PartialPath("root.isp.d2"),
-            new String[] {"s1", "s2", "s3", "s4", "s5", "s6"},
-            dataTypes);
-    tabletPlan.setTimes(times);
-    tabletPlan.setColumns(columns);
-    tabletPlan.setRowCount(times.length);
-    tabletPlan.setAligned(isAligned);
-
-    PlanExecutor executor = new PlanExecutor();
-    executor.insertTablet(tabletPlan);
-  }
-}
+// public class IDTableRecoverTest {
+//  private final Planner processor = new Planner();
+//
+//  private boolean isEnableIDTable = false;
+//
+//  private String originalDeviceIDTransformationMethod = null;
+//
+//  private boolean isEnableIDTableLogFile = false;
+//
+//  @Before
+//  public void before() {
+//    IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(true);
+//    isEnableIDTable = IoTDBDescriptor.getInstance().getConfig().isEnableIDTable();
+//    originalDeviceIDTransformationMethod =
+//        IoTDBDescriptor.getInstance().getConfig().getDeviceIDTransformationMethod();
+//    isEnableIDTableLogFile = IoTDBDescriptor.getInstance().getConfig().isEnableIDTableLogFile();
+//
+//    IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(true);
+//    IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("SHA256");
+//    IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(true);
+//    EnvironmentUtils.envSetUp();
+//  }
+//
+//  @After
+//  public void clean() throws IOException, StorageEngineException {
+//    IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(isEnableIDTable);
+//    IoTDBDescriptor.getInstance()
+//        .getConfig()
+//        .setDeviceIDTransformationMethod(originalDeviceIDTransformationMethod);
+//    IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(isEnableIDTableLogFile);
+//
+//    EnvironmentUtils.cleanEnv();
+//  }
+//
+//  @Test
+//  public void testRecover() throws Exception {
+//    insertDataInMemoryWithTablet(false);
+//    insertDataInMemoryWithRecord(false);
+//
+//    PlanExecutor executor = new PlanExecutor();
+//    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
+//    executor.processNonQuery(flushPlan);
+//
+//    IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.isp"));
+//    List<DeviceEntry> memoryList = idTable.getAllDeviceEntry();
+//
+//    // restart
+//    try {
+//      EnvironmentUtils.restartDaemon();
+//    } catch (Exception e) {
+//      Assert.fail();
+//    }
+//
+//    // check id table fields
+//
+//    idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.isp.d1"));
+//    List<DeviceEntry> recoverList = idTable.getAllDeviceEntry();
+//
+//    assertEquals(memoryList, recoverList);
+//  }
+//
+//  @Test
+//  public void testRecoverAligned() throws Exception {
+//    insertDataInMemoryWithTablet(true);
+//    insertDataInMemoryWithRecord(false);
+//
+//    PlanExecutor executor = new PlanExecutor();
+//    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
+//    executor.processNonQuery(flushPlan);
+//
+//    IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.isp"));
+//    List<DeviceEntry> memoryList = idTable.getAllDeviceEntry();
+//
+//    // restart
+//    try {
+//      EnvironmentUtils.restartDaemon();
+//    } catch (Exception e) {
+//      Assert.fail();
+//    }
+//
+//    // check id table fields
+//
+//    idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.isp.d1"));
+//    List<DeviceEntry> recoverList = idTable.getAllDeviceEntry();
+//
+//    assertEquals(memoryList, recoverList);
+//  }
+//
+//  private void insertDataInMemoryWithRecord(boolean isAligned)
+//      throws IllegalPathException, QueryProcessException {
+//    long time = 100L;
+//    TSDataType[] dataTypes =
+//        new TSDataType[] {
+//          TSDataType.DOUBLE,
+//          TSDataType.FLOAT,
+//          TSDataType.INT64,
+//          TSDataType.INT32,
+//          TSDataType.BOOLEAN,
+//          TSDataType.TEXT
+//        };
+//
+//    String[] columns = new String[6];
+//    columns[0] = 1.0 + "";
+//    columns[1] = 2 + "";
+//    columns[2] = 10000 + "";
+//    columns[3] = 100 + "";
+//    columns[4] = false + "";
+//    columns[5] = "hh" + 0;
+//
+//    InsertRowPlan insertRowPlan =
+//        new InsertRowPlan(
+//            new PartialPath("root.isp.d1"),
+//            time,
+//            new String[] {"s1", "s2", "s3", "s4", "s5", "s6"},
+//            dataTypes,
+//            columns);
+//    insertRowPlan.setAligned(isAligned);
+//
+//    PlanExecutor executor = new PlanExecutor();
+//    executor.insert(insertRowPlan);
+//  }
+//
+//  private void insertDataInMemoryWithTablet(boolean isAligned)
+//      throws IllegalPathException, QueryProcessException {
+//    long[] times = new long[] {110L, 111L, 112L, 113L};
+//    List<Integer> dataTypes = new ArrayList<>();
+//    dataTypes.add(TSDataType.DOUBLE.ordinal());
+//    dataTypes.add(TSDataType.FLOAT.ordinal());
+//    dataTypes.add(TSDataType.INT64.ordinal());
+//    dataTypes.add(TSDataType.INT32.ordinal());
+//    dataTypes.add(TSDataType.BOOLEAN.ordinal());
+//    dataTypes.add(TSDataType.TEXT.ordinal());
+//
+//    Object[] columns = new Object[6];
+//    columns[0] = new double[4];
+//    columns[1] = new float[4];
+//    columns[2] = new long[4];
+//    columns[3] = new int[4];
+//    columns[4] = new boolean[4];
+//    columns[5] = new Binary[4];
+//
+//    for (int r = 0; r < 4; r++) {
+//      ((double[]) columns[0])[r] = 10.0 + r;
+//      ((float[]) columns[1])[r] = 20 + r;
+//      ((long[]) columns[2])[r] = 100000 + r;
+//      ((int[]) columns[3])[r] = 1000 + r;
+//      ((boolean[]) columns[4])[r] = false;
+//      ((Binary[]) columns[5])[r] = new Binary("mm" + r);
+//    }
+//
+//    InsertTabletPlan tabletPlan =
+//        new InsertTabletPlan(
+//            new PartialPath("root.isp.d2"),
+//            new String[] {"s1", "s2", "s3", "s4", "s5", "s6"},
+//            dataTypes);
+//    tabletPlan.setTimes(times);
+//    tabletPlan.setColumns(columns);
+//    tabletPlan.setRowCount(times.length);
+//    tabletPlan.setAligned(isAligned);
+//
+//    PlanExecutor executor = new PlanExecutor();
+//    executor.insertTablet(tabletPlan);
+//  }
+// }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/LastQueryWithIDTable.java b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/LastQueryWithIDTable.java
index 33bbd75d0a..50d8163788 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/LastQueryWithIDTable.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/LastQueryWithIDTable.java
@@ -18,163 +18,128 @@
  */
 package org.apache.iotdb.db.metadata.idtable;
 
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.idtable.entry.TimeseriesID;
-import org.apache.iotdb.db.qp.Planner;
-import org.apache.iotdb.db.qp.executor.PlanExecutor;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
-import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.read.common.RowRecord;
-import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
-import org.apache.iotdb.tsfile.utils.Binary;
-
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-public class LastQueryWithIDTable {
-  private final Planner processor = new Planner();
-
-  private boolean isEnableIDTable = false;
-
-  private boolean originalEnableCache = false;
-
-  private String originalDeviceIDTransformationMethod = null;
-
-  Set<String> retSet =
-      new HashSet<>(
-          Arrays.asList(
-              "113\troot.isp.d1.s3\t100003\tINT64",
-              "113\troot.isp.d1.s4\t1003\tINT32",
-              "113\troot.isp.d1.s5\tfalse\tBOOLEAN",
-              "113\troot.isp.d1.s6\tmm3\tTEXT",
-              "113\troot.isp.d1.s1\t13.0\tDOUBLE",
-              "113\troot.isp.d1.s2\t23.0\tFLOAT"));
-
-  @Before
-  public void before() {
-    IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(true);
-    isEnableIDTable = IoTDBDescriptor.getInstance().getConfig().isEnableIDTable();
-    originalDeviceIDTransformationMethod =
-        IoTDBDescriptor.getInstance().getConfig().getDeviceIDTransformationMethod();
-    originalEnableCache = IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled();
-    IoTDBDescriptor.getInstance().getConfig().setEnableLastCache(false);
-
-    IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(true);
-    IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("SHA256");
-    EnvironmentUtils.envSetUp();
-  }
-
-  @After
-  public void clean() throws IOException, StorageEngineException {
-    IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(isEnableIDTable);
-    IoTDBDescriptor.getInstance()
-        .getConfig()
-        .setDeviceIDTransformationMethod(originalDeviceIDTransformationMethod);
-    IoTDBDescriptor.getInstance().getConfig().setEnableLastCache(originalEnableCache);
-    EnvironmentUtils.cleanEnv();
-  }
-
-  @Test
-  public void testLastCacheQueryWithoutCache()
-      throws QueryProcessException, MetadataException, InterruptedException,
-          QueryFilterOptimizationException, StorageEngineException, IOException {
-
-    insertDataInMemory();
-
-    PlanExecutor executor = new PlanExecutor();
-    QueryPlan queryPlan =
-        (QueryPlan) processor.parseSQLToPhysicalPlan("select last * from root.isp.d1");
-    QueryDataSet dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
-    Assert.assertEquals(3, dataSet.getPaths().size());
-    int count = 0;
-    while (dataSet.hasNext()) {
-      RowRecord record = dataSet.next();
-      assertTrue(retSet.contains(record.toString()));
-      count++;
-    }
-
-    assertEquals(retSet.size(), count);
-
-    // flush and test again
-    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
-    executor.processNonQuery(flushPlan);
-
-    dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
-    Assert.assertEquals(3, dataSet.getPaths().size());
-    count = 0;
-    while (dataSet.hasNext()) {
-      RowRecord record = dataSet.next();
-      assertTrue(retSet.contains(record.toString()));
-      count++;
-    }
-    assertEquals(retSet.size(), count);
-
-    // assert id table is not refresh
-    assertNull(
-        IDTableManager.getInstance()
-            .getIDTable(new PartialPath("root.isp.d1"))
-            .getLastCache(new TimeseriesID(new PartialPath("root.isp.d1.s1"))));
-  }
-
-  private void insertDataInMemory() throws IllegalPathException, QueryProcessException {
-    long[] times = new long[] {110L, 111L, 112L, 113L};
-    List<Integer> dataTypes = new ArrayList<>();
-    dataTypes.add(TSDataType.DOUBLE.ordinal());
-    dataTypes.add(TSDataType.FLOAT.ordinal());
-    dataTypes.add(TSDataType.INT64.ordinal());
-    dataTypes.add(TSDataType.INT32.ordinal());
-    dataTypes.add(TSDataType.BOOLEAN.ordinal());
-    dataTypes.add(TSDataType.TEXT.ordinal());
-
-    Object[] columns = new Object[6];
-    columns[0] = new double[4];
-    columns[1] = new float[4];
-    columns[2] = new long[4];
-    columns[3] = new int[4];
-    columns[4] = new boolean[4];
-    columns[5] = new Binary[4];
-
-    for (int r = 0; r < 4; r++) {
-      ((double[]) columns[0])[r] = 10.0 + r;
-      ((float[]) columns[1])[r] = 20 + r;
-      ((long[]) columns[2])[r] = 100000 + r;
-      ((int[]) columns[3])[r] = 1000 + r;
-      ((boolean[]) columns[4])[r] = false;
-      ((Binary[]) columns[5])[r] = new Binary("mm" + r);
-    }
-
-    InsertTabletPlan tabletPlan =
-        new InsertTabletPlan(
-            new PartialPath("root.isp.d1"),
-            new String[] {"s1", "s2", "s3", "s4", "s5", "s6"},
-            dataTypes);
-    tabletPlan.setTimes(times);
-    tabletPlan.setColumns(columns);
-    tabletPlan.setRowCount(times.length);
-
-    PlanExecutor executor = new PlanExecutor();
-    executor.insertTablet(tabletPlan);
-  }
-}
+// public class LastQueryWithIDTable {
+//  private final Planner processor = new Planner();
+//
+//  private boolean isEnableIDTable = false;
+//
+//  private boolean originalEnableCache = false;
+//
+//  private String originalDeviceIDTransformationMethod = null;
+//
+//  Set<String> retSet =
+//      new HashSet<>(
+//          Arrays.asList(
+//              "113\troot.isp.d1.s3\t100003\tINT64",
+//              "113\troot.isp.d1.s4\t1003\tINT32",
+//              "113\troot.isp.d1.s5\tfalse\tBOOLEAN",
+//              "113\troot.isp.d1.s6\tmm3\tTEXT",
+//              "113\troot.isp.d1.s1\t13.0\tDOUBLE",
+//              "113\troot.isp.d1.s2\t23.0\tFLOAT"));
+//
+//  @Before
+//  public void before() {
+//    IoTDBDescriptor.getInstance().getConfig().setAutoCreateSchemaEnabled(true);
+//    isEnableIDTable = IoTDBDescriptor.getInstance().getConfig().isEnableIDTable();
+//    originalDeviceIDTransformationMethod =
+//        IoTDBDescriptor.getInstance().getConfig().getDeviceIDTransformationMethod();
+//    originalEnableCache = IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled();
+//    IoTDBDescriptor.getInstance().getConfig().setEnableLastCache(false);
+//
+//    IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(true);
+//    IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("SHA256");
+//    EnvironmentUtils.envSetUp();
+//  }
+//
+//  @After
+//  public void clean() throws IOException, StorageEngineException {
+//    IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(isEnableIDTable);
+//    IoTDBDescriptor.getInstance()
+//        .getConfig()
+//        .setDeviceIDTransformationMethod(originalDeviceIDTransformationMethod);
+//    IoTDBDescriptor.getInstance().getConfig().setEnableLastCache(originalEnableCache);
+//    EnvironmentUtils.cleanEnv();
+//  }
+//
+//  @Test
+//  public void testLastCacheQueryWithoutCache()
+//      throws QueryProcessException, MetadataException, InterruptedException,
+//          QueryFilterOptimizationException, StorageEngineException, IOException {
+//
+//    insertDataInMemory();
+//
+//    PlanExecutor executor = new PlanExecutor();
+//    QueryPlan queryPlan =
+//        (QueryPlan) processor.parseSQLToPhysicalPlan("select last * from root.isp.d1");
+//    QueryDataSet dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+//    Assert.assertEquals(3, dataSet.getPaths().size());
+//    int count = 0;
+//    while (dataSet.hasNext()) {
+//      RowRecord record = dataSet.next();
+//      assertTrue(retSet.contains(record.toString()));
+//      count++;
+//    }
+//
+//    assertEquals(retSet.size(), count);
+//
+//    // flush and test again
+//    PhysicalPlan flushPlan = processor.parseSQLToPhysicalPlan("flush");
+//    executor.processNonQuery(flushPlan);
+//
+//    dataSet = executor.processQuery(queryPlan, EnvironmentUtils.TEST_QUERY_CONTEXT);
+//    Assert.assertEquals(3, dataSet.getPaths().size());
+//    count = 0;
+//    while (dataSet.hasNext()) {
+//      RowRecord record = dataSet.next();
+//      assertTrue(retSet.contains(record.toString()));
+//      count++;
+//    }
+//    assertEquals(retSet.size(), count);
+//
+//    // assert id table is not refresh
+//    assertNull(
+//        IDTableManager.getInstance()
+//            .getIDTable(new PartialPath("root.isp.d1"))
+//            .getLastCache(new TimeseriesID(new PartialPath("root.isp.d1.s1"))));
+//  }
+//
+//  private void insertDataInMemory() throws IllegalPathException, QueryProcessException {
+//    long[] times = new long[] {110L, 111L, 112L, 113L};
+//    List<Integer> dataTypes = new ArrayList<>();
+//    dataTypes.add(TSDataType.DOUBLE.ordinal());
+//    dataTypes.add(TSDataType.FLOAT.ordinal());
+//    dataTypes.add(TSDataType.INT64.ordinal());
+//    dataTypes.add(TSDataType.INT32.ordinal());
+//    dataTypes.add(TSDataType.BOOLEAN.ordinal());
+//    dataTypes.add(TSDataType.TEXT.ordinal());
+//
+//    Object[] columns = new Object[6];
+//    columns[0] = new double[4];
+//    columns[1] = new float[4];
+//    columns[2] = new long[4];
+//    columns[3] = new int[4];
+//    columns[4] = new boolean[4];
+//    columns[5] = new Binary[4];
+//
+//    for (int r = 0; r < 4; r++) {
+//      ((double[]) columns[0])[r] = 10.0 + r;
+//      ((float[]) columns[1])[r] = 20 + r;
+//      ((long[]) columns[2])[r] = 100000 + r;
+//      ((int[]) columns[3])[r] = 1000 + r;
+//      ((boolean[]) columns[4])[r] = false;
+//      ((Binary[]) columns[5])[r] = new Binary("mm" + r);
+//    }
+//
+//    InsertTabletPlan tabletPlan =
+//        new InsertTabletPlan(
+//            new PartialPath("root.isp.d1"),
+//            new String[] {"s1", "s2", "s3", "s4", "s5", "s6"},
+//            dataTypes);
+//    tabletPlan.setTimes(times);
+//    tabletPlan.setColumns(columns);
+//    tabletPlan.setRowCount(times.length);
+//
+//    PlanExecutor executor = new PlanExecutor();
+//    executor.insertTablet(tabletPlan);
+//  }
+// }
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/physical/ConcatOptimizerTest.java b/server/src/test/java/org/apache/iotdb/db/qp/physical/ConcatOptimizerTest.java
deleted file mode 100644
index 7f63328147..0000000000
--- a/server/src/test/java/org/apache/iotdb/db/qp/physical/ConcatOptimizerTest.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.qp.physical;
-
-import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.Planner;
-import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
-import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
-import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
-import org.apache.iotdb.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.read.expression.IExpression;
-import org.apache.iotdb.tsfile.read.expression.impl.BinaryExpression;
-import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
-import org.apache.iotdb.tsfile.read.filter.ValueFilter;
-
-import org.antlr.v4.runtime.RecognitionException;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-
-/** test the correctness of {@linkplain ConcatPathOptimizer ConcatPathOptimizer} */
-public class ConcatOptimizerTest {
-
-  private Planner processor;
-
-  @Before
-  public void before() throws MetadataException {
-    processor = new Planner();
-    IoTDB.configManager.init();
-    IoTDB.schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
-    IoTDB.schemaProcessor.createTimeseries(
-        new PartialPath("root.laptop.d1.s1"),
-        TSDataType.INT64,
-        TSEncoding.PLAIN,
-        CompressionType.UNCOMPRESSED,
-        null);
-    IoTDB.schemaProcessor.createTimeseries(
-        new PartialPath("root.laptop.d1.s2"),
-        TSDataType.INT64,
-        TSEncoding.PLAIN,
-        CompressionType.UNCOMPRESSED,
-        null);
-    IoTDB.schemaProcessor.createTimeseries(
-        new PartialPath("root.laptop.d2.s1"),
-        TSDataType.INT64,
-        TSEncoding.PLAIN,
-        CompressionType.UNCOMPRESSED,
-        null);
-    IoTDB.schemaProcessor.createTimeseries(
-        new PartialPath("root.laptop.d2.s2"),
-        TSDataType.INT64,
-        TSEncoding.PLAIN,
-        CompressionType.UNCOMPRESSED,
-        null);
-    IoTDB.schemaProcessor.createTimeseries(
-        new PartialPath("root.laptop.d3.s1"),
-        TSDataType.INT64,
-        TSEncoding.PLAIN,
-        CompressionType.UNCOMPRESSED,
-        null);
-    IoTDB.schemaProcessor.createTimeseries(
-        new PartialPath("root.laptop.d3.s2"),
-        TSDataType.INT64,
-        TSEncoding.PLAIN,
-        CompressionType.UNCOMPRESSED,
-        null);
-  }
-
-  @After
-  public void after() throws IOException {
-    IoTDB.configManager.clear();
-    EnvironmentUtils.cleanAllDir();
-  }
-
-  @Test
-  public void testConcat1() throws QueryProcessException, RecognitionException {
-    String inputSQL = "select s1 from root.laptop.d1";
-    PhysicalPlan plan = processor.parseSQLToPhysicalPlan(inputSQL);
-    assertEquals("root.laptop.d1.s1", plan.getPaths().get(0).getFullPath());
-  }
-
-  @Test
-  public void testConcat2() throws QueryProcessException, RecognitionException {
-    String inputSQL = "select s1 from root.laptop.*";
-    PhysicalPlan plan = processor.parseSQLToPhysicalPlan(inputSQL);
-    assertEquals("root.laptop.d1.s1", plan.getPaths().get(0).getFullPath());
-    assertEquals("root.laptop.d2.s1", plan.getPaths().get(1).getFullPath());
-    assertEquals("root.laptop.d3.s1", plan.getPaths().get(2).getFullPath());
-  }
-
-  @Test
-  public void testConcat3() throws QueryProcessException, RecognitionException {
-    String inputSQL = "select s1 from root.laptop.d1 where s1 < 10";
-    PhysicalPlan plan = processor.parseSQLToPhysicalPlan(inputSQL);
-    SingleSeriesExpression seriesExpression =
-        new SingleSeriesExpression(new Path("root.laptop.d1", "s1", true), ValueFilter.lt(10));
-    assertEquals(seriesExpression.toString(), ((RawDataQueryPlan) plan).getExpression().toString());
-  }
-
-  @Test
-  public void testConcatMultipleDeviceInFilter() throws QueryProcessException {
-    String inputSQL = "select s1 from root.laptop.* where s1 < 10";
-    PhysicalPlan plan = processor.parseSQLToPhysicalPlan(inputSQL);
-    IExpression expression =
-        BinaryExpression.and(
-            BinaryExpression.and(
-                new SingleSeriesExpression(
-                    new Path("root.laptop.d1", "s1", true), ValueFilter.lt(10)),
-                new SingleSeriesExpression(
-                    new Path("root.laptop.d2", "s1", true), ValueFilter.lt(10))),
-            new SingleSeriesExpression(new Path("root.laptop.d3", "s1", true), ValueFilter.lt(10)));
-    assertEquals(expression.toString(), ((RawDataQueryPlan) plan).getExpression().toString());
-  }
-}
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertRowsOfOneDevicePlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertRowsOfOneDevicePlanTest.java
deleted file mode 100644
index 13c2b8503d..0000000000
--- a/server/src/test/java/org/apache/iotdb/db/qp/physical/InsertRowsOfOneDevicePlanTest.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
... 4567 lines suppressed ...