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/10 04:20:11 UTC

[iotdb] 01/01: Remove old storage engine step 2

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

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

commit 39803c82e6685fc2730337daac9eb1ba1a1f3b00
Author: HTHou <hh...@outlook.com>
AuthorDate: Thu Nov 10 12:19:42 2022 +0800

    Remove old storage engine step 2
---
 .../schemaregion/rocksdb/RSchemaRegion.java        |  131 --
 .../metadata/tagSchemaRegion/TagSchemaRegion.java  |   85 --
 .../selectinto/InsertTabletPlanGenerator.java      |  252 ----
 .../selectinto/InsertTabletPlansIterator.java      |  190 ---
 .../iotdb/db/engine/storagegroup/DataRegion.java   |   14 -
 .../db/engine/trigger/executor/TriggerEngine.java  |   64 -
 .../trigger/sink/local/LocalIoTDBHandler.java      |   12 +-
 .../iotdb/db/metadata/LocalSchemaProcessor.java    |   33 -
 .../apache/iotdb/db/metadata/idtable/IDTable.java  |    3 +-
 .../db/metadata/idtable/IDTableHashmapImpl.java    |  134 --
 .../db/metadata/schemaregion/ISchemaRegion.java    |    3 -
 .../schemaregion/SchemaRegionMemoryImpl.java       |  211 ---
 .../schemaregion/SchemaRegionSchemaFileImpl.java   |  230 ---
 .../metadata/schemaregion/SchemaRegionUtils.java   |   48 -
 .../planner/plan/node/write/InsertTabletNode.java  |    2 +-
 .../iotdb/db/protocol/influxdb/dto/IoTDBPoint.java |   14 -
 .../db/protocol/influxdb/meta/TagInfoRecords.java  |   24 -
 .../db/qp/logical/crud/DeleteDataOperator.java     |   15 +-
 .../iotdb/db/qp/logical/crud/InsertOperator.java   |   31 +-
 .../db/qp/logical/sys/DeletePartitionOperator.java |    3 +-
 .../apache/iotdb/db/qp/physical/PhysicalPlan.java  |   24 -
 .../db/qp/physical/crud/DeletePartitionPlan.java   |   53 -
 .../iotdb/db/qp/physical/crud/DeletePlan.java      |  221 ---
 .../qp/physical/crud/InsertMultiTabletsPlan.java   |  453 ------
 .../iotdb/db/qp/physical/crud/InsertPlan.java      |  239 ----
 .../iotdb/db/qp/physical/crud/InsertRowPlan.java   |  812 -----------
 .../physical/crud/InsertRowsOfOneDevicePlan.java   |  296 ----
 .../iotdb/db/qp/physical/crud/InsertRowsPlan.java  |  302 ----
 .../db/qp/physical/crud/InsertTabletPlan.java      |  895 ------------
 .../service/thrift/impl/ClientRPCServiceImpl.java  |   10 +-
 .../db/service/thrift/impl/TSServiceImpl.java      |  548 +-------
 .../java/org/apache/iotdb/db/utils/MemUtils.java   |   54 -
 .../apache/iotdb/db/wal/buffer/WALEntryType.java   |    4 -
 .../apache/iotdb/db/wal/buffer/WALInfoEntry.java   |    2 +-
 .../db/wal/recover/file/TsFilePlanRedoer.java      |   28 -
 .../apache/iotdb/db/metadata/SchemaBasicTest.java  |  298 ----
 .../org/apache/iotdb/db/metadata/TemplateTest.java |   14 -
 .../db/metadata/idtable/IDTableLogFileTest.java    |  239 ++--
 .../iotdb/db/metadata/idtable/IDTableTest.java     | 1479 ++++++++++----------
 .../java/org/apache/iotdb/db/qp/PlannerTest.java   |   24 -
 .../db/utils/writelog/LogWriterReaderTest.java     |   50 +-
 .../db/utils/writelog/MultiFileLogReaderTest.java  |   15 +-
 .../db/wal/recover/file/TsFilePlanRedoerTest.java  |    4 +-
 43 files changed, 921 insertions(+), 6642 deletions(-)

diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
index e7a41ef35b..95bf1f77f4 100644
--- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
+++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java
@@ -35,7 +35,6 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.metadata.AcquireLockTimeoutException;
 import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
 import org.apache.iotdb.db.exception.metadata.MNodeTypeMismatchException;
 import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
@@ -67,15 +66,11 @@ import org.apache.iotdb.db.metadata.schemaregion.rocksdb.mnode.RMeasurementMNode
 import org.apache.iotdb.db.metadata.template.Template;
 import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
 import org.apache.iotdb.db.mpp.common.schematree.DeviceSchemaInfo;
-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.qp.physical.sys.ShowDevicesPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
 import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
-import org.apache.iotdb.db.utils.EncodingInferenceUtils;
 import org.apache.iotdb.db.utils.SchemaUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -102,7 +97,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
@@ -1837,131 +1831,6 @@ public class RSchemaRegion implements ISchemaRegion {
     }
   }
 
-  @Override
-  public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
-      throws MetadataException, IOException {
-    // devicePath is a logical path which is parent of measurement, whether in template or not
-    PartialPath devicePath = plan.getDevicePath();
-    String[] measurementList = plan.getMeasurements();
-    IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
-
-    IMNode deviceMNode = getDeviceNodeWithAutoCreate(devicePath, plan.isAligned());
-
-    if (deviceMNode == null) {
-      throw new MetadataException(
-          String.format("Failed to create deviceMNode,device path:[%s]", plan.getDevicePath()));
-    }
-    // check insert non-aligned InsertPlan for aligned timeseries
-    if (deviceMNode.isEntity()) {
-      if (plan.isAligned() && !deviceMNode.getAsEntityMNode().isAligned()) {
-        throw new AlignedTimeseriesException(
-            "timeseries under this device are not aligned, " + "please use non-aligned interface",
-            devicePath.getFullPath());
-      }
-
-      if (!plan.isAligned() && deviceMNode.getAsEntityMNode().isAligned()) {
-        throw new AlignedTimeseriesException(
-            "timeseries under this device are aligned, " + "please use aligned interface",
-            devicePath.getFullPath());
-      }
-    }
-
-    // get node for each measurement
-    Map<Integer, IMeasurementMNode> nodeMap = new HashMap<>();
-    Map<Integer, PartialPath> missingNodeIndex = new HashMap<>();
-    for (int i = 0; i < measurementList.length; i++) {
-      PartialPath path = new PartialPath(devicePath.getFullPath(), measurementList[i]);
-      IMeasurementMNode node = getMeasurementMNode(path);
-      if (node == null) {
-        if (!config.isAutoCreateSchemaEnabled()) {
-          throw new PathNotExistException(path.getFullPath());
-        }
-        missingNodeIndex.put(i, path);
-      } else {
-        nodeMap.put(i, node);
-      }
-    }
-
-    // create missing nodes
-    if (!missingNodeIndex.isEmpty()) {
-      if (!(plan instanceof InsertRowPlan) && !(plan instanceof InsertTabletPlan)) {
-        throw new MetadataException(
-            String.format(
-                "Only support insertRow and insertTablet, plan is [%s]", plan.getOperatorType()));
-      }
-
-      if (plan.isAligned()) {
-        List<String> measurements = new ArrayList<>();
-        List<TSDataType> dataTypes = new ArrayList<>();
-        List<TSEncoding> encodings = new ArrayList<>();
-        for (Integer index : missingNodeIndex.keySet()) {
-          measurements.add(measurementList[index]);
-          TSDataType type = plan.getDataTypes()[index];
-          dataTypes.add(type);
-          encodings.add(EncodingInferenceUtils.getDefaultEncoding(type));
-        }
-        createAlignedTimeSeries(devicePath, measurements, dataTypes, encodings);
-      } else {
-        for (Map.Entry<Integer, PartialPath> entry : missingNodeIndex.entrySet()) {
-          IMeasurementSchema schema =
-              new MeasurementSchema(
-                  entry.getValue().getMeasurement(), plan.getDataTypes()[entry.getKey()]);
-          createTimeseries(entry.getValue(), schema, null, null, null);
-        }
-      }
-
-      // get the latest node
-      for (Entry<Integer, PartialPath> entry : missingNodeIndex.entrySet()) {
-        nodeMap.put(entry.getKey(), getMeasurementMNode(entry.getValue()));
-      }
-    }
-
-    // check datatype
-    for (int i = 0; i < measurementList.length; i++) {
-      try {
-        // check type is match
-        if (plan instanceof InsertRowPlan || plan instanceof InsertTabletPlan) {
-          try {
-            SchemaRegionUtils.checkDataTypeMatch(plan, i, nodeMap.get(i).getSchema().getType());
-          } catch (DataTypeMismatchException mismatchException) {
-            logger.warn(mismatchException.getMessage());
-            if (!config.isEnablePartialInsert()) {
-              throw mismatchException;
-            } else {
-              // mark failed measurement
-              plan.markFailedMeasurementInsertion(i, mismatchException);
-              continue;
-            }
-          }
-          measurementMNodes[i] = nodeMap.get(i);
-          // set measurementName instead of alias
-          measurementList[i] = nodeMap.get(i).getName();
-        }
-      } catch (MetadataException e) {
-        if (config.isClusterMode()) {
-          logger.debug(
-              "meet error when check {}.{}, message: {}",
-              devicePath,
-              measurementList[i],
-              e.getMessage());
-        } else {
-          logger.warn(
-              "meet error when check {}.{}, message: {}",
-              devicePath,
-              measurementList[i],
-              e.getMessage());
-        }
-        if (config.isEnablePartialInsert()) {
-          // mark failed measurement
-          plan.markFailedMeasurementInsertion(i, e);
-        } else {
-          throw e;
-        }
-      }
-    }
-    return deviceMNode;
-  }
-
   @Override
   public DeviceSchemaInfo getDeviceSchemaInfoWithAutoCreate(
       PartialPath devicePath,
diff --git a/schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaRegion.java b/schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaRegion.java
index 38cc03c953..174f989e86 100644
--- a/schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaRegion.java
+++ b/schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaRegion.java
@@ -30,7 +30,6 @@ 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.exception.metadata.AlignedTimeseriesException;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
 import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
 import org.apache.iotdb.db.exception.metadata.SchemaDirCreationFailureException;
@@ -38,7 +37,6 @@ import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
 import org.apache.iotdb.db.metadata.idtable.entry.DeviceEntry;
 import org.apache.iotdb.db.metadata.idtable.entry.DiskSchemaEntry;
 import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
-import org.apache.iotdb.db.metadata.idtable.entry.InsertMeasurementMNode;
 import org.apache.iotdb.db.metadata.idtable.entry.SHA256DeviceID;
 import org.apache.iotdb.db.metadata.idtable.entry.SchemaEntry;
 import org.apache.iotdb.db.metadata.mnode.EntityMNode;
@@ -65,7 +63,6 @@ import org.apache.iotdb.db.metadata.tagSchemaRegion.utils.ShowTimeSeriesResultUt
 import org.apache.iotdb.db.metadata.template.Template;
 import org.apache.iotdb.db.mpp.common.schematree.DeviceSchemaInfo;
 import org.apache.iotdb.db.mpp.common.schematree.MeasurementSchemaInfo;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
@@ -87,7 +84,6 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -739,44 +735,6 @@ public class TagSchemaRegion implements ISchemaRegion {
     throw new UnsupportedOperationException("renameTagOrAttributeKey");
   }
 
-  @Override
-  public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
-      throws MetadataException, IOException {
-    PartialPath devicePath = plan.getDevicePath();
-    devicePath =
-        new PartialPath(
-            PathTagConverterUtils.pathToTagsSortPath(
-                storageGroupFullPath, devicePath.getFullPath()));
-    plan.setDevicePath(devicePath);
-    String[] measurementList = plan.getMeasurements();
-    IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
-    checkAlignedAndAutoCreateSeries(plan);
-    IMNode deviceMNode = getDeviceNode(devicePath);
-    IMeasurementMNode measurementMNode;
-    DeviceEntry deviceEntry = idTableWithDeviceIDList.getDeviceEntry(devicePath.getFullPath());
-    Map<String, SchemaEntry> schemaMap = deviceEntry.getMeasurementMap();
-    for (int i = 0; i < measurementList.length; i++) {
-      SchemaEntry schemaEntry = schemaMap.get(measurementList[i]);
-      measurementMNode = new InsertMeasurementMNode(measurementList[i], schemaEntry, null);
-      // check type is match
-      try {
-        SchemaRegionUtils.checkDataTypeMatch(plan, i, schemaEntry.getTSDataType());
-      } catch (DataTypeMismatchException mismatchException) {
-        if (!config.isEnablePartialInsert()) {
-          throw mismatchException;
-        } else {
-          // mark failed measurement
-          plan.markFailedMeasurementInsertion(i, mismatchException);
-          continue;
-        }
-      }
-      measurementMNodes[i] = measurementMNode;
-    }
-    plan.setDeviceID(deviceEntry.getDeviceID());
-    plan.setDevicePath(new PartialPath(deviceEntry.getDeviceID().toStringID(), false));
-    return deviceMNode;
-  }
-
   @Override
   public DeviceSchemaInfo getDeviceSchemaInfoWithAutoCreate(
       PartialPath devicePath,
@@ -833,36 +791,6 @@ public class TagSchemaRegion implements ISchemaRegion {
     return deviceEntry.getSchemaEntry(measurementName);
   }
 
-  private void checkAlignedAndAutoCreateSeries(InsertPlan plan) throws MetadataException {
-    String[] measurementList = plan.getMeasurements();
-    try {
-      if (plan.isAligned()) {
-        internalAlignedCreateTimeseries(
-            plan.getDevicePath(),
-            Arrays.asList(measurementList),
-            Arrays.asList(plan.getDataTypes()));
-      } else {
-        internalCreateTimeseries(
-            plan.getDevicePath().concatNode(measurementList[0]), plan.getDataTypes()[0]);
-      }
-    } catch (MetadataException e) {
-      if (!(e instanceof PathAlreadyExistException)) {
-        throw e;
-      }
-    }
-  }
-
-  /** create timeseries ignoring PathAlreadyExistException */
-  private void internalCreateTimeseries(PartialPath path, TSDataType dataType)
-      throws MetadataException {
-    createTimeseries(
-        path,
-        dataType,
-        getDefaultEncoding(dataType),
-        TSFileDescriptor.getInstance().getConfig().getCompressor(),
-        Collections.emptyMap());
-  }
-
   /** create timeseries ignoring PathAlreadyExistException */
   private void internalCreateTimeseries(
       PartialPath path, TSDataType dataType, TSEncoding encoding, CompressionType compressor)
@@ -876,19 +804,6 @@ public class TagSchemaRegion implements ISchemaRegion {
     createTimeseries(path, dataType, encoding, compressor, Collections.emptyMap());
   }
 
-  /** create aligned timeseries ignoring PathAlreadyExistException */
-  private void internalAlignedCreateTimeseries(
-      PartialPath prefixPath, List<String> measurements, List<TSDataType> dataTypes)
-      throws MetadataException {
-    List<TSEncoding> encodings = new ArrayList<>();
-    List<CompressionType> compressors = new ArrayList<>();
-    for (TSDataType dataType : dataTypes) {
-      encodings.add(getDefaultEncoding(dataType));
-      compressors.add(TSFileDescriptor.getInstance().getConfig().getCompressor());
-    }
-    createAlignedTimeSeries(prefixPath, measurements, dataTypes, encodings, compressors);
-  }
-
   /** create aligned timeseries ignoring PathAlreadyExistException */
   private void internalAlignedCreateTimeseries(
       PartialPath prefixPath,
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlanGenerator.java b/server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlanGenerator.java
deleted file mode 100644
index affa549ddf..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlanGenerator.java
+++ /dev/null
@@ -1,252 +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.selectinto;
-
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.read.common.Field;
-import org.apache.iotdb.tsfile.read.common.RowRecord;
-import org.apache.iotdb.tsfile.utils.Binary;
-import org.apache.iotdb.tsfile.utils.BitMap;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-/** internallyConstructNewPlan -> collectRowRecord * N -> generateInsertTabletPlan */
-public class InsertTabletPlanGenerator {
-
-  private final String targetDevice;
-  // the index of column in InsertTabletPlan -> the index of output column of query data set
-  private final List<Integer> queryDataSetIndexes;
-  // the index of column in InsertTabletPlan -> the measurement id of the column
-  private final List<String> targetMeasurementIds;
-
-  private final int tabletRowLimit;
-  private final boolean isIntoPathsAligned;
-
-  // the following fields are used to construct plan
-  private int rowCount;
-  private long[] times;
-  private Object[] columns;
-  private BitMap[] bitMaps;
-  private TSDataType[] dataTypes;
-
-  private int numberOfInitializedColumns;
-
-  public InsertTabletPlanGenerator(
-      String targetDevice, int tabletRowLimit, boolean isIntoPathsAligned) {
-    this.targetDevice = targetDevice;
-    queryDataSetIndexes = new ArrayList<>();
-    targetMeasurementIds = new ArrayList<>();
-
-    this.tabletRowLimit = tabletRowLimit;
-
-    this.isIntoPathsAligned = isIntoPathsAligned;
-  }
-
-  public void collectTargetPathInformation(String targetMeasurementId, int queryDataSetIndex) {
-    targetMeasurementIds.add(targetMeasurementId);
-    queryDataSetIndexes.add(queryDataSetIndex);
-  }
-
-  public void internallyConstructNewPlan() {
-    rowCount = 0;
-    times = new long[tabletRowLimit];
-    columns = new Object[targetMeasurementIds.size()];
-    bitMaps = new BitMap[targetMeasurementIds.size()];
-    for (int i = 0; i < bitMaps.length; ++i) {
-      bitMaps[i] = new BitMap(tabletRowLimit);
-      bitMaps[i].markAll();
-    }
-    dataTypes = new TSDataType[targetMeasurementIds.size()];
-
-    numberOfInitializedColumns = 0;
-  }
-
-  public void collectRowRecord(RowRecord rowRecord) {
-    if (numberOfInitializedColumns != columns.length) {
-      List<Integer> initializedDataTypeIndexes = trySetDataTypes(rowRecord);
-      tryInitColumns(initializedDataTypeIndexes);
-      numberOfInitializedColumns += initializedDataTypeIndexes.size();
-    }
-
-    times[rowCount] = rowRecord.getTimestamp();
-
-    for (int i = 0; i < columns.length; ++i) {
-      Field field = rowRecord.getFields().get(queryDataSetIndexes.get(i));
-
-      // if the field is NULL
-      if (field == null || field.getDataType() == null) {
-        // bit in bitMaps are marked as 1 (NULL) by default
-        continue;
-      }
-
-      bitMaps[i].unmark(rowCount);
-      switch (field.getDataType()) {
-        case INT32:
-          ((int[]) columns[i])[rowCount] = field.getIntV();
-          break;
-        case INT64:
-          ((long[]) columns[i])[rowCount] = field.getLongV();
-          break;
-        case FLOAT:
-          ((float[]) columns[i])[rowCount] = field.getFloatV();
-          break;
-        case DOUBLE:
-          ((double[]) columns[i])[rowCount] = field.getDoubleV();
-          break;
-        case BOOLEAN:
-          ((boolean[]) columns[i])[rowCount] = field.getBoolV();
-          break;
-        case TEXT:
-          ((Binary[]) columns[i])[rowCount] = field.getBinaryV();
-          break;
-        default:
-          throw new UnSupportedDataTypeException(
-              String.format(
-                  "data type %s is not supported when convert data at client",
-                  field.getDataType()));
-      }
-    }
-
-    ++rowCount;
-  }
-
-  private List<Integer> trySetDataTypes(RowRecord rowRecord) {
-    List<Integer> initializedDataTypeIndexes = new ArrayList<>();
-    List<Field> fields = rowRecord.getFields();
-
-    for (int i = 0; i < dataTypes.length; ++i) {
-      // if the data type is already set
-      if (dataTypes[i] != null) {
-        continue;
-      }
-
-      // get the field index of the row record
-      int queryDataSetIndex = queryDataSetIndexes.get(i);
-      // if the field is not null
-      if (fields.get(queryDataSetIndex) != null
-          && fields.get(queryDataSetIndex).getDataType() != null) {
-        // set the data type to the field type
-        dataTypes[i] = fields.get(queryDataSetIndex).getDataType();
-        initializedDataTypeIndexes.add(i);
-      }
-    }
-    return initializedDataTypeIndexes;
-  }
-
-  private void tryInitColumns(List<Integer> initializedDataTypeIndexes) {
-    for (int i : initializedDataTypeIndexes) {
-      switch (dataTypes[i]) {
-        case BOOLEAN:
-          columns[i] = new boolean[tabletRowLimit];
-          break;
-        case INT32:
-          columns[i] = new int[tabletRowLimit];
-          break;
-        case INT64:
-          columns[i] = new long[tabletRowLimit];
-          break;
-        case FLOAT:
-          columns[i] = new float[tabletRowLimit];
-          break;
-        case DOUBLE:
-          columns[i] = new double[tabletRowLimit];
-          break;
-        case TEXT:
-          columns[i] = new Binary[tabletRowLimit];
-          Arrays.fill((Binary[]) columns[i], Binary.EMPTY_VALUE);
-          break;
-        default:
-          throw new UnSupportedDataTypeException(
-              String.format("Data type %s is not supported.", dataTypes[i]));
-      }
-    }
-  }
-
-  public InsertTabletPlan generateInsertTabletPlan() throws IllegalPathException {
-    List<String> nonEmptyColumnNames = new ArrayList<>();
-
-    int countOfNonEmptyColumns = 0;
-    for (int i = 0; i < columns.length; ++i) {
-      if (columns[i] == null) {
-        continue;
-      }
-
-      nonEmptyColumnNames.add(targetMeasurementIds.get(i));
-      columns[countOfNonEmptyColumns] = columns[i];
-      bitMaps[countOfNonEmptyColumns] = bitMaps[i];
-      dataTypes[countOfNonEmptyColumns] = dataTypes[i];
-
-      ++countOfNonEmptyColumns;
-    }
-
-    InsertTabletPlan insertTabletPlan =
-        new InsertTabletPlan(new PartialPath(targetDevice), nonEmptyColumnNames);
-    insertTabletPlan.setAligned(isIntoPathsAligned);
-    insertTabletPlan.setRowCount(rowCount);
-
-    if (countOfNonEmptyColumns != columns.length) {
-      columns = Arrays.copyOf(columns, countOfNonEmptyColumns);
-      bitMaps = Arrays.copyOf(bitMaps, countOfNonEmptyColumns);
-      dataTypes = Arrays.copyOf(dataTypes, countOfNonEmptyColumns);
-    }
-
-    if (rowCount != tabletRowLimit) {
-      times = Arrays.copyOf(times, rowCount);
-      for (int i = 0; i < columns.length; ++i) {
-        switch (dataTypes[i]) {
-          case BOOLEAN:
-            columns[i] = Arrays.copyOf((boolean[]) columns[i], rowCount);
-            break;
-          case INT32:
-            columns[i] = Arrays.copyOf((int[]) columns[i], rowCount);
-            break;
-          case INT64:
-            columns[i] = Arrays.copyOf((long[]) columns[i], rowCount);
-            break;
-          case FLOAT:
-            columns[i] = Arrays.copyOf((float[]) columns[i], rowCount);
-            break;
-          case DOUBLE:
-            columns[i] = Arrays.copyOf((double[]) columns[i], rowCount);
-            break;
-          case TEXT:
-            columns[i] = Arrays.copyOf((Binary[]) columns[i], rowCount);
-            break;
-          default:
-            throw new UnSupportedDataTypeException(
-                String.format("Data type %s is not supported.", dataTypes[i]));
-        }
-      }
-    }
-
-    insertTabletPlan.setTimes(times);
-    insertTabletPlan.setColumns(columns);
-    insertTabletPlan.setBitMaps(bitMaps);
-    insertTabletPlan.setDataTypes(dataTypes);
-
-    return insertTabletPlan;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlansIterator.java b/server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlansIterator.java
deleted file mode 100644
index da291e6ef4..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/engine/selectinto/InsertTabletPlansIterator.java
+++ /dev/null
@@ -1,190 +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.selectinto;
-
-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.mpp.plan.expression.ResultColumn;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
-import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
-import org.apache.iotdb.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.read.common.RowRecord;
-import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-public class InsertTabletPlansIterator {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(InsertTabletPlansIterator.class);
-
-  private static final Pattern LEVELED_PATH_NODE_PATTERN = Pattern.compile("\\$\\{\\w+}");
-
-  private final QueryPlan queryPlan;
-  private final QueryDataSet queryDataSet;
-
-  private final PartialPath fromPath;
-  private final List<PartialPath> intoPaths;
-  private final boolean isIntoPathsAligned;
-
-  private final int tabletRowLimit;
-
-  private InsertTabletPlanGenerator[] insertTabletPlanGenerators;
-
-  public InsertTabletPlansIterator(
-      QueryPlan queryPlan,
-      QueryDataSet queryDataSet,
-      PartialPath fromPath,
-      List<PartialPath> intoPaths,
-      boolean isIntoPathsAligned)
-      throws IllegalPathException {
-    this.queryPlan = queryPlan;
-    this.queryDataSet = queryDataSet;
-    this.fromPath = fromPath;
-    this.intoPaths = intoPaths;
-    this.isIntoPathsAligned = isIntoPathsAligned;
-
-    tabletRowLimit =
-        IoTDBDescriptor.getInstance().getConfig().getSelectIntoInsertTabletPlanRowLimit();
-
-    generateActualIntoPaths();
-    constructInsertTabletPlanGenerators();
-  }
-
-  private void generateActualIntoPaths() throws IllegalPathException {
-    for (int i = 0; i < intoPaths.size(); ++i) {
-      intoPaths.set(i, generateActualIntoPath(i));
-    }
-  }
-
-  private PartialPath generateActualIntoPath(int index) throws IllegalPathException {
-    String[] nodes = fromPath.getNodes();
-    StringBuffer sb = new StringBuffer();
-    Matcher m = LEVELED_PATH_NODE_PATTERN.matcher(intoPaths.get(index).getFullPath());
-    while (m.find()) {
-      String param = m.group();
-      String value = nodes[Integer.parseInt(param.substring(2, param.length() - 1).trim())];
-      m.appendReplacement(sb, value == null ? "" : value);
-    }
-    m.appendTail(sb);
-    return new PartialPath(sb.toString());
-  }
-
-  private void constructInsertTabletPlanGenerators() {
-    final Map<String, Integer> sourcePathToQueryDataSetIndex = queryPlan.getPathToIndex();
-    final List<String> sourcePaths = findSourcePaths();
-
-    Map<String, InsertTabletPlanGenerator> deviceToPlanGeneratorMap = new HashMap<>();
-    for (int i = 0, intoPathsSize = intoPaths.size(); i < intoPathsSize; i++) {
-      String device = intoPaths.get(i).getDevice();
-      if (!deviceToPlanGeneratorMap.containsKey(device)) {
-        deviceToPlanGeneratorMap.put(
-            device, new InsertTabletPlanGenerator(device, tabletRowLimit, isIntoPathsAligned));
-      }
-      deviceToPlanGeneratorMap
-          .get(device)
-          .collectTargetPathInformation(
-              intoPaths.get(i).getMeasurement(),
-              sourcePathToQueryDataSetIndex.get(sourcePaths.get(i)));
-    }
-
-    insertTabletPlanGenerators =
-        deviceToPlanGeneratorMap.values().toArray(new InsertTabletPlanGenerator[0]);
-  }
-
-  private List<String> findSourcePaths() {
-    // sourcePaths can be in queryPlanColumns or in queryDataSetPaths
-    final List<ResultColumn> queryPlanColumns = queryPlan.getResultColumns();
-    final List<Path> queryDataSetPaths = queryDataSet.getPaths();
-
-    final Map<String, Integer> sourcePathToQueryDataSetIndex = queryPlan.getPathToIndex();
-    final List<String> sourcePaths =
-        new ArrayList<>(Math.max(queryPlanColumns.size(), queryDataSetPaths.size()));
-
-    if (queryPlanColumns.size() == intoPaths.size()) {
-      for (ResultColumn resultColumn : queryPlanColumns) {
-        String path = resultColumn.getResultColumnName();
-        if (!sourcePathToQueryDataSetIndex.containsKey(path)) {
-          sourcePaths.clear();
-          break;
-        }
-        sourcePaths.add(path);
-      }
-    }
-
-    if (sourcePaths.isEmpty() && queryDataSetPaths.size() == intoPaths.size()) {
-      for (Path path : queryDataSetPaths) {
-        if (!sourcePathToQueryDataSetIndex.containsKey(path.getFullPath())) {
-          sourcePaths.clear();
-          break;
-        }
-        sourcePaths.add(path.getFullPath());
-      }
-    }
-
-    if (sourcePaths.isEmpty()) {
-      LOGGER.warn("select into: sourcePaths.isEmpty()");
-    }
-
-    return sourcePaths;
-  }
-
-  public boolean hasNext() throws IOException {
-    return queryDataSet.hasNext();
-  }
-
-  public List<InsertTabletPlan> next() throws IOException, IllegalPathException {
-    for (InsertTabletPlanGenerator insertTabletPlanGenerator : insertTabletPlanGenerators) {
-      insertTabletPlanGenerator.internallyConstructNewPlan();
-    }
-
-    collectRowRecordIntoInsertTabletPlanGenerators();
-
-    List<InsertTabletPlan> insertTabletPlans = new ArrayList<>();
-    for (InsertTabletPlanGenerator insertTabletPlanGenerator : insertTabletPlanGenerators) {
-      // all values can be null in a batch of the query dataset
-      InsertTabletPlan insertTabletPlan = insertTabletPlanGenerator.generateInsertTabletPlan();
-      if (insertTabletPlan.getColumns().length != 0) {
-        insertTabletPlans.add(insertTabletPlan);
-      }
-    }
-    return insertTabletPlans;
-  }
-
-  private void collectRowRecordIntoInsertTabletPlanGenerators() throws IOException {
-    int count = 0;
-    while (queryDataSet.hasNext() && count < tabletRowLimit) {
-      RowRecord rowRecord = queryDataSet.next();
-      for (InsertTabletPlanGenerator insertTabletPlanGenerator : insertTabletPlanGenerators) {
-        insertTabletPlanGenerator.collectRowRecord(rowRecord);
-      }
-      ++count;
-    }
-  }
-}
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 aef9ff8ac5..5e9a99658f 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
@@ -78,7 +78,6 @@ import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.FileReaderManager;
 import org.apache.iotdb.db.query.control.QueryFileManager;
 import org.apache.iotdb.db.rescon.TsFileResourceManager;
-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.sync.sender.manager.ISyncManager;
@@ -2016,19 +2015,6 @@ public class DataRegion {
     }
   }
 
-  private void tryToDeleteLastCache(
-      PartialPath deviceId, PartialPath originalPath, long startTime, long endTime)
-      throws WriteProcessException {
-    if (!IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled()) {
-      return;
-    }
-    try {
-      IoTDB.schemaProcessor.deleteLastCacheByDevice(deviceId, originalPath, startTime, endTime);
-    } catch (MetadataException e) {
-      throw new WriteProcessException(e);
-    }
-  }
-
   /**
    * when close an TsFileProcessor, update its EndTimeMap immediately
    *
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/trigger/executor/TriggerEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/trigger/executor/TriggerEngine.java
index 07605f6d07..96ccf5598a 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/trigger/executor/TriggerEngine.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/trigger/executor/TriggerEngine.java
@@ -21,83 +21,19 @@ package org.apache.iotdb.db.engine.trigger.executor;
 
 import org.apache.iotdb.db.engine.trigger.service.TriggerRegistrationInformation;
 import org.apache.iotdb.db.engine.trigger.service.TriggerRegistrationService;
-import org.apache.iotdb.db.exception.TriggerExecutionException;
 import org.apache.iotdb.db.exception.TriggerManagementException;
 import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.qp.physical.sys.DropTriggerPlan;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Arrays;
 import java.util.List;
 
 public class TriggerEngine {
 
   private static final Logger LOGGER = LoggerFactory.getLogger(TriggerEngine.class);
 
-  public static void fire(TriggerEvent event, InsertRowPlan insertRowPlan)
-      throws TriggerExecutionException {
-    if (TriggerRegistrationService.getInstance().executorSize() == 0) {
-      return;
-    }
-
-    IMeasurementMNode[] mNodes = insertRowPlan.getMeasurementMNodes();
-    int size = mNodes.length;
-
-    long timestamp = insertRowPlan.getTime();
-    Object[] values = insertRowPlan.getValues();
-
-    for (int i = 0; i < size; ++i) {
-      IMeasurementMNode mNode = mNodes[i];
-      if (mNode == null) {
-        continue;
-      }
-      for (TriggerExecutor executor : mNode.getUpperTriggerExecutorList()) {
-        executor.fireIfActivated(
-            event,
-            timestamp,
-            values[i],
-            mNode.getSchema().getType(),
-            insertRowPlan.getPaths().get(i));
-      }
-    }
-  }
-
-  public static void fire(TriggerEvent event, InsertTabletPlan insertTabletPlan, int firePosition)
-      throws TriggerExecutionException {
-    if (TriggerRegistrationService.getInstance().executorSize() == 0) {
-      return;
-    }
-
-    IMeasurementMNode[] mNodes = insertTabletPlan.getMeasurementMNodes();
-    int size = mNodes.length;
-
-    long[] timestamps = insertTabletPlan.getTimes();
-    Object[] columns = insertTabletPlan.getColumns();
-    if (firePosition != 0) {
-      timestamps = Arrays.copyOfRange(timestamps, firePosition, timestamps.length);
-      columns = Arrays.copyOfRange(columns, firePosition, columns.length);
-    }
-
-    for (int i = 0; i < size; ++i) {
-      IMeasurementMNode mNode = mNodes[i];
-      if (mNode == null) {
-        continue;
-      }
-      for (TriggerExecutor executor : mNode.getUpperTriggerExecutorList()) {
-        executor.fireIfActivated(
-            event,
-            timestamps,
-            columns[i],
-            mNode.getSchema().getType(),
-            insertTabletPlan.getPaths().get(i));
-      }
-    }
-  }
-
   public static void drop(IMeasurementMNode measurementMNode) {
     TriggerExecutor executor = measurementMNode.getTriggerExecutor();
     if (executor == null) {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/trigger/sink/local/LocalIoTDBHandler.java b/server/src/main/java/org/apache/iotdb/db/engine/trigger/sink/local/LocalIoTDBHandler.java
index a5567b5a7c..7d73860c40 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/trigger/sink/local/LocalIoTDBHandler.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/trigger/sink/local/LocalIoTDBHandler.java
@@ -30,7 +30,6 @@ import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.executor.IPlanExecutor;
 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.service.IoTDB;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -82,16 +81,7 @@ public class LocalIoTDBHandler implements Handler<LocalIoTDBConfiguration, Local
 
   @Override
   public void onEvent(LocalIoTDBEvent event)
-      throws QueryProcessException, StorageEngineException, StorageGroupNotSetException {
-    InsertRowPlan plan = new InsertRowPlan();
-    plan.setNeedInferType(false);
-    plan.setDevicePath(device);
-    plan.setMeasurements(measurements);
-    plan.setDataTypes(dataTypes);
-    plan.setTime(event.getTimestamp());
-    plan.setValues(event.getValues());
-    executeNonQuery(plan);
-  }
+      throws QueryProcessException, StorageEngineException, StorageGroupNotSetException {}
 
   private void executeNonQuery(PhysicalPlan plan)
       throws QueryProcessException, StorageGroupNotSetException, StorageEngineException {
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/LocalSchemaProcessor.java b/server/src/main/java/org/apache/iotdb/db/metadata/LocalSchemaProcessor.java
index 273c4a179a..100511c9e9 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/LocalSchemaProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/LocalSchemaProcessor.java
@@ -44,7 +44,6 @@ import org.apache.iotdb.db.metadata.template.Template;
 import org.apache.iotdb.db.metadata.template.TemplateManager;
 import org.apache.iotdb.db.qp.constant.SQLConstant;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.AppendTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.AutoCreateDeviceMNodePlan;
@@ -119,7 +118,6 @@ import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARA
  *   <li>Interfaces for alias and tag/attribute operations
  *   <li>Interfaces only for Cluster module usage
  *   <li>Interfaces for lastCache operations
- *   <li>Interfaces and Implementation for InsertPlan process
  *   <li>Interfaces and Implementation for Template operations
  *   <li>Interfaces for Trigger
  *   <li>TestOnly Interfaces
@@ -1224,20 +1222,6 @@ public class LocalSchemaProcessor {
     LastCacheManager.resetLastCache(node);
   }
 
-  /**
-   * delete all the last cache value of any timeseries or aligned timeseries under the device
-   *
-   * <p>Invoking scenario (1) after upload tsfile
-   *
-   * @param deviceId path of device
-   */
-  public void deleteLastCacheByDevice(PartialPath deviceId) throws MetadataException {
-    IMNode node = getDeviceNode(deviceId);
-    if (node.isEntity()) {
-      LastCacheManager.deleteLastCacheByDevice(node.getAsEntityMNode());
-    }
-  }
-
   /**
    * delete the last cache value of timeseries or subMeasurement of some aligned timeseries, which
    * is under the device and matching the originalPath
@@ -1260,23 +1244,6 @@ public class LocalSchemaProcessor {
   }
   // endregion
 
-  // region Interfaces and Implementation for InsertPlan process
-  /** get schema for device. Attention!!! Only support insertPlan */
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
-      throws MetadataException, IOException {
-    ISchemaRegion schemaRegion;
-    if (config.isAutoCreateSchemaEnabled()) {
-      schemaRegion = getBelongedSchemaRegionWithAutoCreate(plan.getDevicePath());
-    } else {
-      schemaRegion = getBelongedSchemaRegion(plan.getDevicePath());
-    }
-
-    return schemaRegion.getSeriesSchemasAndReadLockDevice(plan);
-  }
-
-  // endregion
-
   // region Interfaces and Implementation for Template operations
   public void createSchemaTemplate(CreateTemplatePlan plan) throws MetadataException {
     configManager.createSchemaTemplate(plan);
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTable.java b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTable.java
index 8723e26375..a067cf98b8 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTable.java
@@ -35,7 +35,6 @@ import org.apache.iotdb.db.metadata.idtable.entry.TimeseriesID;
 import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateAlignedTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateTimeSeriesPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
@@ -87,7 +86,7 @@ public interface IDTable {
    * @return reusable device id
    * @throws MetadataException if insert plan's aligned value is inconsistent with device
    */
-  IDeviceID getSeriesSchemas(InsertPlan plan) throws MetadataException;
+  // IDeviceID getSeriesSchemas(InsertPlan plan) throws MetadataException;
 
   /**
    * register trigger to the timeseries
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java
index 5880cb7c28..31a39a45a2 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java
@@ -24,20 +24,15 @@ import org.apache.iotdb.commons.path.PartialPath;
 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.exception.metadata.DataTypeMismatchException;
 import org.apache.iotdb.db.metadata.idtable.entry.DeviceEntry;
 import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
 import org.apache.iotdb.db.metadata.idtable.entry.DiskSchemaEntry;
 import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
-import org.apache.iotdb.db.metadata.idtable.entry.InsertMeasurementMNode;
 import org.apache.iotdb.db.metadata.idtable.entry.SchemaEntry;
 import org.apache.iotdb.db.metadata.idtable.entry.TimeseriesID;
 import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateAlignedTimeSeriesPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.ICreateTimeSeriesPlan;
-import org.apache.iotdb.db.metadata.schemaregion.SchemaRegionUtils;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
@@ -176,73 +171,6 @@ public class IDTableHashmapImpl implements IDTable {
     return new Pair<>(deletedNum, failedNames);
   }
 
-  /**
-   * check inserting timeseries existence and fill their measurement mnode
-   *
-   * @param plan insert plan
-   * @return reusable device id
-   * @throws MetadataException if insert plan's aligned value is inconsistent with device
-   */
-  @Override
-  public synchronized IDeviceID getSeriesSchemas(InsertPlan plan) throws MetadataException {
-    PartialPath devicePath = plan.getDevicePath();
-    String[] measurementList = plan.getMeasurements();
-    IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
-
-    // 1. get device entry and check align
-    DeviceEntry deviceEntry =
-        getDeviceEntryWithAlignedCheck(devicePath.toString(), plan.isAligned());
-
-    // 2. get schema of each measurement
-    for (int i = 0; i < measurementList.length; i++) {
-      try {
-        // get MeasurementMNode, auto create if absent
-        try {
-          IMeasurementMNode measurementMNode =
-              getOrCreateMeasurementIfNotExist(deviceEntry, plan, i);
-
-          SchemaRegionUtils.checkDataTypeMatch(plan, i, measurementMNode.getSchema().getType());
-          measurementMNodes[i] = measurementMNode;
-        } catch (DataTypeMismatchException mismatchException) {
-          logger.warn(mismatchException.getMessage());
-          if (!config.isEnablePartialInsert()) {
-            throw mismatchException;
-          } else {
-            // mark failed measurement
-            plan.markFailedMeasurementInsertion(i, mismatchException);
-          }
-        }
-      } catch (MetadataException e) {
-        if (config.isClusterMode()) {
-          logger.debug(
-              "meet error when check {}.{}, message: {}",
-              devicePath,
-              measurementList[i],
-              e.getMessage());
-        } else {
-          logger.warn(
-              "meet error when check {}.{}, message: {}",
-              devicePath,
-              measurementList[i],
-              e.getMessage());
-        }
-        if (config.isEnablePartialInsert()) {
-          // mark failed measurement
-          plan.markFailedMeasurementInsertion(i, e);
-        } else {
-          throw e;
-        }
-      }
-    }
-
-    // set reusable device id
-    plan.setDeviceID(deviceEntry.getDeviceID());
-    // change device path to device id string for insertion
-    plan.setDevicePath(new PartialPath(deviceEntry.getDeviceID().toStringID(), false));
-
-    return deviceEntry.getDeviceID();
-  }
-
   /**
    * register trigger to the timeseries
    *
@@ -389,68 +317,6 @@ public class IDTableHashmapImpl implements IDTable {
     return getIDiskSchemaManager().getDiskSchemaEntriesByOffset(offsets);
   }
 
-  /**
-   * check whether a time series is exist if exist, check the type consistency if not exist, call
-   * SchemaProcessor to create it
-   *
-   * @return measurement MNode of the time series or null if type is not match
-   */
-  private IMeasurementMNode getOrCreateMeasurementIfNotExist(
-      DeviceEntry deviceEntry, InsertPlan plan, int loc) throws MetadataException {
-    String measurementName = plan.getMeasurements()[loc];
-    PartialPath seriesKey = new PartialPath(plan.getDevicePath().toString(), measurementName);
-
-    SchemaEntry schemaEntry = deviceEntry.getSchemaEntry(measurementName);
-
-    // if not exist, we create it
-    if (schemaEntry == null) {
-      // we have to copy plan's mnode for using id table's last cache
-      IMeasurementMNode[] insertPlanMNodeBackup =
-          new IMeasurementMNode[plan.getMeasurementMNodes().length];
-      System.arraycopy(
-          plan.getMeasurementMNodes(), 0, insertPlanMNodeBackup, 0, insertPlanMNodeBackup.length);
-      try {
-        IoTDB.schemaProcessor.getSeriesSchemasAndReadLockDevice(plan);
-      } catch (IOException e) {
-        throw new MetadataException(e);
-      }
-
-      // if the timeseries is in template, SchemaProcessor will not create timeseries. so we have to
-      // put it
-      // in id table here
-      for (IMeasurementMNode measurementMNode : plan.getMeasurementMNodes()) {
-        if (measurementMNode != null && !deviceEntry.contains(measurementMNode.getName())) {
-          IMeasurementSchema schema = measurementMNode.getSchema();
-          SchemaEntry curEntry =
-              new SchemaEntry(
-                  schema.getType(),
-                  schema.getEncodingType(),
-                  schema.getCompressor(),
-                  deviceEntry.getDeviceID(),
-                  seriesKey,
-                  deviceEntry.isAligned(),
-                  IDiskSchemaManager);
-          deviceEntry.putSchemaEntry(measurementMNode.getName(), curEntry);
-        }
-      }
-
-      // copy back measurement mnode list
-      System.arraycopy(
-          insertPlanMNodeBackup, 0, plan.getMeasurementMNodes(), 0, insertPlanMNodeBackup.length);
-
-      schemaEntry = deviceEntry.getSchemaEntry(measurementName);
-    }
-
-    // timeseries is using trigger, we should get trigger from SchemaProcessor
-    if (schemaEntry.isUsingTrigger()) {
-      IMeasurementMNode measurementMNode = IoTDB.schemaProcessor.getMeasurementMNode(seriesKey);
-      return new InsertMeasurementMNode(
-          measurementName, schemaEntry, measurementMNode.getTriggerExecutor());
-    }
-
-    return new InsertMeasurementMNode(measurementName, schemaEntry);
-  }
-
   /**
    * get device id from device path and check is aligned,
    *
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java
index e0319ee600..6979bbffa8 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java
@@ -41,7 +41,6 @@ import org.apache.iotdb.db.metadata.plan.schemaregion.write.ISetTemplatePlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.write.IUnsetTemplatePlan;
 import org.apache.iotdb.db.metadata.template.Template;
 import org.apache.iotdb.db.mpp.common.schematree.DeviceSchemaInfo;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
@@ -396,8 +395,6 @@ public interface ISchemaRegion {
   // endregion
 
   // region Interfaces for InsertPlan process
-  /** get schema for device. Attention!!! Only support insertPlan */
-  IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan) throws MetadataException, IOException;
 
   DeviceSchemaInfo getDeviceSchemaInfoWithAutoCreate(
       PartialPath devicePath,
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 040250dc4c..5b3458f1e0 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
@@ -30,8 +30,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.engine.trigger.executor.TriggerEngine;
-import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
 import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
 import org.apache.iotdb.db.exception.metadata.MeasurementAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
@@ -54,7 +52,6 @@ import org.apache.iotdb.db.metadata.mnode.IEntityMNode;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
 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.metadata.mtree.MTreeBelowSGMemoryImpl;
 import org.apache.iotdb.db.metadata.plan.schemaregion.ISchemaRegionPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.SchemaRegionPlanVisitor;
@@ -83,9 +80,6 @@ import org.apache.iotdb.db.metadata.template.Template;
 import org.apache.iotdb.db.metadata.template.TemplateManager;
 import org.apache.iotdb.db.mpp.common.schematree.DeviceSchemaInfo;
 import org.apache.iotdb.db.mpp.common.schematree.MeasurementSchemaInfo;
-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.qp.physical.sys.ShowDevicesPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
@@ -111,7 +105,6 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -1622,199 +1615,6 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
   // endregion
 
   // region Interfaces and Implementation for InsertPlan process
-  /** get schema for device. Attention!!! Only support insertPlan */
-  @Override
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
-      throws MetadataException, IOException {
-    // devicePath is a logical path which is parent of measurement, whether in template or not
-    PartialPath devicePath = plan.getDevicePath();
-    String[] measurementList = plan.getMeasurements();
-    IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
-    IMNode deviceMNode;
-
-    // 1. get device node, set using template if accessed.
-    deviceMNode = getDeviceInTemplateIfUsingTemplate(devicePath, measurementList);
-    boolean isDeviceInTemplate = deviceMNode != null;
-
-    // get logical device node, may be in template. will be multiple if overlap is allowed.
-    if (!isDeviceInTemplate) {
-      deviceMNode = getDeviceNodeWithAutoCreate(devicePath);
-    }
-
-    // check insert non-aligned InsertPlan for aligned timeseries
-    if (deviceMNode.isEntity()) {
-      if (plan.isAligned()) {
-        if (!deviceMNode.getAsEntityMNode().isAligned()) {
-          throw new AlignedTimeseriesException(
-              "timeseries under this device are not aligned, " + "please use non-aligned interface",
-              devicePath.getFullPath());
-        }
-      } else {
-        if (deviceMNode.getAsEntityMNode().isAligned()) {
-          throw new AlignedTimeseriesException(
-              "timeseries under this device are aligned, " + "please use aligned interface",
-              devicePath.getFullPath());
-        }
-      }
-    }
-
-    // 2. get schema of each measurement
-    IMeasurementMNode measurementMNode;
-    for (int i = 0; i < measurementList.length; i++) {
-      try {
-        // get MeasurementMNode, auto create if absent
-        Pair<IMNode, IMeasurementMNode> pair =
-            getMeasurementMNodeForInsertPlan(plan, i, deviceMNode, isDeviceInTemplate);
-        deviceMNode = pair.left;
-        measurementMNode = pair.right;
-
-        // check type is match
-        if (plan instanceof InsertRowPlan || plan instanceof InsertTabletPlan) {
-          try {
-            SchemaRegionUtils.checkDataTypeMatch(plan, i, measurementMNode.getSchema().getType());
-          } catch (DataTypeMismatchException mismatchException) {
-            logger.warn(mismatchException.getMessage());
-            if (!config.isEnablePartialInsert()) {
-              throw mismatchException;
-            } else {
-              // mark failed measurement
-              plan.markFailedMeasurementInsertion(i, mismatchException);
-              continue;
-            }
-          }
-          measurementMNodes[i] = measurementMNode;
-          // set measurementName instead of alias
-          measurementList[i] = measurementMNode.getName();
-        }
-      } catch (MetadataException e) {
-        if (config.isClusterMode()) {
-          logger.debug(
-              "meet error when check {}.{}, message: {}",
-              devicePath,
-              measurementList[i],
-              e.getMessage());
-        } else {
-          logger.warn(
-              "meet error when check {}.{}, message: {}",
-              devicePath,
-              measurementList[i],
-              e.getMessage());
-        }
-        if (config.isEnablePartialInsert()) {
-          // mark failed measurement
-          plan.markFailedMeasurementInsertion(i, e);
-        } else {
-          throw e;
-        }
-      }
-    }
-
-    return deviceMNode;
-  }
-
-  private IMNode getDeviceInTemplateIfUsingTemplate(
-      PartialPath devicePath, String[] measurementList) throws MetadataException, IOException {
-    // 1. get device node, set using template if accessed.
-    IMNode deviceMNode = null;
-
-    // check every measurement path
-    int index = mtree.getMountedNodeIndexOnMeasurementPath(devicePath, measurementList);
-    if (index == devicePath.getNodeLength()) {
-      return null;
-    }
-
-    // this measurement is in template, need to assure mounted node exists and set using
-    // template.
-    // Without allowing overlap of template and MTree, this block run only once
-    String[] mountedPathNodes = Arrays.copyOfRange(devicePath.getNodes(), 0, index + 1);
-    IMNode mountedNode = getDeviceNodeWithAutoCreate(new PartialPath(mountedPathNodes));
-    if (!mountedNode.isUseTemplate()) {
-      mountedNode = setUsingSchemaTemplate(mountedNode);
-    }
-    if (index < devicePath.getNodeLength() - 1) {
-      deviceMNode =
-          mountedNode
-              .getUpperTemplate()
-              .getPathNodeInTemplate(
-                  new PartialPath(
-                      Arrays.copyOfRange(
-                          devicePath.getNodes(), index + 1, devicePath.getNodeLength())));
-    }
-
-    return deviceMNode;
-  }
-
-  private Pair<IMNode, IMeasurementMNode> getMeasurementMNodeForInsertPlan(
-      InsertPlan plan, int loc, IMNode deviceMNode, boolean isDeviceInTemplate)
-      throws MetadataException {
-    PartialPath devicePath = plan.getDevicePath();
-    String[] measurementList = plan.getMeasurements();
-    String measurement = measurementList[loc];
-    IMeasurementMNode measurementMNode = null;
-    if (isDeviceInTemplate) {
-      measurementMNode = deviceMNode.getChild(measurement).getAsMeasurementMNode();
-    } else {
-      measurementMNode = getMeasurementMNode(deviceMNode, measurement);
-      if (measurementMNode == null) {
-        measurementMNode = findMeasurementInTemplate(deviceMNode, measurement);
-      }
-    }
-    if (measurementMNode == null) {
-      if (!config.isAutoCreateSchemaEnabled() || isDeviceInTemplate) {
-        throw new PathNotExistException(devicePath + PATH_SEPARATOR + measurement);
-      } else {
-        if (plan instanceof InsertRowPlan || plan instanceof InsertTabletPlan) {
-          if (!plan.isAligned()) {
-            internalCreateTimeseries(devicePath.concatNode(measurement), plan.getDataTypes()[loc]);
-          } else {
-            internalAlignedCreateTimeseries(devicePath, measurement, plan.getDataTypes()[loc]);
-          }
-          // after creating timeseries, the deviceMNode has been replaced by a new entityMNode
-          deviceMNode = mtree.getNodeByPath(devicePath);
-          measurementMNode = getMeasurementMNode(deviceMNode, measurement);
-        } else {
-          throw new MetadataException(
-              String.format(
-                  "Only support insertRow and insertTablet, plan is [%s]", plan.getOperatorType()));
-        }
-      }
-    }
-    return new Pair<>(deviceMNode, measurementMNode);
-  }
-
-  /** get dataType of plan, in loc measurements only support InsertRowPlan and InsertTabletPlan */
-  private IMeasurementMNode findMeasurementInTemplate(IMNode deviceMNode, String measurement)
-      throws MetadataException {
-    Template curTemplate = deviceMNode.getUpperTemplate();
-
-    if (curTemplate == null) {
-      return null;
-    }
-
-    IMeasurementSchema schema = curTemplate.getSchema(measurement);
-
-    if (schema == null) {
-      return null;
-    }
-
-    if (!deviceMNode.isUseTemplate()) {
-      deviceMNode = setUsingSchemaTemplate(deviceMNode);
-    }
-
-    return MeasurementMNode.getMeasurementMNode(
-        deviceMNode.getAsEntityMNode(), measurement, schema, null);
-  }
-
-  /** create timeseries ignoring PathAlreadyExistException */
-  private void internalCreateTimeseries(PartialPath path, TSDataType dataType)
-      throws MetadataException {
-    internalCreateTimeseries(
-        path,
-        dataType,
-        getDefaultEncoding(dataType),
-        TSFileDescriptor.getInstance().getConfig().getCompressor());
-  }
 
   /** create timeseries ignoring MeasurementAlreadyExistException */
   private void internalCreateTimeseries(
@@ -1863,17 +1663,6 @@ public class SchemaRegionMemoryImpl implements ISchemaRegion {
     }
   }
 
-  /** create aligned timeseries ignoring MeasurementAlreadyExistException */
-  private void internalAlignedCreateTimeseries(
-      PartialPath devicePath, String measurement, TSDataType dataType) throws MetadataException {
-    internalAlignedCreateTimeseries(
-        devicePath,
-        measurement,
-        dataType,
-        getDefaultEncoding(dataType),
-        TSFileDescriptor.getInstance().getConfig().getCompressor());
-  }
-
   @Override
   public DeviceSchemaInfo getDeviceSchemaInfoWithAutoCreate(
       PartialPath devicePath,
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 972a1e3e82..3c11546e60 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
@@ -29,8 +29,6 @@ import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 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;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
 import org.apache.iotdb.db.exception.metadata.DeleteFailedException;
 import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
@@ -51,7 +49,6 @@ import org.apache.iotdb.db.metadata.logfile.SchemaLogWriter;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
 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.metadata.mtree.MTreeBelowSGCachedImpl;
 import org.apache.iotdb.db.metadata.plan.schemaregion.ISchemaRegionPlan;
 import org.apache.iotdb.db.metadata.plan.schemaregion.SchemaRegionPlanVisitor;
@@ -77,9 +74,6 @@ import org.apache.iotdb.db.metadata.tag.TagManager;
 import org.apache.iotdb.db.metadata.template.Template;
 import org.apache.iotdb.db.metadata.template.TemplateManager;
 import org.apache.iotdb.db.mpp.common.schematree.DeviceSchemaInfo;
-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.qp.physical.sys.ShowDevicesPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
@@ -87,7 +81,6 @@ import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
 import org.apache.iotdb.db.query.dataset.ShowTimeSeriesResult;
 import org.apache.iotdb.db.utils.SchemaUtils;
 import org.apache.iotdb.external.api.ISeriesNumerMonitor;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 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;
@@ -105,7 +98,6 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
@@ -117,7 +109,6 @@ import java.util.function.Function;
 import java.util.stream.Stream;
 
 import static java.util.stream.Collectors.toList;
-import static org.apache.iotdb.db.utils.EncodingInferenceUtils.getDefaultEncoding;
 import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR;
 
 /**
@@ -1644,101 +1635,6 @@ public class SchemaRegionSchemaFileImpl implements ISchemaRegion {
   // endregion
 
   // region Interfaces and Implementation for InsertPlan process
-  /** get schema for device. Attention!!! Only support insertPlan */
-  @Override
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  public IMNode getSeriesSchemasAndReadLockDevice(InsertPlan plan)
-      throws MetadataException, IOException {
-    // devicePath is a logical path which is parent of measurement, whether in template or not
-    PartialPath devicePath = plan.getDevicePath();
-    String[] measurementList = plan.getMeasurements();
-    IMeasurementMNode[] measurementMNodes = plan.getMeasurementMNodes();
-    IMNode deviceMNode;
-
-    // 1. get device node, set using template if accessed.
-    deviceMNode = getDeviceInTemplateIfUsingTemplate(devicePath, measurementList);
-    boolean isDeviceInTemplate = deviceMNode != null;
-    // get logical device node, may be in template. will be multiple if overlap is allowed.
-    if (!isDeviceInTemplate) {
-      deviceMNode = getDeviceNodeWithAutoCreate(devicePath);
-    }
-    try {
-      // check insert non-aligned InsertPlan for aligned timeseries
-      if (deviceMNode.isEntity()) {
-        if (plan.isAligned()) {
-          if (!deviceMNode.getAsEntityMNode().isAligned()) {
-            throw new AlignedTimeseriesException(
-                "timeseries under this device are not aligned, "
-                    + "please use non-aligned interface",
-                devicePath.getFullPath());
-          }
-        } else {
-          if (deviceMNode.getAsEntityMNode().isAligned()) {
-            throw new AlignedTimeseriesException(
-                "timeseries under this device are aligned, " + "please use aligned interface",
-                devicePath.getFullPath());
-          }
-        }
-      }
-
-      // 2. get schema of each measurement
-      IMeasurementMNode measurementMNode;
-      for (int i = 0; i < measurementList.length; i++) {
-        try {
-          // get MeasurementMNode, auto create if absent
-          Pair<IMNode, IMeasurementMNode> pair =
-              getMeasurementMNodeForInsertPlan(plan, i, deviceMNode, isDeviceInTemplate);
-          deviceMNode = pair.left;
-          measurementMNode = pair.right;
-
-          // check type is match
-          if (plan instanceof InsertRowPlan || plan instanceof InsertTabletPlan) {
-            try {
-              SchemaRegionUtils.checkDataTypeMatch(plan, i, measurementMNode.getSchema().getType());
-            } catch (DataTypeMismatchException mismatchException) {
-              logger.warn(mismatchException.getMessage());
-              if (!config.isEnablePartialInsert()) {
-                throw mismatchException;
-              } else {
-                // mark failed measurement
-                plan.markFailedMeasurementInsertion(i, mismatchException);
-                continue;
-              }
-            }
-            measurementMNodes[i] = measurementMNode;
-            // set measurementName instead of alias
-            measurementList[i] = measurementMNode.getName();
-          }
-        } catch (MetadataException e) {
-          if (config.isClusterMode()) {
-            logger.debug(
-                "meet error when check {}.{}, message: {}",
-                devicePath,
-                measurementList[i],
-                e.getMessage());
-          } else {
-            logger.warn(
-                "meet error when check {}.{}, message: {}",
-                devicePath,
-                measurementList[i],
-                e.getMessage());
-          }
-          if (config.isEnablePartialInsert()) {
-            // mark failed measurement
-            plan.markFailedMeasurementInsertion(i, e);
-          } else {
-            throw e;
-          }
-        }
-      }
-    } finally {
-      if (!isDeviceInTemplate) {
-        mtree.unPinMNode(deviceMNode);
-      }
-    }
-
-    return deviceMNode;
-  }
 
   @Override
   public DeviceSchemaInfo getDeviceSchemaInfoWithAutoCreate(
@@ -1752,132 +1648,6 @@ public class SchemaRegionSchemaFileImpl implements ISchemaRegion {
     throw new UnsupportedOperationException();
   }
 
-  private IMNode getDeviceInTemplateIfUsingTemplate(
-      PartialPath devicePath, String[] measurementList) throws MetadataException, IOException {
-    // 1. get device node, set using template if accessed.
-    IMNode deviceMNode = null;
-
-    // check every measurement path
-    int index = mtree.getMountedNodeIndexOnMeasurementPath(devicePath, measurementList);
-    if (index == devicePath.getNodeLength()) {
-      return null;
-    }
-
-    // this measurement is in template, need to assure mounted node exists and set using
-    // template.
-    // Without allowing overlap of template and MTree, this block run only once
-    String[] mountedPathNodes = Arrays.copyOfRange(devicePath.getNodes(), 0, index + 1);
-    IMNode mountedNode = getDeviceNodeWithAutoCreate(new PartialPath(mountedPathNodes));
-    try {
-      if (!mountedNode.isUseTemplate()) {
-        mountedNode = setUsingSchemaTemplate(mountedNode);
-      }
-    } finally {
-      mtree.unPinMNode(mountedNode);
-    }
-
-    if (index < devicePath.getNodeLength() - 1) {
-      deviceMNode =
-          mountedNode
-              .getUpperTemplate()
-              .getPathNodeInTemplate(
-                  new PartialPath(
-                      Arrays.copyOfRange(
-                          devicePath.getNodes(), index + 1, devicePath.getNodeLength())));
-    }
-
-    return deviceMNode;
-  }
-
-  private Pair<IMNode, IMeasurementMNode> getMeasurementMNodeForInsertPlan(
-      InsertPlan plan, int loc, IMNode deviceMNode, boolean isDeviceInTemplate)
-      throws MetadataException {
-    PartialPath devicePath = plan.getDevicePath();
-    String[] measurementList = plan.getMeasurements();
-    String measurement = measurementList[loc];
-    IMeasurementMNode measurementMNode = null;
-    if (isDeviceInTemplate) {
-      measurementMNode = deviceMNode.getChild(measurement).getAsMeasurementMNode();
-    } else {
-      measurementMNode = getMeasurementMNode(deviceMNode, measurement);
-      if (measurementMNode == null) {
-        measurementMNode = findMeasurementInTemplate(deviceMNode, measurement);
-      }
-    }
-    if (measurementMNode == null) {
-      if (!config.isAutoCreateSchemaEnabled() || isDeviceInTemplate) {
-        throw new PathNotExistException(devicePath + PATH_SEPARATOR + measurement);
-      } else {
-        if (plan instanceof InsertRowPlan || plan instanceof InsertTabletPlan) {
-          if (!plan.isAligned()) {
-            internalCreateTimeseries(devicePath.concatNode(measurement), plan.getDataTypes()[loc]);
-          } else {
-            internalAlignedCreateTimeseries(
-                devicePath,
-                Collections.singletonList(measurement),
-                Collections.singletonList(plan.getDataTypes()[loc]));
-          }
-          // after creating timeseries, the deviceMNode has been replaced by a new entityMNode
-          deviceMNode = mtree.getNodeByPath(devicePath);
-          mtree.unPinMNode(deviceMNode);
-          measurementMNode = getMeasurementMNode(deviceMNode, measurement);
-        } else {
-          throw new MetadataException(
-              String.format(
-                  "Only support insertRow and insertTablet, plan is [%s]", plan.getOperatorType()));
-        }
-      }
-    }
-    return new Pair<>(deviceMNode, measurementMNode);
-  }
-
-  /** get dataType of plan, in loc measurements only support InsertRowPlan and InsertTabletPlan */
-  private IMeasurementMNode findMeasurementInTemplate(IMNode deviceMNode, String measurement)
-      throws MetadataException {
-    Template curTemplate = deviceMNode.getUpperTemplate();
-
-    if (curTemplate == null) {
-      return null;
-    }
-
-    IMeasurementSchema schema = curTemplate.getSchema(measurement);
-
-    if (schema == null) {
-      return null;
-    }
-
-    if (!deviceMNode.isUseTemplate()) {
-      deviceMNode = setUsingSchemaTemplate(deviceMNode);
-    }
-
-    return MeasurementMNode.getMeasurementMNode(
-        deviceMNode.getAsEntityMNode(), measurement, schema, null);
-  }
-
-  /** create timeseries ignoring PathAlreadyExistException */
-  private void internalCreateTimeseries(PartialPath path, TSDataType dataType)
-      throws MetadataException {
-    createTimeseries(
-        path,
-        dataType,
-        getDefaultEncoding(dataType),
-        TSFileDescriptor.getInstance().getConfig().getCompressor(),
-        Collections.emptyMap());
-  }
-
-  /** create aligned timeseries ignoring PathAlreadyExistException */
-  private void internalAlignedCreateTimeseries(
-      PartialPath prefixPath, List<String> measurements, List<TSDataType> dataTypes)
-      throws MetadataException {
-    List<TSEncoding> encodings = new ArrayList<>();
-    List<CompressionType> compressors = new ArrayList<>();
-    for (TSDataType dataType : dataTypes) {
-      encodings.add(getDefaultEncoding(dataType));
-      compressors.add(TSFileDescriptor.getInstance().getConfig().getCompressor());
-    }
-    createAlignedTimeSeries(prefixPath, measurements, dataTypes, encodings, compressors);
-  }
-
   // endregion
 
   // region Interfaces and Implementation for Template operations
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionUtils.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionUtils.java
index c95ea5b7f8..2a8d972b5f 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionUtils.java
@@ -21,12 +21,6 @@ package org.apache.iotdb.db.metadata.schemaregion;
 
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.file.SystemFileFactory;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
-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.utils.TypeInferenceUtils;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
 import org.slf4j.Logger;
 
@@ -66,46 +60,4 @@ public class SchemaRegionUtils {
       storageGroupDir.delete();
     }
   }
-
-  public static void checkDataTypeMatch(InsertPlan plan, int loc, TSDataType dataTypeInSchema)
-      throws MetadataException {
-    TSDataType insertDataType;
-    if (plan instanceof InsertRowPlan) {
-      if (!((InsertRowPlan) plan).isNeedInferType()) {
-        // only when InsertRowPlan's values is object[], we should check type
-        insertDataType = getTypeInLoc(plan, loc);
-      } else {
-        insertDataType = dataTypeInSchema;
-      }
-    } else {
-      insertDataType = getTypeInLoc(plan, loc);
-    }
-    if (dataTypeInSchema != insertDataType) {
-      String measurement = plan.getMeasurements()[loc];
-      String device = plan.getDevicePath().getFullPath();
-      throw new DataTypeMismatchException(
-          device,
-          measurement,
-          insertDataType,
-          dataTypeInSchema,
-          plan.getMinTime(),
-          plan.getFirstValueOfIndex(loc));
-    }
-  }
-
-  private static TSDataType getTypeInLoc(InsertPlan plan, int loc) throws MetadataException {
-    TSDataType dataType;
-    if (plan instanceof InsertRowPlan) {
-      InsertRowPlan tPlan = (InsertRowPlan) plan;
-      dataType =
-          TypeInferenceUtils.getPredictedDataType(tPlan.getValues()[loc], tPlan.isNeedInferType());
-    } else if (plan instanceof InsertTabletPlan) {
-      dataType = (plan).getDataTypes()[loc];
-    } else {
-      throw new MetadataException(
-          String.format(
-              "Only support insert and insertTablet, plan is [%s]", plan.getOperatorType()));
-    }
-    return dataType;
-  }
 }
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 606ee5dd93..c0fec29dfd 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
@@ -81,7 +81,7 @@ public class InsertTabletNode extends InsertNode implements WALEntryValue {
 
   private int rowCount = 0;
 
-  // when this plan is sub-plan split from another InsertTabletPlan, this indicates the original
+  // when this plan is sub-plan split from another InsertTabletNode, this indicates the original
   // positions of values in
   // this plan. For example, if the plan contains 5 timestamps, and range = [1,4,10,12], then it
   // means that the first 3
diff --git a/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/dto/IoTDBPoint.java b/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/dto/IoTDBPoint.java
index a19f3682fa..57f27cb2f4 100644
--- a/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/dto/IoTDBPoint.java
+++ b/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/dto/IoTDBPoint.java
@@ -18,11 +18,7 @@
  */
 package org.apache.iotdb.db.protocol.influxdb.dto;
 
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.protocol.influxdb.meta.IInfluxDBMetaManager;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
 import org.apache.iotdb.db.utils.DataTypeUtils;
 import org.apache.iotdb.db.utils.ParameterUtils;
 import org.apache.iotdb.rpc.IoTDBConnectionException;
@@ -147,16 +143,6 @@ public class IoTDBPoint {
     return values;
   }
 
-  public InsertRowPlan convertToInsertRowPlan()
-      throws IllegalPathException, IoTDBConnectionException, QueryProcessException {
-    return new InsertRowPlan(
-        new PartialPath(getDeviceId()),
-        getTime(),
-        getMeasurements().toArray(new String[0]),
-        DataTypeUtils.getValueBuffer(getTypes(), getValues()),
-        false);
-  }
-
   /**
    * Convert IoTDB point to InsertRecordReq
    *
diff --git a/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/meta/TagInfoRecords.java b/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/meta/TagInfoRecords.java
index c38df89d0f..d604956429 100644
--- a/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/meta/TagInfoRecords.java
+++ b/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/meta/TagInfoRecords.java
@@ -18,17 +18,11 @@
  */
 package org.apache.iotdb.db.protocol.influxdb.meta;
 
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
 import org.apache.iotdb.db.utils.DataTypeUtils;
 import org.apache.iotdb.rpc.IoTDBConnectionException;
 import org.apache.iotdb.service.rpc.thrift.TSInsertRecordReq;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 
-import org.influxdb.InfluxDBException;
-
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicLong;
@@ -81,24 +75,6 @@ public class TagInfoRecords {
     valuesList.add(values);
   }
 
-  public List<InsertRowPlan> convertToInsertRowPlans() {
-    ArrayList<InsertRowPlan> insertRowPlans = new ArrayList<>();
-    for (int i = 0; i < deviceIds.size(); i++) {
-      try {
-        insertRowPlans.add(
-            new InsertRowPlan(
-                new PartialPath(deviceIds.get(i)),
-                times.get(i),
-                measurementsList.get(i).toArray(new String[0]),
-                DataTypeUtils.getValueBuffer(typesList.get(i), valuesList.get(i)),
-                false));
-      } catch (QueryProcessException | IllegalPathException | IoTDBConnectionException e) {
-        throw new InfluxDBException(e.getMessage());
-      }
-    }
-    return insertRowPlans;
-  }
-
   public List<TSInsertRecordReq> convertToInsertRecordsReq(long sessionID)
       throws IoTDBConnectionException {
     ArrayList<TSInsertRecordReq> reqs = new ArrayList<>();
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/DeleteDataOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/DeleteDataOperator.java
index d424985b9b..a0898ed1e9 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/DeleteDataOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/DeleteDataOperator.java
@@ -22,15 +22,11 @@ import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
 import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 
 import java.util.ArrayList;
-import java.util.LinkedList;
 import java.util.List;
 
-import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD;
-
 /** this class extends {@code RootOperator} and process delete statement. */
 public class DeleteDataOperator extends Operator {
 
@@ -72,15 +68,6 @@ public class DeleteDataOperator extends Operator {
   @Override
   public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
       throws QueryProcessException {
-    List<PartialPath> originPath = getPaths();
-    if (isPrefixMatchPath()) {
-      // adapt to prefix match of 0.12
-      List<PartialPath> addedPath = new LinkedList<>();
-      for (PartialPath path : originPath) {
-        addedPath.add(path.concatNode(MULTI_LEVEL_PATH_WILDCARD));
-      }
-      originPath.addAll(addedPath);
-    }
-    return new DeletePlan(getStartTime(), getEndTime(), originPath);
+    return null;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InsertOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InsertOperator.java
index adc3988088..12f43e9110 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InsertOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InsertOperator.java
@@ -20,11 +20,8 @@ package org.apache.iotdb.db.qp.logical.crud;
 
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.exception.sql.SQLParserException;
 import org.apache.iotdb.db.qp.logical.Operator;
 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.InsertRowsPlan;
 import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 
 import java.util.List;
@@ -88,32 +85,6 @@ public class InsertOperator extends Operator {
   @Override
   public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
       throws QueryProcessException {
-    int measurementsNum = measurementList.length;
-    if (times.length == 1) {
-      if (measurementsNum != valueLists.get(0).length) {
-        throw new SQLParserException(
-            String.format(
-                "the measurementList's size %d is not consistent with the valueList's size %d",
-                measurementsNum, valueLists.get(0).length));
-      }
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(device, times[0], measurementList, valueLists.get(0));
-      insertRowPlan.setAligned(isAligned);
-      return insertRowPlan;
-    }
-    InsertRowsPlan insertRowsPlan = new InsertRowsPlan();
-    for (int i = 0; i < times.length; i++) {
-      if (measurementsNum != valueLists.get(i).length) {
-        throw new SQLParserException(
-            String.format(
-                "the measurementList's size %d is not consistent with the valueList's size %d",
-                measurementsNum, valueLists.get(i).length));
-      }
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(device, times[i], measurementList.clone(), valueLists.get(i));
-      insertRowPlan.setAligned(isAligned);
-      insertRowsPlan.addOneInsertRowPlan(insertRowPlan, i);
-    }
-    return insertRowsPlan;
+    return null;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DeletePartitionOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DeletePartitionOperator.java
index 8b0e673970..e026e62beb 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DeletePartitionOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/DeletePartitionOperator.java
@@ -23,7 +23,6 @@ import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePartitionPlan;
 import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 
 import java.util.Set;
@@ -57,6 +56,6 @@ public class DeletePartitionOperator extends Operator {
   @Override
   public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
       throws QueryProcessException {
-    return new DeletePartitionPlan(storageGroupName, partitionIds);
+    return null;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
index 39ef94b632..52df1bef5f 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java
@@ -25,12 +25,6 @@ import org.apache.iotdb.consensus.common.request.IConsensusRequest;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-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.qp.physical.crud.SelectIntoPlan;
 import org.apache.iotdb.db.qp.physical.sys.ActivateTemplateInClusterPlan;
 import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
@@ -313,18 +307,6 @@ public abstract class PhysicalPlan implements IConsensusRequest {
       PhysicalPlan plan;
       // TODO-Cluster: support more plans
       switch (type) {
-        case INSERT:
-          plan = new InsertRowPlan();
-          break;
-        case BATCHINSERT:
-          plan = new InsertTabletPlan();
-          break;
-        case MULTI_BATCH_INSERT:
-          plan = new InsertMultiTabletsPlan();
-          break;
-        case DELETE:
-          plan = new DeletePlan();
-          break;
         case SET_STORAGE_GROUP:
           plan = new SetStorageGroupPlan();
           break;
@@ -421,12 +403,6 @@ public abstract class PhysicalPlan implements IConsensusRequest {
         case STORAGE_GROUP_MNODE:
           plan = new StorageGroupMNodePlan();
           break;
-        case BATCH_INSERT_ROWS:
-          plan = new InsertRowsPlan();
-          break;
-        case BATCH_INSERT_ONE_DEVICE:
-          plan = new InsertRowsOfOneDevicePlan();
-          break;
         case CREATE_TRIGGER:
           plan = new CreateTriggerPlan();
           break;
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePartitionPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePartitionPlan.java
deleted file mode 100644
index 285d6d396a..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePartitionPlan.java
+++ /dev/null
@@ -1,53 +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.crud;
-
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-
-public class DeletePartitionPlan extends PhysicalPlan {
-
-  private PartialPath storageGroupName;
-  private Set<Long> partitionId;
-
-  public DeletePartitionPlan(PartialPath storageGroupName, Set<Long> partitionId) {
-    super(OperatorType.DELETE_PARTITION);
-    this.storageGroupName = storageGroupName;
-    this.partitionId = partitionId;
-  }
-
-  @Override
-  public List<PartialPath> getPaths() {
-    return Collections.emptyList();
-  }
-
-  public PartialPath getStorageGroupName() {
-    return storageGroupName;
-  }
-
-  public Set<Long> getPartitionId() {
-    return partitionId;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java
deleted file mode 100644
index 74155eb30f..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java
+++ /dev/null
@@ -1,221 +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.crud;
-
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.engine.storagegroup.DataRegion.TimePartitionFilter;
-import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
-import org.apache.iotdb.db.wal.buffer.WALEntryValue;
-import org.apache.iotdb.db.wal.utils.WALWriteUtils;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-
-public class DeletePlan extends PhysicalPlan implements WALEntryValue {
-  /** byte: type, integer: paths.size(), long: deleteStartTime, deleteEndTime, index */
-  private static final int FIXED_SERIALIZED_SIZE = Byte.BYTES + Integer.BYTES + Long.BYTES * 3;
-
-  private long deleteStartTime;
-  private long deleteEndTime;
-  private List<PartialPath> paths = new ArrayList<>();
-  /**
-   * This deletion only affects those time partitions that evaluate true by the filter. If the
-   * filter is null, all partitions are processed. This is to avoid redundant data deletions when
-   * one timeseries deletion is split and executed into different replication groups.
-   */
-  private TimePartitionFilter partitionFilter;
-
-  public DeletePlan() {
-    super(Operator.OperatorType.DELETE);
-  }
-
-  /**
-   * constructor of DeletePlan with single path.
-   *
-   * @param startTime delete time range start
-   * @param endTime delete time range end
-   * @param path time series path
-   */
-  public DeletePlan(long startTime, long endTime, PartialPath path) {
-    super(Operator.OperatorType.DELETE);
-    this.deleteStartTime = startTime;
-    this.deleteEndTime = endTime;
-    this.paths.add(path);
-  }
-
-  /**
-   * constructor of DeletePlan with multiple paths.
-   *
-   * @param startTime delete time range start
-   * @param endTime delete time range end
-   * @param paths time series paths in List structure
-   */
-  public DeletePlan(long startTime, long endTime, List<PartialPath> paths) {
-    super(Operator.OperatorType.DELETE);
-    this.deleteStartTime = startTime;
-    this.deleteEndTime = endTime;
-    this.paths = paths;
-  }
-
-  public long getDeleteStartTime() {
-    return deleteStartTime;
-  }
-
-  public void setDeleteStartTime(long delTime) {
-    this.deleteStartTime = delTime;
-  }
-
-  public long getDeleteEndTime() {
-    return deleteEndTime;
-  }
-
-  public void setDeleteEndTime(long delTime) {
-    this.deleteEndTime = delTime;
-  }
-
-  public void addPath(PartialPath path) {
-    this.paths.add(path);
-  }
-
-  public void addPaths(List<PartialPath> paths) {
-    this.paths.addAll(paths);
-  }
-
-  @Override
-  public List<PartialPath> getPaths() {
-    return paths;
-  }
-
-  @Override
-  public void setPaths(List<PartialPath> paths) {
-    this.paths = paths;
-  }
-
-  public TimePartitionFilter getPartitionFilter() {
-    return partitionFilter;
-  }
-
-  public void setPartitionFilter(TimePartitionFilter partitionFilter) {
-    this.partitionFilter = partitionFilter;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(deleteStartTime, deleteEndTime, paths);
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    DeletePlan that = (DeletePlan) o;
-    return deleteStartTime == that.deleteStartTime
-        && deleteEndTime == that.deleteEndTime
-        && Objects.equals(paths, that.paths);
-  }
-
-  @Override
-  public int serializedSize() {
-    int size = FIXED_SERIALIZED_SIZE;
-    for (PartialPath path : paths) {
-      size += ReadWriteIOUtils.sizeToWrite(path.getFullPath());
-    }
-    return size;
-  }
-
-  @Override
-  public void serialize(DataOutputStream stream) throws IOException {
-    int type = PhysicalPlanType.DELETE.ordinal();
-    stream.writeByte((byte) type);
-    stream.writeLong(deleteStartTime);
-    stream.writeLong(deleteEndTime);
-    stream.writeInt(paths.size());
-    for (PartialPath path : paths) {
-      putString(stream, path.getFullPath());
-    }
-
-    stream.writeLong(index);
-  }
-
-  @Override
-  public void serializeImpl(ByteBuffer buffer) {
-    int type = PhysicalPlanType.DELETE.ordinal();
-    buffer.put((byte) type);
-    buffer.putLong(deleteStartTime);
-    buffer.putLong(deleteEndTime);
-    buffer.putInt(paths.size());
-    for (PartialPath path : paths) {
-      putString(buffer, path.getFullPath());
-    }
-
-    buffer.putLong(index);
-  }
-
-  @Override
-  public void serializeToWAL(IWALByteBufferView buffer) {
-    int type = PhysicalPlanType.DELETE.ordinal();
-    buffer.put((byte) type);
-    buffer.putLong(deleteStartTime);
-    buffer.putLong(deleteEndTime);
-    buffer.putInt(paths.size());
-    for (PartialPath path : paths) {
-      WALWriteUtils.write(path.getFullPath(), buffer);
-    }
-    buffer.putLong(index);
-  }
-
-  @Override
-  public void deserialize(DataInputStream stream) throws IOException, IllegalPathException {
-    this.deleteStartTime = stream.readLong();
-    this.deleteEndTime = stream.readLong();
-    int pathSize = stream.readInt();
-    this.paths = new ArrayList<>();
-    for (int i = 0; i < pathSize; i++) {
-      paths.add(new PartialPath(ReadWriteIOUtils.readString(stream)));
-    }
-
-    this.index = stream.readLong();
-  }
-
-  @Override
-  public void deserialize(ByteBuffer buffer) throws IllegalPathException {
-    this.deleteStartTime = buffer.getLong();
-    this.deleteEndTime = buffer.getLong();
-    int pathSize = buffer.getInt();
-    this.paths = new ArrayList<>();
-    for (int i = 0; i < pathSize; i++) {
-      paths.add(new PartialPath(readString(buffer)));
-    }
-
-    this.index = buffer.getLong();
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertMultiTabletsPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertMultiTabletsPlan.java
deleted file mode 100644
index d653cfc83d..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertMultiTabletsPlan.java
+++ /dev/null
@@ -1,453 +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.crud;
-
-import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.commons.utils.StatusUtils;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-import org.apache.iotdb.db.qp.physical.BatchPlan;
-import org.apache.iotdb.tsfile.exception.NotImplementedException;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.TreeMap;
-
-/**
- * Mainly used in the distributed version, when multiple InsertTabletPlans belong to a raft
- * replication group, we merge these InsertTabletPlans into one InsertMultiTabletsPlan, which can
- * reduce the number of raft logs. For details, please refer to
- * https://issues.apache.org/jira/browse/IOTDB-1099
- */
-public class InsertMultiTabletsPlan extends InsertPlan implements BatchPlan {
-
-  /**
-   * the value is used to indict the parent InsertTabletPlan's index when the parent
-   * InsertTabletPlan is split to multi sub InsertTabletPlans. if the InsertTabletPlan have no
-   * parent plan, the value is zero;
-   *
-   * <p>suppose we originally have three InsertTabletPlans in one InsertMultiTabletsPlan, then the
-   * initial InsertMultiTabletsPlan would have the following two attributes:
-   *
-   * <p>insertTabletPlanList={InsertTabletPlan_1,InsertTabletPlan_2,InsertTabletPlan_3}
-   *
-   * <p>parentInsetTablePlanIndexList={0,0,0} both have three values.
-   *
-   * <p>if the InsertTabletPlan_1 is split into two sub InsertTabletPlans, InsertTabletPlan_2 is
-   * split into three sub InsertTabletPlans, InsertTabletPlan_3 is split into four sub
-   * InsertTabletPlans.
-   *
-   * <p>InsertTabletPlan_1={InsertTabletPlan_1_subPlan1, InsertTabletPlan_1_subPlan2}
-   *
-   * <p>InsertTabletPlan_2={InsertTabletPlan_2_subPlan1, InsertTabletPlan_2_subPlan2,
-   * InsertTabletPlan_2_subPlan3}
-   *
-   * <p>InsertTabletPlan_3={InsertTabletPlan_3_subPlan1, InsertTabletPlan_3_subPlan2,
-   * InsertTabletPlan_3_subPlan3, InsertTabletPlan_3_subPlan4}
-   *
-   * <p>those sub plans belong to two different raft data groups, so will generate two new
-   * InsertMultiTabletPlans
-   *
-   * <p>InsertMultiTabletPlant1.insertTabletPlanList={InsertTabletPlan_1_subPlan1,
-   * InsertTabletPlan_3_subPlan1, InsertTabletPlan_3_subPlan3, InsertTabletPlan_3_subPlan4}
-   *
-   * <p>InsertMultiTabletPlant1.parentInsetTablePlanIndexList={0,2,2,2}
-   *
-   * <p>InsertMultiTabletPlant2.insertTabletPlanList={InsertTabletPlan_1_subPlan2,
-   * InsertTabletPlan_2_subPlan1, InsertTabletPlan_2_subPlan2, InsertTabletPlan_2_subPlan3,
-   * InsertTabletPlan_3_subPlan2}
-   *
-   * <p>InsertMultiTabletPlant2.parentInsetTablePlanIndexList={0,1,1,1,2}
-   *
-   * <p>this is usually used to back-propagate exceptions to the parent plan without losing their
-   * proper positions.
-   */
-  List<Integer> parentInsertTabletPlanIndexList;
-
-  /** the InsertTabletPlan list */
-  List<InsertTabletPlan> insertTabletPlanList;
-
-  /** record the result of creation of time series */
-  private Map<Integer, TSStatus> results = new TreeMap<>();
-
-  private List<PartialPath> prefixPaths;
-
-  boolean[] isExecuted;
-
-  Boolean isEnableMultithreading;
-
-  Integer differentStorageGroupsCount;
-
-  public InsertMultiTabletsPlan() {
-    super(OperatorType.MULTI_BATCH_INSERT);
-    this.insertTabletPlanList = new ArrayList<>();
-    this.parentInsertTabletPlanIndexList = new ArrayList<>();
-  }
-
-  public InsertMultiTabletsPlan(List<InsertTabletPlan> insertTabletPlanList) {
-    super(OperatorType.MULTI_BATCH_INSERT);
-    this.insertTabletPlanList = insertTabletPlanList;
-    this.parentInsertTabletPlanIndexList = new ArrayList<>();
-  }
-
-  public InsertMultiTabletsPlan(
-      List<InsertTabletPlan> insertTabletPlanList, List<Integer> parentInsertTabletPlanIndexList) {
-    super(OperatorType.MULTI_BATCH_INSERT);
-    this.insertTabletPlanList = insertTabletPlanList;
-    this.parentInsertTabletPlanIndexList = parentInsertTabletPlanIndexList;
-  }
-
-  public void addInsertTabletPlan(InsertTabletPlan plan, Integer parentIndex) {
-    insertTabletPlanList.add(plan);
-    parentInsertTabletPlanIndexList.add(parentIndex);
-  }
-
-  @Override
-  public List<PartialPath> getPaths() {
-    List<PartialPath> result = new ArrayList<>();
-    for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-      result.addAll(insertTabletPlan.getPaths());
-    }
-    return result;
-  }
-
-  @Override
-  public List<PartialPath> getPrefixPaths() {
-    if (prefixPaths != null) {
-      return prefixPaths;
-    }
-    prefixPaths = new ArrayList<>(insertTabletPlanList.size());
-    for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-      prefixPaths.add(insertTabletPlan.getDevicePath());
-    }
-    return prefixPaths;
-  }
-
-  @Override
-  public long getMinTime() {
-    long minTime = Long.MAX_VALUE;
-    for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-      if (minTime > insertTabletPlan.getMinTime()) {
-        minTime = insertTabletPlan.getMinTime();
-      }
-    }
-    return minTime;
-  }
-
-  public long getMaxTime() {
-    long maxTime = Long.MIN_VALUE;
-    for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-      if (maxTime < insertTabletPlan.getMaxTime()) {
-        maxTime = insertTabletPlan.getMaxTime();
-      }
-    }
-    return maxTime;
-  }
-
-  @Override
-  public Object getFirstValueOfIndex(int index) {
-    throw new NotImplementedException();
-  }
-
-  public int getTabletsSize() {
-    return insertTabletPlanList.size();
-  }
-
-  @Override
-  public Map<Integer, TSStatus> getResults() {
-    return results;
-  }
-
-  /** @return the total row of the whole InsertTabletPlan */
-  public int getTotalRowCount() {
-    int rowCount = 0;
-    for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-      rowCount += insertTabletPlan.getRowCount();
-    }
-    return rowCount;
-  }
-
-  /**
-   * Gets the number of rows in the InsertTabletPlan of the index, zero will be returned if the
-   * index is out of bound.
-   *
-   * @param index the index of the insertTabletPlanList
-   * @return the total row count of the insertTabletPlanList.get(i)
-   */
-  public int getRowCount(int index) {
-    if (index >= insertTabletPlanList.size() || index < 0) {
-      return 0;
-    }
-    return insertTabletPlanList.get(index).getRowCount();
-  }
-
-  public PartialPath getFirstDeviceId() {
-    return insertTabletPlanList.get(0).getDevicePath();
-  }
-
-  public InsertTabletPlan getInsertTabletPlan(int index) {
-    if (index >= insertTabletPlanList.size() || index < 0) {
-      return null;
-    }
-    return insertTabletPlanList.get(index);
-  }
-
-  /**
-   * @param index the index of the sub plan in this InsertMultiTabletsPlan
-   * @return the parent's index in the parent InsertMultiTabletsPlan
-   */
-  public int getParentIndex(int index) {
-    if (index >= parentInsertTabletPlanIndexList.size() || index < 0) {
-      return -1;
-    }
-    return parentInsertTabletPlanIndexList.get(index);
-  }
-
-  @Override
-  public void checkIntegrity() throws QueryProcessException {
-    if (insertTabletPlanList.isEmpty()) {
-      throw new QueryProcessException("sub tablet is empty.");
-    }
-    for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-      insertTabletPlan.checkIntegrity();
-    }
-  }
-
-  public void setParentInsertTabletPlanIndexList(List<Integer> parentInsertTabletPlanIndexList) {
-    this.parentInsertTabletPlanIndexList = parentInsertTabletPlanIndexList;
-  }
-
-  public List<Integer> getParentInsertTabletPlanIndexList() {
-    return parentInsertTabletPlanIndexList;
-  }
-
-  public void setInsertTabletPlanList(List<InsertTabletPlan> insertTabletPlanList) {
-    this.insertTabletPlanList = insertTabletPlanList;
-  }
-
-  public List<InsertTabletPlan> getInsertTabletPlanList() {
-    return insertTabletPlanList;
-  }
-
-  public TSStatus[] getFailingStatus() {
-    return StatusUtils.getFailingStatus(results, insertTabletPlanList.size());
-  }
-
-  public void setResults(Map<Integer, TSStatus> results) {
-    this.results = results;
-  }
-
-  @Override
-  public void recoverFromFailure() {
-    for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-      insertTabletPlan.recoverFromFailure();
-    }
-  }
-
-  @Override
-  public void serializeImpl(ByteBuffer buffer) {
-    int type = PhysicalPlanType.MULTI_BATCH_INSERT.ordinal();
-    buffer.put((byte) type);
-    buffer.putInt(insertTabletPlanList.size());
-    for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-      insertTabletPlan.subSerialize(buffer, 0, insertTabletPlan.getRowCount());
-    }
-
-    buffer.putInt(parentInsertTabletPlanIndexList.size());
-    for (Integer index : parentInsertTabletPlanIndexList) {
-      buffer.putInt(index);
-    }
-  }
-
-  @Override
-  public void serialize(DataOutputStream stream) throws IOException {
-    int type = PhysicalPlanType.MULTI_BATCH_INSERT.ordinal();
-    stream.writeByte((byte) type);
-    stream.writeInt(insertTabletPlanList.size());
-    for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-      insertTabletPlan.subSerialize(stream, 0, insertTabletPlan.getRowCount());
-    }
-
-    stream.writeInt(parentInsertTabletPlanIndexList.size());
-    for (Integer index : parentInsertTabletPlanIndexList) {
-      stream.writeInt(index);
-    }
-  }
-
-  @Override
-  public void deserialize(ByteBuffer buffer) throws IllegalPathException {
-    int tmpInsertTabletPlanListSize = buffer.getInt();
-    this.insertTabletPlanList = new ArrayList<>(tmpInsertTabletPlanListSize);
-    for (int i = 0; i < tmpInsertTabletPlanListSize; i++) {
-      InsertTabletPlan tmpPlan = new InsertTabletPlan();
-      tmpPlan.deserialize(buffer);
-      this.insertTabletPlanList.add(tmpPlan);
-    }
-
-    int tmpParentInsetTablePlanIndexListSize = buffer.getInt();
-    this.parentInsertTabletPlanIndexList = new ArrayList<>(tmpParentInsetTablePlanIndexListSize);
-    for (int i = 0; i < tmpParentInsetTablePlanIndexListSize; i++) {
-      this.parentInsertTabletPlanIndexList.add(buffer.getInt());
-    }
-  }
-
-  @Override
-  public void setIndex(long index) {
-    super.setIndex(index);
-    for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-      // use the InsertMultiTabletsPlan's index as the sub InsertTabletPlan's index
-      insertTabletPlan.setIndex(index);
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "InsertMultiTabletsPlan{"
-        + " insertTabletPlanList="
-        + insertTabletPlanList
-        + ", parentInsetTablePlanIndexList="
-        + parentInsertTabletPlanIndexList
-        + "}";
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-
-    InsertMultiTabletsPlan that = (InsertMultiTabletsPlan) o;
-
-    if (!Objects.equals(insertTabletPlanList, that.insertTabletPlanList)) {
-      return false;
-    }
-    return Objects.equals(parentInsertTabletPlanIndexList, that.parentInsertTabletPlanIndexList);
-  }
-
-  @Override
-  public int hashCode() {
-    int result = insertTabletPlanList != null ? insertTabletPlanList.hashCode() : 0;
-    result =
-        31 * result
-            + (parentInsertTabletPlanIndexList != null
-                ? parentInsertTabletPlanIndexList.hashCode()
-                : 0);
-    return result;
-  }
-
-  @Override
-  public void setIsExecuted(int i) {
-    if (isExecuted == null) {
-      isExecuted = new boolean[getBatchSize()];
-    }
-    isExecuted[i] = true;
-  }
-
-  @Override
-  public boolean isExecuted(int i) {
-    if (isExecuted == null) {
-      isExecuted = new boolean[getBatchSize()];
-    }
-    return isExecuted[i];
-  }
-
-  @Override
-  public int getBatchSize() {
-    return insertTabletPlanList.size();
-  }
-
-  @Override
-  public void unsetIsExecuted(int i) {
-    if (isExecuted == null) {
-      isExecuted = new boolean[getBatchSize()];
-    }
-    isExecuted[i] = false;
-    if (parentInsertTabletPlanIndexList != null && !parentInsertTabletPlanIndexList.isEmpty()) {
-      results.remove(getParentIndex(i));
-    } else {
-      results.remove(i);
-    }
-  }
-
-  public int getDifferentStorageGroupsCount() {
-    if (differentStorageGroupsCount == null) {
-      Set<String> insertPlanSGSet = new HashSet<>();
-      int defaultStorageGroupLevel =
-          IoTDBDescriptor.getInstance().getConfig().getDefaultStorageGroupLevel();
-      for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-        String[] nodes = insertTabletPlan.getDevicePath().getNodes();
-        StringBuilder stringBuilder = new StringBuilder();
-        for (int i = 0; i <= defaultStorageGroupLevel && i < nodes.length; i++) {
-          stringBuilder.append(nodes[i]).append(".");
-        }
-        insertPlanSGSet.add(stringBuilder.toString());
-      }
-      differentStorageGroupsCount = insertPlanSGSet.size();
-    }
-    return differentStorageGroupsCount;
-  }
-
-  public boolean isEnableMultiThreading() {
-    // If we enable multithreading, we need to consider the loss of switching between threads,
-    // so we need to judge the core threads of the thread pool and the size of the tablet.
-
-    // Therefore, we set the number of core threads in the thread pool to
-    // min(the number of different sg, availableProcessors()/2),
-    // and need columns >= insertMultiTabletEnableMultithreadingColumnThreshold.
-
-    // It should be noted that if the number of sg is large and exceeds twice of the recommended
-    // number of CPU threads,
-    // it may lead to failure to allocate out of heap memory and NPE.
-    // Therefore, we will also turn off multithreading in this case.
-    if (isEnableMultithreading == null) {
-      int sgSize = getDifferentStorageGroupsCount();
-      // SG should be >= 1 so that it will not be locked and degenerate into serial.
-      // SG should be <= Runtime.getRuntime().availableProcessors()*2  so that to avoid failure to
-      // allocate out of heap memory and NPE
-      if (sgSize <= 1 || sgSize >= Runtime.getRuntime().availableProcessors() * 2) {
-        isEnableMultithreading = false;
-      } else {
-        int count = 0;
-        for (InsertTabletPlan insertTabletPlan : insertTabletPlanList) {
-          if (insertTabletPlan.getColumns().length
-              >= IoTDBDescriptor.getInstance()
-                  .getConfig()
-                  .getInsertMultiTabletEnableMultithreadingColumnThreshold()) {
-            count++;
-          }
-        }
-        isEnableMultithreading = count * 2 >= insertTabletPlanList.size();
-      }
-    }
-    return isEnableMultithreading;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java
deleted file mode 100644
index a5ef81ff12..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java
+++ /dev/null
@@ -1,239 +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.crud;
-
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.qp.logical.Operator;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-public abstract class InsertPlan extends PhysicalPlan {
-
-  /**
-   * if use id table, this filed is id form of device path <br>
-   * if not, this filed is device path<br>
-   */
-  protected PartialPath devicePath;
-
-  protected boolean isAligned;
-  protected String[] measurements;
-  // get from client
-  protected TSDataType[] dataTypes;
-  // get from SchemaProcessor
-  protected IMeasurementMNode[] measurementMNodes;
-
-  /**
-   * device id reference, for reuse device id in both id table and memtable <br>
-   * used in memtable
-   */
-  protected IDeviceID deviceID;
-
-  // record the failed measurements, their reasons, and positions in "measurements"
-  List<String> failedMeasurements;
-  private List<Exception> failedExceptions;
-  List<Integer> failedIndices;
-
-  public InsertPlan(Operator.OperatorType operatorType) {
-    super(operatorType);
-    super.canBeSplit = false;
-  }
-
-  /**
-   * if use id table, this filed is id form of device path <br>
-   * if not, this filed is device path<br>
-   * used in flush time manager, last cache, tsfile processor
-   */
-  public PartialPath getDevicePath() {
-    return devicePath;
-  }
-
-  public void setDevicePath(PartialPath devicePath) {
-    this.devicePath = devicePath;
-  }
-
-  public String[] getMeasurements() {
-    return this.measurements;
-  }
-
-  public void setMeasurements(String[] measurements) {
-    this.measurements = measurements;
-  }
-
-  public boolean hasValidMeasurements() {
-    for (Object o : measurements) {
-      if (o != null) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  public TSDataType[] getDataTypes() {
-    return dataTypes;
-  }
-
-  public void setDataTypes(TSDataType[] dataTypes) {
-    this.dataTypes = dataTypes;
-  }
-
-  public IMeasurementMNode[] getMeasurementMNodes() {
-    return measurementMNodes;
-  }
-
-  public void setMeasurementMNodes(IMeasurementMNode[] mNodes) {
-    this.measurementMNodes = mNodes;
-  }
-
-  public List<String> getFailedMeasurements() {
-    return failedMeasurements;
-  }
-
-  public List<Exception> getFailedExceptions() {
-    return failedExceptions;
-  }
-
-  public int getFailedMeasurementNumber() {
-    return failedMeasurements == null ? 0 : failedMeasurements.size();
-  }
-
-  public boolean isAligned() {
-    return isAligned;
-  }
-
-  public void setAligned(boolean aligned) {
-    isAligned = aligned;
-  }
-
-  public abstract long getMinTime();
-
-  public abstract Object getFirstValueOfIndex(int index);
-
-  /**
-   * This method is overrided in InsertRowPlan and InsertTabletPlan. After marking failed
-   * measurements, the failed values or columns would be null as well. We'd better use
-   * "measurements[index] == null" to determine if the measurement failed.
-   *
-   * @param index failed measurement index
-   */
-  public void markFailedMeasurementInsertion(int index, Exception e) {
-    if (measurements[index] == null) {
-      return;
-    }
-    if (failedMeasurements == null) {
-      failedMeasurements = new ArrayList<>();
-      failedExceptions = new ArrayList<>();
-      failedIndices = new ArrayList<>();
-    }
-    failedMeasurements.add(measurements[index]);
-    failedExceptions.add(e);
-    failedIndices.add(index);
-    measurements[index] = null;
-  }
-
-  /**
-   * Reconstruct this plan with the failed measurements.
-   *
-   * @return the plan itself, with measurements replaced with the previously failed ones.
-   */
-  public InsertPlan getPlanFromFailed() {
-    if (failedMeasurements == null) {
-      return null;
-    }
-    measurements = failedMeasurements.toArray(new String[0]);
-    failedMeasurements = null;
-    if (dataTypes != null) {
-      TSDataType[] temp = dataTypes.clone();
-      dataTypes = new TSDataType[failedIndices.size()];
-      for (int i = 0; i < failedIndices.size(); i++) {
-        dataTypes[i] = temp[failedIndices.get(i)];
-      }
-    }
-    if (measurementMNodes != null) {
-      IMeasurementMNode[] temp = measurementMNodes.clone();
-      measurementMNodes = new IMeasurementMNode[failedIndices.size()];
-      for (int i = 0; i < failedIndices.size(); i++) {
-        measurementMNodes[i] = temp[failedIndices.get(i)];
-      }
-    }
-
-    failedIndices = null;
-    failedExceptions = null;
-    return this;
-  }
-
-  /** Reset measurements from failed measurements (if any), as if no failure had ever happened. */
-  public void recoverFromFailure() {
-    if (failedMeasurements == null) {
-      return;
-    }
-
-    for (int i = 0; i < failedMeasurements.size(); i++) {
-      int index = failedIndices.get(i);
-      measurements[index] = failedMeasurements.get(i);
-    }
-    failedIndices = null;
-    failedExceptions = null;
-    failedMeasurements = null;
-  }
-
-  @Override
-  public void checkIntegrity() throws QueryProcessException {
-    if (devicePath == null) {
-      throw new QueryProcessException("DeviceId is null");
-    }
-    if (measurements == null) {
-      throw new QueryProcessException("Measurements are null");
-    }
-    Set<String> deduplicatedMeasurements = new HashSet<>();
-    for (String measurement : measurements) {
-      if (measurement == null || measurement.isEmpty()) {
-        throw new QueryProcessException(
-            "Measurement contains null or empty string: " + Arrays.toString(measurements));
-      }
-      if (deduplicatedMeasurements.contains(measurement)) {
-        throw new QueryProcessException(
-            "Insertion contains duplicated measurement: " + measurement);
-      } else {
-        deduplicatedMeasurements.add(measurement);
-      }
-    }
-  }
-
-  /**
-   * device id reference, for reuse device id in both id table and memtable <br>
-   * used in memtable
-   */
-  public IDeviceID getDeviceID() {
-    return deviceID;
-  }
-
-  public void setDeviceID(IDeviceID deviceID) {
-    this.deviceID = deviceID;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java
deleted file mode 100644
index 0e7bc1e7fe..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowPlan.java
+++ /dev/null
@@ -1,812 +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.crud;
-
-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.commons.utils.TestOnly;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.metadata.PathNotExistException;
-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.logical.Operator;
-import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-import org.apache.iotdb.db.utils.CommonUtils;
-import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
-import org.apache.iotdb.db.wal.buffer.WALEntryValue;
-import org.apache.iotdb.db.wal.utils.WALWriteUtils;
-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.utils.Binary;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
-import org.apache.iotdb.tsfile.write.record.TSRecord;
-import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Objects;
-
-public class InsertRowPlan extends InsertPlan implements WALEntryValue {
-  private static final Logger logger = LoggerFactory.getLogger(InsertRowPlan.class);
-  private static final byte TYPE_RAW_STRING = -1;
-
-  private long time;
-  private Object[] values;
-
-  // if isNeedInferType is true, the values must be String[], so we could infer types from them
-  // if values is object[], we could use the raw type of them, and we should set this to false
-  private boolean isNeedInferType = false;
-
-  private List<Object> failedValues;
-  private List<PartialPath> paths;
-
-  public InsertRowPlan() {
-    super(OperatorType.INSERT);
-  }
-
-  public InsertRowPlan(InsertRowPlan another) {
-    super(OperatorType.INSERT);
-    this.devicePath = another.devicePath;
-    this.time = another.time;
-    this.measurements = new String[another.measurements.length];
-    System.arraycopy(another.measurements, 0, this.measurements, 0, another.measurements.length);
-    this.values = new Object[another.values.length];
-    System.arraycopy(another.values, 0, this.values, 0, another.values.length);
-    this.dataTypes = new TSDataType[another.dataTypes.length];
-    System.arraycopy(another.dataTypes, 0, this.dataTypes, 0, another.dataTypes.length);
-  }
-
-  public InsertRowPlan(
-      PartialPath prefixPath, long insertTime, String[] measurementList, String[] insertValues) {
-    super(Operator.OperatorType.INSERT);
-    this.time = insertTime;
-    this.devicePath = prefixPath;
-    this.measurements = measurementList;
-    this.dataTypes = new TSDataType[insertValues.length];
-    // We need to create an Object[] for the data type casting, because we can not set Float, Long
-    // to String[i]
-    this.values = new Object[insertValues.length];
-    System.arraycopy(insertValues, 0, values, 0, insertValues.length);
-    isNeedInferType = true;
-  }
-
-  public InsertRowPlan(
-      PartialPath prefixPath,
-      long insertTime,
-      String[] measurementList,
-      ByteBuffer values,
-      boolean isAligned)
-      throws QueryProcessException {
-    super(Operator.OperatorType.INSERT);
-    this.time = insertTime;
-    this.devicePath = prefixPath;
-    this.measurements = measurementList;
-    this.dataTypes = new TSDataType[measurementList.length];
-    this.values = new Object[measurementList.length];
-    this.fillValues(values);
-    isNeedInferType = false;
-    this.isAligned = isAligned;
-  }
-
-  @TestOnly
-  public InsertRowPlan(
-      PartialPath prefixPath,
-      long insertTime,
-      String[] measurements,
-      TSDataType[] dataTypes,
-      String[] insertValues) {
-    super(OperatorType.INSERT);
-    this.time = insertTime;
-    this.devicePath = prefixPath;
-    this.measurements = measurements;
-    this.dataTypes = dataTypes;
-    this.values = new Object[dataTypes.length];
-    for (int i = 0; i < dataTypes.length; i++) {
-      try {
-        values[i] = CommonUtils.parseValueForTest(dataTypes[i], insertValues[i]);
-      } catch (QueryProcessException e) {
-        e.printStackTrace();
-      }
-    }
-  }
-
-  @TestOnly
-  public InsertRowPlan(
-      PartialPath prefixPath,
-      long insertTime,
-      String[] measurements,
-      TSDataType[] dataTypes,
-      String[] insertValues,
-      boolean isAligned) {
-    super(OperatorType.INSERT);
-    this.time = insertTime;
-    this.devicePath = prefixPath;
-    this.measurements = measurements;
-    this.dataTypes = dataTypes;
-    this.values = new Object[dataTypes.length];
-    for (int i = 0; i < dataTypes.length; i++) {
-      try {
-        values[i] = CommonUtils.parseValueForTest(dataTypes[i], insertValues[i]);
-      } catch (QueryProcessException e) {
-        e.printStackTrace();
-      }
-    }
-    this.isAligned = isAligned;
-  }
-
-  @TestOnly
-  public InsertRowPlan(
-      PartialPath prefixPath,
-      long insertTime,
-      String measurement,
-      TSDataType type,
-      String insertValue) {
-    super(OperatorType.INSERT);
-    this.time = insertTime;
-    this.devicePath = prefixPath;
-    this.measurements = new String[] {measurement};
-    this.dataTypes = new TSDataType[] {type};
-    this.values = new Object[1];
-    try {
-      values[0] = CommonUtils.parseValueForTest(dataTypes[0], insertValue);
-    } catch (QueryProcessException e) {
-      logger.error(e.getMessage());
-    }
-  }
-
-  @TestOnly
-  public InsertRowPlan(TSRecord tsRecord) throws IllegalPathException {
-    super(OperatorType.INSERT);
-    this.devicePath = new PartialPath(tsRecord.deviceId);
-    this.time = tsRecord.time;
-    this.measurements = new String[tsRecord.dataPointList.size()];
-    this.measurementMNodes = new IMeasurementMNode[tsRecord.dataPointList.size()];
-    this.dataTypes = new TSDataType[tsRecord.dataPointList.size()];
-    this.values = new Object[tsRecord.dataPointList.size()];
-    for (int i = 0; i < tsRecord.dataPointList.size(); i++) {
-      measurements[i] = tsRecord.dataPointList.get(i).getMeasurementId();
-      measurementMNodes[i] =
-          MeasurementMNode.getMeasurementMNode(
-              null,
-              measurements[i],
-              new MeasurementSchema(
-                  measurements[i], tsRecord.dataPointList.get(i).getType(), TSEncoding.PLAIN),
-              null);
-      dataTypes[i] = tsRecord.dataPointList.get(i).getType();
-      values[i] = tsRecord.dataPointList.get(i).getValue();
-    }
-  }
-
-  public long getTime() {
-    return time;
-  }
-
-  public void setTime(long time) {
-    this.time = time;
-  }
-
-  public boolean isNeedInferType() {
-    return isNeedInferType;
-  }
-
-  public void setNeedInferType(boolean inferType) {
-    this.isNeedInferType = inferType;
-  }
-
-  /**
-   * if inferType is true, transfer String[] values to specific data types (Integer, Long, Float,
-   * Double, Binary)
-   */
-  @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
-  public void transferType() throws QueryProcessException {
-    if (isNeedInferType) {
-      for (int i = 0; i < measurementMNodes.length; i++) {
-        if (measurementMNodes[i] == null) {
-          if (IoTDBDescriptor.getInstance().getConfig().isEnablePartialInsert()) {
-            markFailedMeasurementInsertion(
-                i,
-                new QueryProcessException(
-                    new PathNotExistException(
-                        devicePath.getFullPath()
-                            + IoTDBConstant.PATH_SEPARATOR
-                            + measurements[i])));
-          } else {
-            throw new QueryProcessException(
-                new PathNotExistException(
-                    devicePath.getFullPath() + IoTDBConstant.PATH_SEPARATOR + measurements[i]));
-          }
-          continue;
-        }
-
-        dataTypes[i] = measurementMNodes[i].getSchema().getType();
-        try {
-          values[i] = CommonUtils.parseValue(dataTypes[i], values[i].toString());
-        } catch (Exception e) {
-          logger.warn(
-              "data type of {}.{} is not consistent, registered type {}, inserting timestamp {}, value {}",
-              devicePath,
-              measurements[i],
-              dataTypes[i],
-              time,
-              values[i]);
-          if (IoTDBDescriptor.getInstance().getConfig().isEnablePartialInsert()) {
-            markFailedMeasurementInsertion(i, e);
-            measurementMNodes[i] = null;
-          } else {
-            throw e;
-          }
-        }
-      }
-    }
-  }
-
-  @Override
-  public long getMinTime() {
-    return getTime();
-  }
-
-  @Override
-  public Object getFirstValueOfIndex(int index) {
-    return values[index];
-  }
-
-  @Override
-  public void markFailedMeasurementInsertion(int index, Exception e) {
-    if (measurements[index] == null) {
-      return;
-    }
-    super.markFailedMeasurementInsertion(index, e);
-    if (failedValues == null) {
-      failedValues = new ArrayList<>();
-    }
-    failedValues.add(values[index]);
-    values[index] = null;
-    if (isNeedInferType) {
-      dataTypes[index] = null;
-    }
-  }
-
-  @Override
-  public List<PartialPath> getPaths() {
-    if (paths != null) {
-      return paths;
-    }
-    paths = new ArrayList<>(measurements.length);
-    for (String m : measurements) {
-      PartialPath fullPath = devicePath.concatNode(m);
-      paths.add(fullPath);
-    }
-    return paths;
-  }
-
-  public Object[] getValues() {
-    return this.values;
-  }
-
-  public void setValues(Object[] values) {
-    this.values = values;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    InsertRowPlan that = (InsertRowPlan) o;
-    return time == that.time
-        && Objects.equals(devicePath, that.devicePath)
-        && Arrays.equals(measurements, that.measurements)
-        && Arrays.equals(values, that.values)
-        && Objects.equals(isAligned, that.isAligned);
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hash(devicePath, time);
-  }
-
-  @Override
-  public int serializedSize() {
-    int size = 0;
-    size += Byte.BYTES;
-    return size + subSerializeSize();
-  }
-
-  int subSerializeSize() {
-    int size = 0;
-    size += Long.BYTES;
-    size += ReadWriteIOUtils.sizeToWrite(devicePath.getFullPath());
-    return size + serializeMeasurementsAndValuesSize();
-  }
-
-  int serializeMeasurementsAndValuesSize() {
-    int size = 0;
-    size += Integer.BYTES;
-    for (String m : measurements) {
-      if (m != null) {
-        size += ReadWriteIOUtils.sizeToWrite(m);
-      }
-    }
-    // putValues
-    size += Integer.BYTES;
-    for (int i = 0; i < values.length; i++) {
-      if (values[i] == null) {
-        continue;
-      }
-      if (dataTypes == null || dataTypes[i] == null) {
-        size += Byte.BYTES + ReadWriteIOUtils.sizeToWrite(values[i].toString());
-      } else {
-        size += Byte.BYTES;
-        switch (dataTypes[i]) {
-          case BOOLEAN:
-            size += Byte.BYTES;
-            break;
-          case INT32:
-            size += Integer.BYTES;
-            break;
-          case INT64:
-            size += Long.BYTES;
-            break;
-          case FLOAT:
-            size += Float.BYTES;
-            break;
-          case DOUBLE:
-            size += Double.BYTES;
-            break;
-          case TEXT:
-            size += ReadWriteIOUtils.sizeToWrite((Binary) values[i]);
-            break;
-        }
-      }
-    }
-    size += Byte.BYTES;
-    size += Long.BYTES;
-    size += Byte.BYTES;
-    return size;
-  }
-
-  @Override
-  public void serialize(DataOutputStream stream) throws IOException {
-    int type = PhysicalPlanType.INSERT.ordinal();
-    stream.writeByte((byte) type);
-    subSerialize(stream);
-  }
-
-  public void subSerialize(DataOutputStream stream) throws IOException {
-    stream.writeLong(time);
-    putString(stream, devicePath.getFullPath());
-    serializeMeasurementsAndValues(stream);
-  }
-
-  void serializeMeasurementsAndValues(DataOutputStream stream) throws IOException {
-    stream.writeInt(measurements.length - getFailedMeasurementNumber());
-
-    for (String m : measurements) {
-      if (m != null) {
-        putString(stream, m);
-      }
-    }
-
-    try {
-      stream.writeInt(values.length - getFailedMeasurementNumber());
-      putValues(stream);
-    } catch (QueryProcessException e) {
-      throw new IOException(e);
-    }
-
-    // the types are not inferred before the plan is serialized
-    stream.write((byte) (isNeedInferType ? 1 : 0));
-
-    stream.writeLong(index);
-    stream.write((byte) (isAligned ? 1 : 0));
-  }
-
-  private void putValues(DataOutputStream outputStream) throws QueryProcessException, IOException {
-    for (int i = 0; i < values.length; i++) {
-      if (values[i] == null) {
-        continue;
-      }
-      // types are not determined, the situation mainly occurs when the plan uses string values
-      // and is forwarded to other nodes
-      if (dataTypes == null || dataTypes[i] == null) {
-        ReadWriteIOUtils.write(TYPE_RAW_STRING, outputStream);
-        ReadWriteIOUtils.write(values[i].toString(), outputStream);
-      } else {
-        ReadWriteIOUtils.write(dataTypes[i], outputStream);
-        switch (dataTypes[i]) {
-          case BOOLEAN:
-            ReadWriteIOUtils.write((Boolean) values[i], outputStream);
-            break;
-          case INT32:
-            ReadWriteIOUtils.write((Integer) values[i], outputStream);
-            break;
-          case INT64:
-            ReadWriteIOUtils.write((Long) values[i], outputStream);
-            break;
-          case FLOAT:
-            ReadWriteIOUtils.write((Float) values[i], outputStream);
-            break;
-          case DOUBLE:
-            ReadWriteIOUtils.write((Double) values[i], outputStream);
-            break;
-          case TEXT:
-            ReadWriteIOUtils.write((Binary) values[i], outputStream);
-            break;
-          default:
-            throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
-        }
-      }
-    }
-  }
-
-  public void fillValues(DataInputStream stream) throws IOException, QueryProcessException {
-    for (int i = 0; i < dataTypes.length; i++) {
-      // types are not determined, the situation mainly occurs when the plan uses string values
-      // and is forwarded to other nodes
-      byte typeNum = stream.readByte();
-      if (typeNum == TYPE_RAW_STRING) {
-        values[i] = ReadWriteIOUtils.readString(stream);
-        continue;
-      }
-      dataTypes[i] = TSDataType.values()[typeNum];
-      switch (dataTypes[i]) {
-        case BOOLEAN:
-          values[i] = ReadWriteIOUtils.readBool(stream);
-          break;
-        case INT32:
-          values[i] = ReadWriteIOUtils.readInt(stream);
-          break;
-        case INT64:
-          values[i] = ReadWriteIOUtils.readLong(stream);
-          break;
-        case FLOAT:
-          values[i] = ReadWriteIOUtils.readFloat(stream);
-          break;
-        case DOUBLE:
-          values[i] = ReadWriteIOUtils.readDouble(stream);
-          break;
-        case TEXT:
-          values[i] = ReadWriteIOUtils.readBinary(stream);
-          break;
-        default:
-          throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
-      }
-    }
-  }
-
-  /** Make sure the values is already inited before calling this */
-  public void fillValues(ByteBuffer buffer) throws QueryProcessException {
-    for (int i = 0; i < dataTypes.length; i++) {
-      // types are not determined, the situation mainly occurs when the plan uses string values
-      // and is forwarded to other nodes
-      byte typeNum = (byte) ReadWriteIOUtils.read(buffer);
-      if (typeNum == TYPE_RAW_STRING) {
-        values[i] = ReadWriteIOUtils.readString(buffer);
-        continue;
-      }
-      dataTypes[i] = TSDataType.values()[typeNum];
-      switch (dataTypes[i]) {
-        case BOOLEAN:
-          values[i] = ReadWriteIOUtils.readBool(buffer);
-          break;
-        case INT32:
-          values[i] = ReadWriteIOUtils.readInt(buffer);
-          break;
-        case INT64:
-          values[i] = ReadWriteIOUtils.readLong(buffer);
-          break;
-        case FLOAT:
-          values[i] = ReadWriteIOUtils.readFloat(buffer);
-          break;
-        case DOUBLE:
-          values[i] = ReadWriteIOUtils.readDouble(buffer);
-          break;
-        case TEXT:
-          values[i] = ReadWriteIOUtils.readBinary(buffer);
-          break;
-        default:
-          throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
-      }
-    }
-  }
-
-  @Override
-  public void serializeImpl(ByteBuffer buffer) {
-    int type = PhysicalPlanType.INSERT.ordinal();
-    buffer.put((byte) type);
-    subSerialize(buffer);
-  }
-
-  void subSerialize(ByteBuffer buffer) {
-    buffer.putLong(time);
-    putString(buffer, devicePath.getFullPath());
-    serializeMeasurementsAndValues(buffer);
-  }
-
-  void serializeMeasurementsAndValues(ByteBuffer buffer) {
-    buffer.putInt(measurements.length - getFailedMeasurementNumber());
-
-    for (String measurement : measurements) {
-      if (measurement != null) {
-        putString(buffer, measurement);
-      }
-    }
-    try {
-      buffer.putInt(values.length - getFailedMeasurementNumber());
-      putValues(buffer);
-    } catch (QueryProcessException e) {
-      logger.error("Failed to serialize values for {}", this, e);
-    }
-
-    // the types are not inferred before the plan is serialized
-    buffer.put((byte) (isNeedInferType ? 1 : 0));
-    buffer.putLong(index);
-
-    buffer.put((byte) (isAligned ? 1 : 0));
-  }
-
-  private void putValues(ByteBuffer buffer) throws QueryProcessException {
-    for (int i = 0; i < values.length; i++) {
-      if (values[i] == null) {
-        continue;
-      }
-      // types are not determined, the situation mainly occurs when the plan uses string values
-      // and is forwarded to other nodes
-      if (dataTypes == null || dataTypes[i] == null) {
-        ReadWriteIOUtils.write(TYPE_RAW_STRING, buffer);
-        ReadWriteIOUtils.write(values[i].toString(), buffer);
-      } else {
-        ReadWriteIOUtils.write(dataTypes[i], buffer);
-        switch (dataTypes[i]) {
-          case BOOLEAN:
-            ReadWriteIOUtils.write((Boolean) values[i], buffer);
-            break;
-          case INT32:
-            ReadWriteIOUtils.write((Integer) values[i], buffer);
-            break;
-          case INT64:
-            ReadWriteIOUtils.write((Long) values[i], buffer);
-            break;
-          case FLOAT:
-            ReadWriteIOUtils.write((Float) values[i], buffer);
-            break;
-          case DOUBLE:
-            ReadWriteIOUtils.write((Double) values[i], buffer);
-            break;
-          case TEXT:
-            ReadWriteIOUtils.write((Binary) values[i], buffer);
-            break;
-          default:
-            throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
-        }
-      }
-    }
-  }
-
-  @Override
-  public void serializeToWAL(IWALByteBufferView buffer) {
-    int type = PhysicalPlanType.INSERT.ordinal();
-    buffer.put((byte) type);
-    subSerialize(buffer);
-  }
-
-  void subSerialize(IWALByteBufferView buffer) {
-    buffer.putLong(time);
-    WALWriteUtils.write(devicePath.getFullPath(), buffer);
-    serializeMeasurementsAndValues(buffer);
-  }
-
-  void serializeMeasurementsAndValues(IWALByteBufferView buffer) {
-    buffer.putInt(measurements.length - getFailedMeasurementNumber());
-
-    for (String measurement : measurements) {
-      if (measurement != null) {
-        WALWriteUtils.write(measurement, buffer);
-      }
-    }
-    try {
-      buffer.putInt(values.length - getFailedMeasurementNumber());
-      putValues(buffer);
-    } catch (QueryProcessException e) {
-      logger.error("Failed to serialize values for {}", this, e);
-    }
-
-    // the types are not inferred before the plan is serialized
-    buffer.put((byte) (isNeedInferType ? 1 : 0));
-    buffer.putLong(index);
-
-    buffer.put((byte) (isAligned ? 1 : 0));
-  }
-
-  private void putValues(IWALByteBufferView buffer) throws QueryProcessException {
-    for (int i = 0; i < values.length; i++) {
-      if (values[i] == null) {
-        continue;
-      }
-      // types are not determined, the situation mainly occurs when the plan uses string values
-      // and is forwarded to other nodes
-      if (dataTypes == null || dataTypes[i] == null) {
-        WALWriteUtils.write(TYPE_RAW_STRING, buffer);
-        WALWriteUtils.write(values[i].toString(), buffer);
-      } else {
-        WALWriteUtils.write(dataTypes[i], buffer);
-        switch (dataTypes[i]) {
-          case BOOLEAN:
-            WALWriteUtils.write((Boolean) values[i], buffer);
-            break;
-          case INT32:
-            WALWriteUtils.write((Integer) values[i], buffer);
-            break;
-          case INT64:
-            WALWriteUtils.write((Long) values[i], buffer);
-            break;
-          case FLOAT:
-            WALWriteUtils.write((Float) values[i], buffer);
-            break;
-          case DOUBLE:
-            WALWriteUtils.write((Double) values[i], buffer);
-            break;
-          case TEXT:
-            WALWriteUtils.write((Binary) values[i], buffer);
-            break;
-          default:
-            throw new QueryProcessException("Unsupported data type:" + dataTypes[i]);
-        }
-      }
-    }
-  }
-
-  @Override
-  public void deserialize(DataInputStream stream) throws IOException, IllegalPathException {
-    this.time = stream.readLong();
-    this.devicePath = new PartialPath(ReadWriteIOUtils.readString(stream));
-    deserializeMeasurementsAndValues(stream);
-  }
-
-  void deserializeMeasurementsAndValues(DataInputStream stream) throws IOException {
-    int measurementSize = stream.readInt();
-
-    this.measurements = new String[measurementSize];
-    for (int i = 0; i < measurementSize; i++) {
-      measurements[i] = ReadWriteIOUtils.readString(stream);
-    }
-
-    int dataTypeSize = stream.readInt();
-    this.dataTypes = new TSDataType[dataTypeSize];
-    this.values = new Object[dataTypeSize];
-    try {
-      fillValues(stream);
-    } catch (QueryProcessException e) {
-      e.printStackTrace();
-    }
-
-    isNeedInferType = stream.readByte() == 1;
-    this.index = stream.readLong();
-    isAligned = stream.readByte() == 1;
-  }
-
-  @Override
-  public void deserialize(ByteBuffer buffer) throws IllegalPathException {
-    this.time = buffer.getLong();
-    this.devicePath = new PartialPath(readString(buffer));
-    deserializeMeasurementsAndValues(buffer);
-  }
-
-  void deserializeMeasurementsAndValues(ByteBuffer buffer) {
-    int measurementSize = buffer.getInt();
-
-    this.measurements = new String[measurementSize];
-    for (int i = 0; i < measurementSize; i++) {
-      measurements[i] = readString(buffer);
-    }
-
-    int dataTypeSize = buffer.getInt();
-    this.dataTypes = new TSDataType[dataTypeSize];
-    this.values = new Object[dataTypeSize];
-    try {
-      fillValues(buffer);
-    } catch (QueryProcessException e) {
-      e.printStackTrace();
-    }
-
-    isNeedInferType = buffer.get() == 1;
-    this.index = buffer.getLong();
-    isAligned = buffer.get() == 1;
-  }
-
-  @Override
-  public String toString() {
-    return "prefixPath: "
-        + devicePath
-        + ", time: "
-        + time
-        + ", measurements: "
-        + Arrays.toString(measurements)
-        + ", values: "
-        + Arrays.toString(values);
-  }
-
-  boolean hasFailedValues() {
-    return failedValues != null && !failedValues.isEmpty();
-  }
-
-  public TimeValuePair composeTimeValuePair(int columnIndex) {
-    if (columnIndex >= values.length) {
-      return null;
-    }
-    Object value = values[columnIndex];
-    return new TimeValuePair(time, TsPrimitiveType.getByType(dataTypes[columnIndex], value));
-  }
-
-  @Override
-  public InsertPlan getPlanFromFailed() {
-    if (super.getPlanFromFailed() == null) {
-      return null;
-    }
-    values = failedValues.toArray(new Object[0]);
-    failedValues = null;
-    return this;
-  }
-
-  @Override
-  public void recoverFromFailure() {
-    if (failedMeasurements == null) {
-      return;
-    }
-
-    for (int i = 0; i < failedMeasurements.size(); i++) {
-      int index = failedIndices.get(i);
-      values[index] = failedValues.get(i);
-    }
-    super.recoverFromFailure();
-
-    failedValues = null;
-  }
-
-  @Override
-  public void checkIntegrity() throws QueryProcessException {
-    super.checkIntegrity();
-    if (values == null) {
-      throw new QueryProcessException("Values are null");
-    }
-    if (values.length == 0) {
-      throw new QueryProcessException("The size of values is 0");
-    }
-    for (Object value : values) {
-      if (value == null) {
-        throw new QueryProcessException("Values contain null: " + Arrays.toString(values));
-      }
-    }
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowsOfOneDevicePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowsOfOneDevicePlan.java
deleted file mode 100644
index 09bb299f54..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowsOfOneDevicePlan.java
+++ /dev/null
@@ -1,296 +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.crud;
-
-import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-import org.apache.iotdb.db.qp.physical.BatchPlan;
-import org.apache.iotdb.tsfile.exception.NotImplementedException;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-public class InsertRowsOfOneDevicePlan extends InsertPlan implements BatchPlan {
-
-  /**
-   * This class has some duplicated codes with InsertRowsPlan, they should be refined in the future
-   */
-  boolean[] isExecuted;
-
-  private InsertRowPlan[] rowPlans;
-
-  /**
-   * Suppose there is an InsertRowsOfOneDevicePlan, which contains 5 InsertRowPlans,
-   * rowPlans={InsertRowPlan_0, InsertRowPlan_1, InsertRowPlan_2, InsertRowPlan_3, InsertRowPlan_4},
-   * then the rowPlanIndexList={0, 1, 2, 3, 4} respectively. But when the InsertRowsOfOneDevicePlan
-   * is split into two InsertRowsOfOneDevicePlans according to the time partition in cluster
-   * version, suppose that the InsertRowsOfOneDevicePlan_1's rowPlanIndexList = {InsertRowPlan_0,
-   * InsertRowPlan_3, InsertRowPlan_4}, then InsertRowsOfOneDevicePlan_1's rowPlanIndexList = {0, 3,
-   * 4}; InsertRowsOfOneDevicePlan_2's rowPlanIndexList = {InsertRowPlan_1, InsertRowPlan_2} then
-   * InsertRowsOfOneDevicePlan_2's rowPlanIndexList = {1, 2} respectively;
-   */
-  private int[] rowPlanIndexList;
-
-  /** record the result of insert rows */
-  private Map<Integer, TSStatus> results = new HashMap<>();
-
-  private List<PartialPath> paths;
-
-  public InsertRowsOfOneDevicePlan() {
-    super(OperatorType.BATCH_INSERT_ONE_DEVICE);
-  }
-
-  public InsertRowsOfOneDevicePlan(
-      PartialPath prefixPath,
-      List<Long> insertTimes,
-      List<List<String>> measurements,
-      List<ByteBuffer> insertValues,
-      boolean isAligned)
-      throws QueryProcessException {
-    this();
-    this.devicePath = prefixPath;
-    rowPlans = new InsertRowPlan[insertTimes.size()];
-    rowPlanIndexList = new int[insertTimes.size()];
-    for (int i = 0; i < insertTimes.size(); i++) {
-      rowPlans[i] =
-          new InsertRowPlan(
-              prefixPath,
-              insertTimes.get(i),
-              measurements.get(i).toArray(new String[0]),
-              insertValues.get(i),
-              isAligned);
-      if (rowPlans[i].getMeasurements().length == 0) {
-        throw new QueryProcessException(
-            "The measurements are null, deviceId:" + prefixPath + ", time:" + insertTimes.get(i));
-      }
-      if (rowPlans[i].getValues().length == 0) {
-        throw new QueryProcessException(
-            "The size of values in InsertRowsOfOneDevicePlan is 0, deviceId:"
-                + prefixPath
-                + ", time:"
-                + insertTimes.get(i));
-      }
-      rowPlanIndexList[i] = i;
-    }
-  }
-
-  /**
-   * This constructor is used for splitting parent InsertRowsOfOneDevicePlan into sub ones. So
-   * there's no need to validate rowPlans.
-   */
-  public InsertRowsOfOneDevicePlan(
-      PartialPath prefixPath, InsertRowPlan[] rowPlans, int[] rowPlanIndexList) {
-    this();
-    this.devicePath = prefixPath;
-    this.rowPlans = rowPlans;
-    this.rowPlanIndexList = rowPlanIndexList;
-  }
-
-  @Override
-  public void checkIntegrity() {}
-
-  // TODO I see InsertRowPlan rewrites the hashCode, but do we need to rewrite hashCode?
-
-  @Override
-  public List<PartialPath> getPaths() {
-    if (paths != null) {
-      return paths;
-    }
-    Set<PartialPath> pathSet = new HashSet<>();
-    for (InsertRowPlan plan : rowPlans) {
-      pathSet.addAll(plan.getPaths());
-    }
-    paths = new ArrayList<>(pathSet);
-    return paths;
-  }
-
-  @Override
-  public long getMinTime() {
-    long minTime = Long.MAX_VALUE;
-    for (InsertRowPlan plan : rowPlans) {
-      if (minTime > plan.getTime()) {
-        minTime = plan.getTime();
-      }
-    }
-    return minTime;
-  }
-
-  @Override
-  public Object getFirstValueOfIndex(int index) {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public void serialize(DataOutputStream stream) throws IOException {
-    int type = PhysicalPlanType.BATCH_INSERT_ONE_DEVICE.ordinal();
-    stream.writeByte((byte) type);
-    putString(stream, devicePath.getFullPath());
-
-    stream.writeInt(rowPlans.length);
-    for (InsertRowPlan plan : rowPlans) {
-      stream.writeLong(plan.getTime());
-      plan.serializeMeasurementsAndValues(stream);
-    }
-    for (Integer index : rowPlanIndexList) {
-      stream.writeInt(index);
-    }
-  }
-
-  @Override
-  public void serializeImpl(ByteBuffer buffer) {
-    int type = PhysicalPlanType.BATCH_INSERT_ONE_DEVICE.ordinal();
-    buffer.put((byte) type);
-
-    putString(buffer, devicePath.getFullPath());
-    buffer.putInt(rowPlans.length);
-    for (InsertRowPlan plan : rowPlans) {
-      buffer.putLong(plan.getTime());
-      plan.serializeMeasurementsAndValues(buffer);
-    }
-    for (Integer index : rowPlanIndexList) {
-      buffer.putInt(index);
-    }
-  }
-
-  @Override
-  public void deserialize(ByteBuffer buffer) throws IllegalPathException {
-    this.devicePath = new PartialPath(readString(buffer));
-    this.rowPlans = new InsertRowPlan[buffer.getInt()];
-    for (int i = 0; i < rowPlans.length; i++) {
-      rowPlans[i] = new InsertRowPlan();
-      rowPlans[i].setDevicePath(devicePath);
-      rowPlans[i].setTime(buffer.getLong());
-      rowPlans[i].deserializeMeasurementsAndValues(buffer);
-    }
-    this.rowPlanIndexList = new int[rowPlans.length];
-    for (int i = 0; i < rowPlans.length; i++) {
-      rowPlanIndexList[i] = buffer.getInt();
-    }
-  }
-
-  @Override
-  public void setIndex(long index) {
-    super.setIndex(index);
-    for (InsertRowPlan plan : rowPlans) {
-      // use the InsertRowsOfOneDevicePlan's index as the sub InsertRowPlan's index
-      plan.setIndex(index);
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "deviceId: " + devicePath + ", times: " + rowPlans.length;
-  }
-
-  @Override
-  public InsertPlan getPlanFromFailed() {
-    if (super.getPlanFromFailed() == null) {
-      return null;
-    }
-    List<InsertRowPlan> plans = new ArrayList<>();
-    for (InsertRowPlan plan : rowPlans) {
-      if (plan.hasFailedValues()) {
-        plans.add((InsertRowPlan) plan.getPlanFromFailed());
-      }
-    }
-    this.rowPlans = plans.toArray(new InsertRowPlan[0]);
-    return this;
-  }
-
-  public InsertRowPlan[] getRowPlans() {
-    return rowPlans;
-  }
-
-  @Override
-  public void setIsExecuted(int i) {
-    if (isExecuted == null) {
-      isExecuted = new boolean[getBatchSize()];
-    }
-    isExecuted[i] = true;
-  }
-
-  @Override
-  public boolean isExecuted(int i) {
-    if (isExecuted == null) {
-      isExecuted = new boolean[getBatchSize()];
-    }
-    return isExecuted[i];
-  }
-
-  @Override
-  public Map<Integer, TSStatus> getResults() {
-    return results;
-  }
-
-  @Override
-  public List<PartialPath> getPrefixPaths() {
-    return Collections.singletonList(this.devicePath);
-  }
-
-  @Override
-  public int getBatchSize() {
-    return rowPlans.length;
-  }
-
-  public int[] getRowPlanIndexList() {
-    return rowPlanIndexList;
-  }
-
-  @Override
-  public void unsetIsExecuted(int i) {
-    if (isExecuted == null) {
-      isExecuted = new boolean[getBatchSize()];
-    }
-    isExecuted[i] = false;
-    if (rowPlanIndexList != null && rowPlanIndexList.length > 0) {
-      results.remove(rowPlanIndexList[i]);
-    } else {
-      results.remove(i);
-    }
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    return o instanceof InsertRowsOfOneDevicePlan
-        && Arrays.equals(((InsertRowsOfOneDevicePlan) o).rowPlanIndexList, this.rowPlanIndexList)
-        && Arrays.equals(((InsertRowsOfOneDevicePlan) o).rowPlans, this.rowPlans)
-        && ((InsertRowsOfOneDevicePlan) o).results.equals(this.results)
-        && ((InsertRowsOfOneDevicePlan) o).getDevicePath().equals(this.getDevicePath());
-  }
-
-  @Override
-  public int hashCode() {
-    int result = rowPlans != null ? Arrays.hashCode(rowPlans) : 0;
-    result = 31 * result + (rowPlanIndexList != null ? Arrays.hashCode(rowPlanIndexList) : 0);
-    result = 31 * result + (results != null ? results.hashCode() : 0);
-    return result;
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowsPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowsPlan.java
deleted file mode 100644
index 9faf111599..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertRowsPlan.java
+++ /dev/null
@@ -1,302 +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.crud;
-
-import org.apache.iotdb.common.rpc.thrift.TSStatus;
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.commons.utils.StatusUtils;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-import org.apache.iotdb.db.qp.physical.BatchPlan;
-import org.apache.iotdb.tsfile.exception.NotImplementedException;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-
-public class InsertRowsPlan extends InsertPlan implements BatchPlan {
-
-  /**
-   * Suppose there is an InsertRowsPlan, which contains 5 InsertRowPlans,
-   * insertRowPlanList={InsertRowPlan_0, InsertRowPlan_1, InsertRowPlan_2, InsertRowPlan_3,
-   * InsertRowPlan_4}, then the insertRowPlanIndexList={0, 1, 2, 3, 4} respectively. But when the
-   * InsertRowsPlan is split into two InsertRowsPlans according to different storage group in
-   * cluster version, suppose that the InsertRowsPlan_1's insertRowPlanList = {InsertRowPlan_0,
-   * InsertRowPlan_3, InsertRowPlan_4}, then InsertRowsPlan_1's insertRowPlanIndexList = {0, 3, 4};
-   * InsertRowsPlan_2's insertRowPlanList = {InsertRowPlan_1, * InsertRowPlan_2} then
-   * InsertRowsPlan_2's insertRowPlanIndexList= {1, 2} respectively;
-   */
-  private List<Integer> insertRowPlanIndexList;
-
-  /** the InsertRowsPlan list */
-  private List<InsertRowPlan> insertRowPlanList;
-
-  boolean[] isExecuted;
-
-  /** record the result of insert rows */
-  private Map<Integer, TSStatus> results = new HashMap<>();
-
-  private List<PartialPath> paths;
-  private List<PartialPath> prefixPaths;
-
-  public InsertRowsPlan() {
-    super(OperatorType.BATCH_INSERT_ROWS);
-    insertRowPlanList = new ArrayList<>();
-    insertRowPlanIndexList = new ArrayList<>();
-  }
-
-  @Override
-  public long getMinTime() {
-    long minTime = Long.MAX_VALUE;
-    for (InsertRowPlan insertRowPlan : insertRowPlanList) {
-      if (insertRowPlan.getMinTime() < minTime) {
-        minTime = insertRowPlan.getMinTime();
-      }
-    }
-    return minTime;
-  }
-
-  @Override
-  public Object getFirstValueOfIndex(int index) {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public List<PartialPath> getPaths() {
-    if (paths != null) {
-      return paths;
-    }
-    Set<PartialPath> pathSet = new HashSet<>();
-    for (InsertRowPlan plan : insertRowPlanList) {
-      pathSet.addAll(plan.getPaths());
-    }
-    paths = new ArrayList<>(pathSet);
-    return paths;
-  }
-
-  @Override
-  public List<PartialPath> getPrefixPaths() {
-    if (prefixPaths != null) {
-      return prefixPaths;
-    }
-    prefixPaths = new ArrayList<>(insertRowPlanList.size());
-    for (InsertRowPlan insertRowPlan : insertRowPlanList) {
-      prefixPaths.add(insertRowPlan.getDevicePath());
-    }
-    return prefixPaths;
-  }
-
-  @Override
-  public void checkIntegrity() throws QueryProcessException {
-    if (insertRowPlanList.isEmpty()) {
-      throw new QueryProcessException("sub plan are empty.");
-    }
-    for (InsertRowPlan insertRowPlan : insertRowPlanList) {
-      insertRowPlan.checkIntegrity();
-    }
-  }
-
-  @Override
-  public void recoverFromFailure() {
-    for (InsertRowPlan insertRowPlan : insertRowPlanList) {
-      insertRowPlan.recoverFromFailure();
-    }
-  }
-
-  @Override
-  public InsertPlan getPlanFromFailed() {
-    if (super.getPlanFromFailed() == null) {
-      return null;
-    }
-    List<InsertRowPlan> plans = new ArrayList<>();
-    List<Integer> indexes = new ArrayList<>();
-    for (int i = 0; i < insertRowPlanList.size(); i++) {
-      if (insertRowPlanList.get(i).hasFailedValues()) {
-        plans.add((InsertRowPlan) insertRowPlanList.get(i).getPlanFromFailed());
-        indexes.add(i);
-      }
-    }
-    this.insertRowPlanList = plans;
-    this.insertRowPlanIndexList = indexes;
-    return this;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-
-    InsertRowsPlan that = (InsertRowsPlan) o;
-
-    if (!Objects.equals(insertRowPlanIndexList, that.insertRowPlanIndexList)) {
-      return false;
-    }
-    if (!Objects.equals(insertRowPlanList, that.insertRowPlanList)) {
-      return false;
-    }
-    return Objects.equals(results, that.results);
-  }
-
-  @Override
-  public int hashCode() {
-    int result = insertRowPlanIndexList != null ? insertRowPlanIndexList.hashCode() : 0;
-    result = 31 * result + (insertRowPlanList != null ? insertRowPlanList.hashCode() : 0);
-    result = 31 * result + (results != null ? results.hashCode() : 0);
-    return result;
-  }
-
-  @Override
-  public void serializeImpl(ByteBuffer buffer) {
-    int type = PhysicalPlanType.BATCH_INSERT_ROWS.ordinal();
-    buffer.put((byte) type);
-    buffer.putInt(insertRowPlanList.size());
-    for (InsertRowPlan insertRowPlan : insertRowPlanList) {
-      insertRowPlan.subSerialize(buffer);
-    }
-    for (Integer index : insertRowPlanIndexList) {
-      buffer.putInt(index);
-    }
-  }
-
-  @Override
-  public void serialize(DataOutputStream stream) throws IOException {
-    int type = PhysicalPlanType.BATCH_INSERT_ROWS.ordinal();
-    stream.writeByte((byte) type);
-    stream.writeInt(insertRowPlanList.size());
-    for (InsertRowPlan insertRowPlan : insertRowPlanList) {
-      insertRowPlan.subSerialize(stream);
-    }
-    for (Integer index : insertRowPlanIndexList) {
-      stream.writeInt(index);
-    }
-  }
-
-  @Override
-  public void deserialize(ByteBuffer buffer) throws IllegalPathException {
-    int size = buffer.getInt();
-    this.insertRowPlanList = new ArrayList<>(size);
-    this.insertRowPlanIndexList = new ArrayList<>(size);
-    for (int i = 0; i < size; i++) {
-      InsertRowPlan insertRowPlan = new InsertRowPlan();
-      insertRowPlan.deserialize(buffer);
-      insertRowPlanList.add(insertRowPlan);
-    }
-
-    for (int i = 0; i < size; i++) {
-      insertRowPlanIndexList.add(buffer.getInt());
-    }
-  }
-
-  @Override
-  public void setIndex(long index) {
-    super.setIndex(index);
-    for (InsertRowPlan insertRowPlan : insertRowPlanList) {
-      // use the InsertRowsPlan's index as the sub InsertRowPlan's index
-      insertRowPlan.setIndex(index);
-    }
-  }
-
-  @Override
-  public Map<Integer, TSStatus> getResults() {
-    return results;
-  }
-
-  public void addOneInsertRowPlan(InsertRowPlan plan, int index) {
-    insertRowPlanList.add(plan);
-    insertRowPlanIndexList.add(index);
-  }
-
-  public List<Integer> getInsertRowPlanIndexList() {
-    return insertRowPlanIndexList;
-  }
-
-  public List<InsertRowPlan> getInsertRowPlanList() {
-    return insertRowPlanList;
-  }
-
-  public int getRowCount() {
-    return insertRowPlanList.size();
-  }
-
-  public PartialPath getFirstDeviceId() {
-    return insertRowPlanList.get(0).getDevicePath();
-  }
-
-  @Override
-  public String toString() {
-    return "InsertRowsPlan{"
-        + " insertRowPlanIndexList's size="
-        + insertRowPlanIndexList.size()
-        + ", insertRowPlanList's size="
-        + insertRowPlanList.size()
-        + ", results="
-        + results
-        + "}";
-  }
-
-  public TSStatus[] getFailingStatus() {
-    return StatusUtils.getFailingStatus(results, insertRowPlanList.size());
-  }
-
-  @Override
-  public void setIsExecuted(int i) {
-    if (isExecuted == null) {
-      isExecuted = new boolean[getBatchSize()];
-    }
-    isExecuted[i] = true;
-  }
-
-  @Override
-  public boolean isExecuted(int i) {
-    if (isExecuted == null) {
-      isExecuted = new boolean[getBatchSize()];
-    }
-    return isExecuted[i];
-  }
-
-  @Override
-  public int getBatchSize() {
-    return insertRowPlanList.size();
-  }
-
-  @Override
-  public void unsetIsExecuted(int i) {
-    if (isExecuted == null) {
-      isExecuted = new boolean[getBatchSize()];
-    }
-    isExecuted[i] = false;
-    if (insertRowPlanIndexList != null && !insertRowPlanIndexList.isEmpty()) {
-      results.remove(insertRowPlanIndexList.get(i));
-    } else {
-      results.remove(i);
-    }
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java
deleted file mode 100644
index 57e9995c66..0000000000
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertTabletPlan.java
+++ /dev/null
@@ -1,895 +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.crud;
-
-import org.apache.iotdb.commons.exception.IllegalPathException;
-import org.apache.iotdb.commons.path.PartialPath;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
-import org.apache.iotdb.db.utils.QueryDataSetUtils;
-import org.apache.iotdb.db.wal.buffer.IWALByteBufferView;
-import org.apache.iotdb.db.wal.buffer.WALEntryValue;
-import org.apache.iotdb.db.wal.utils.WALWriteUtils;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.read.TimeValuePair;
-import org.apache.iotdb.tsfile.utils.Binary;
-import org.apache.iotdb.tsfile.utils.BitMap;
-import org.apache.iotdb.tsfile.utils.BytesUtils;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsBinary;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsBoolean;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsDouble;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsFloat;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsInt;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsLong;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Objects;
-
-@SuppressWarnings("java:S1135") // ignore todos
-public class InsertTabletPlan extends InsertPlan implements WALEntryValue {
-  private static final String DATATYPE_UNSUPPORTED = "Data type %s is not supported.";
-
-  private long[] times; // times should be sorted. It is done in the session API.
-
-  private BitMap[] bitMaps;
-  private Object[] columns;
-  private int rowCount = 0;
-  private List<PartialPath> paths;
-  // when this plan is sub-plan split from another InsertTabletPlan, this indicates the original
-  // positions of values in
-  // this plan. For example, if the plan contains 5 timestamps, and range = [1,4,10,12], then it
-  // means that the first 3
-  // timestamps in this plan are from range[1,4) of the parent plan, and the last 2 timestamps are
-  // from range[10,12)
-  // of the parent plan.
-  // this is usually used to back-propagate exceptions to the parent plan without losing their
-  // proper positions.
-  private List<Integer> range;
-
-  private List<Object> failedColumns;
-
-  public InsertTabletPlan() {
-    super(OperatorType.BATCH_INSERT);
-  }
-
-  public InsertTabletPlan(PartialPath prefixPath, List<String> measurements) {
-    super(OperatorType.BATCH_INSERT);
-    this.devicePath = prefixPath;
-    this.measurements = measurements.toArray(new String[0]);
-    this.canBeSplit = true;
-  }
-
-  public InsertTabletPlan(PartialPath prefixPath, String[] measurements) {
-    super(OperatorType.BATCH_INSERT);
-    this.devicePath = prefixPath;
-    this.measurements = measurements;
-    this.canBeSplit = true;
-  }
-
-  public InsertTabletPlan(PartialPath prefixPath, String[] measurements, List<Integer> dataTypes) {
-    super(OperatorType.BATCH_INSERT);
-    this.devicePath = prefixPath;
-    this.measurements = measurements;
-    setDataTypes(dataTypes);
-    this.canBeSplit = true;
-  }
-
-  public InsertTabletPlan(
-      PartialPath prefixPath, String[] measurements, List<Integer> dataTypes, boolean isAligned) {
-    super(OperatorType.BATCH_INSERT);
-    this.devicePath = prefixPath;
-    this.measurements = measurements;
-    setDataTypes(dataTypes);
-    this.canBeSplit = true;
-    this.isAligned = isAligned;
-  }
-
-  public List<Integer> getRange() {
-    return range;
-  }
-
-  public void setRange(List<Integer> range) {
-    this.range = range;
-  }
-
-  @Override
-  public List<PartialPath> getPaths() {
-    if (paths != null) {
-      return paths;
-    }
-    List<PartialPath> ret = new ArrayList<>();
-    for (String m : measurements) {
-      PartialPath fullPath = devicePath.concatNode(m);
-      ret.add(fullPath);
-    }
-    paths = ret;
-    return ret;
-  }
-
-  @Override
-  public int serializedSize() {
-    int size = Byte.BYTES;
-    return size + subSerializeSize(0, rowCount);
-  }
-
-  public int serializedSize(int start, int end) {
-    int size = Byte.BYTES;
-    return size + subSerializeSize(start, end);
-  }
-
-  int subSerializeSize(int start, int end) {
-    int size = 0;
-    size += ReadWriteIOUtils.sizeToWrite(devicePath.getFullPath());
-    // measurements size
-    size += Integer.BYTES;
-    for (String m : measurements) {
-      if (m != null) {
-        size += ReadWriteIOUtils.sizeToWrite(m);
-      }
-    }
-    // data types size
-    size += Integer.BYTES;
-    for (int i = 0; i < dataTypes.length; i++) {
-      if (columns[i] != null) {
-        size += Byte.BYTES;
-      }
-    }
-    // times size
-    size += Integer.BYTES;
-    size += Long.BYTES * (end - start);
-    // bitmaps size
-    size += Byte.BYTES;
-    if (bitMaps != null) {
-      for (int i = 0; i < bitMaps.length; ++i) {
-        if (columns[i] == null) {
-          continue;
-        }
-
-        size += Byte.BYTES;
-        if (bitMaps[i] != null) {
-          int len = end - start;
-          BitMap partBitMap = new BitMap(len);
-          BitMap.copyOfRange(bitMaps[i], start, partBitMap, 0, len);
-          size += partBitMap.getByteArray().length;
-        }
-      }
-    }
-    // values size
-    for (int i = 0; i < dataTypes.length; i++) {
-      if (columns[i] != null) {
-        size += getColumnSize(dataTypes[i], columns[i], start, end);
-      }
-    }
-    size += Long.BYTES;
-    size += Byte.BYTES;
-    return size;
-  }
-
-  private int getColumnSize(TSDataType dataType, Object column, int start, int end) {
-    int size = 0;
-    switch (dataType) {
-      case INT32:
-        size += Integer.BYTES * (end - start);
-        break;
-      case INT64:
-        size += Long.BYTES * (end - start);
-        break;
-      case FLOAT:
-        size += Float.BYTES * (end - start);
-        break;
-      case DOUBLE:
-        size += Double.BYTES * (end - start);
-        break;
-      case BOOLEAN:
-        size += Byte.BYTES * (end - start);
-        break;
-      case TEXT:
-        Binary[] binaryValues = (Binary[]) column;
-        for (int j = start; j < end; j++) {
-          size += ReadWriteIOUtils.sizeToWrite(binaryValues[j]);
-        }
-        break;
-    }
-    return size;
-  }
-
-  @Override
-  public void serialize(DataOutputStream stream) throws IOException {
-    int type = PhysicalPlanType.BATCHINSERT.ordinal();
-    stream.writeByte((byte) type);
-    subSerialize(stream, 0, rowCount);
-  }
-
-  void subSerialize(DataOutputStream stream, int start, int end) throws IOException {
-    putString(stream, devicePath.getFullPath());
-    writeMeasurements(stream);
-    writeDataTypes(stream);
-    writeTimes(stream, start, end);
-    writeBitMaps(stream, start, end);
-    writeValues(stream, start, end);
-    stream.write((byte) (isAligned ? 1 : 0));
-  }
-
-  private void writeMeasurements(DataOutputStream stream) throws IOException {
-    stream.writeInt(measurements.length - getFailedMeasurementNumber());
-    for (String m : measurements) {
-      if (m == null) {
-        continue;
-      }
-      putString(stream, m);
-    }
-  }
-
-  private void writeDataTypes(DataOutputStream stream) throws IOException {
-    stream.writeInt(dataTypes.length - getFailedMeasurementNumber());
-    for (int i = 0; i < dataTypes.length; i++) {
-      if (columns[i] == null) {
-        continue;
-      }
-      dataTypes[i].serializeTo(stream);
-    }
-  }
-
-  private void writeTimes(DataOutputStream stream, int start, int end) throws IOException {
-    stream.writeInt(end - start);
-    for (int i = start; i < end; i++) {
-      stream.writeLong(times[i]);
-    }
-  }
-
-  private void writeBitMaps(DataOutputStream stream, int start, int end) throws IOException {
-    stream.writeBoolean(bitMaps != null);
-    if (bitMaps != null) {
-      for (int i = 0; i < bitMaps.length; ++i) {
-        if (columns[i] == null) {
-          continue;
-        }
-
-        if (bitMaps[i] == null) {
-          stream.writeBoolean(false);
-        } else {
-          stream.writeBoolean(true);
-          int len = end - start;
-          BitMap partBitMap = new BitMap(len);
-          BitMap.copyOfRange(bitMaps[i], start, partBitMap, 0, len);
-          stream.write(partBitMap.getByteArray());
-        }
-      }
-    }
-  }
-
-  private void writeValues(DataOutputStream stream, int start, int end) throws IOException {
-    serializeValues(stream, start, end);
-    stream.writeLong(index);
-  }
-
-  private void serializeValues(DataOutputStream outputStream, int start, int end)
-      throws IOException {
-    for (int i = 0; i < columns.length; i++) {
-      if (columns[i] == null) {
-        continue;
-      }
-      serializeColumn(dataTypes[i], columns[i], outputStream, start, end);
-    }
-  }
-
-  private void serializeColumn(
-      TSDataType dataType, Object column, DataOutputStream outputStream, int start, int end)
-      throws IOException {
-    switch (dataType) {
-      case INT32:
-        int[] intValues = (int[]) column;
-        for (int j = start; j < end; j++) {
-          outputStream.writeInt(intValues[j]);
-        }
-        break;
-      case INT64:
-        long[] longValues = (long[]) column;
-        for (int j = start; j < end; j++) {
-          outputStream.writeLong(longValues[j]);
-        }
-        break;
-      case FLOAT:
-        float[] floatValues = (float[]) column;
-        for (int j = start; j < end; j++) {
-          outputStream.writeFloat(floatValues[j]);
-        }
-        break;
-      case DOUBLE:
-        double[] doubleValues = (double[]) column;
-        for (int j = start; j < end; j++) {
-          outputStream.writeDouble(doubleValues[j]);
-        }
-        break;
-      case BOOLEAN:
-        boolean[] boolValues = (boolean[]) column;
-        for (int j = start; j < end; j++) {
-          outputStream.writeByte(BytesUtils.boolToByte(boolValues[j]));
-        }
-        break;
-      case TEXT:
-        Binary[] binaryValues = (Binary[]) column;
-        for (int j = start; j < end; j++) {
-          outputStream.writeInt(binaryValues[j].getLength());
-          outputStream.write(binaryValues[j].getValues());
-        }
-        break;
-      default:
-        throw new UnSupportedDataTypeException(String.format(DATATYPE_UNSUPPORTED, dataType));
-    }
-  }
-
-  @Override
-  public void serializeImpl(ByteBuffer buffer) {
-    int type = PhysicalPlanType.BATCHINSERT.ordinal();
-    buffer.put((byte) type);
-    subSerialize(buffer, 0, rowCount);
-  }
-
-  void subSerialize(ByteBuffer buffer, int start, int end) {
-    putString(buffer, devicePath.getFullPath());
-    writeMeasurements(buffer);
-    writeDataTypes(buffer);
-    writeTimes(buffer, 0, rowCount);
-    writeBitMaps(buffer, 0, rowCount);
-    writeValues(buffer, 0, rowCount);
-    buffer.put((byte) (isAligned ? 1 : 0));
-  }
-
-  private void writeMeasurements(ByteBuffer buffer) {
-    buffer.putInt(measurements.length - getFailedMeasurementNumber());
-    for (String m : measurements) {
-      if (m != null) {
-        putString(buffer, m);
-      }
-    }
-  }
-
-  private void writeDataTypes(ByteBuffer buffer) {
-    buffer.putInt(dataTypes.length - getFailedMeasurementNumber());
-    for (int i = 0; i < dataTypes.length; i++) {
-      if (columns[i] == null) {
-        continue;
-      }
-      dataTypes[i].serializeTo(buffer);
-    }
-  }
-
-  private void writeTimes(ByteBuffer buffer, int start, int end) {
-    buffer.putInt(rowCount);
-    for (int i = start; i < end; i++) {
-      buffer.putLong(times[i]);
-    }
-  }
-
-  private void writeBitMaps(ByteBuffer buffer, int start, int end) {
-    buffer.put(BytesUtils.boolToByte(bitMaps != null));
-    if (bitMaps != null) {
-      for (int i = 0; i < bitMaps.length; i++) {
-        if (columns[i] == null) {
-          continue;
-        }
-
-        if (bitMaps[i] == null) {
-          buffer.put(BytesUtils.boolToByte(false));
-        } else {
-          buffer.put(BytesUtils.boolToByte(true));
-          int len = end - start;
-          BitMap partBitMap = new BitMap(len);
-          BitMap.copyOfRange(bitMaps[i], start, partBitMap, 0, len);
-          buffer.put(partBitMap.getByteArray());
-        }
-      }
-    }
-  }
-
-  private void writeValues(ByteBuffer buffer, int start, int end) {
-    serializeValues(buffer, start, end);
-    buffer.putLong(index);
-  }
-
-  private void serializeValues(ByteBuffer buffer, int start, int end) {
-    for (int i = 0; i < columns.length; i++) {
-      if (columns[i] == null) {
-        continue;
-      }
-      serializeColumn(dataTypes[i], columns[i], buffer, start, end);
-    }
-  }
-
-  private void serializeColumn(
-      TSDataType dataType, Object column, ByteBuffer buffer, int start, int end) {
-    switch (dataType) {
-      case INT32:
-        int[] intValues = (int[]) column;
-        for (int j = start; j < end; j++) {
-          buffer.putInt(intValues[j]);
-        }
-        break;
-      case INT64:
-        long[] longValues = (long[]) column;
-        for (int j = start; j < end; j++) {
-          buffer.putLong(longValues[j]);
-        }
-        break;
-      case FLOAT:
-        float[] floatValues = (float[]) column;
-        for (int j = start; j < end; j++) {
-          buffer.putFloat(floatValues[j]);
-        }
-        break;
-      case DOUBLE:
-        double[] doubleValues = (double[]) column;
-        for (int j = start; j < end; j++) {
-          buffer.putDouble(doubleValues[j]);
-        }
-        break;
-      case BOOLEAN:
-        boolean[] boolValues = (boolean[]) column;
-        for (int j = start; j < end; j++) {
-          buffer.put(BytesUtils.boolToByte(boolValues[j]));
-        }
-        break;
-      case TEXT:
-        Binary[] binaryValues = (Binary[]) column;
-        for (int j = start; j < end; j++) {
-          buffer.putInt(binaryValues[j].getLength());
-          buffer.put(binaryValues[j].getValues());
-        }
-        break;
-      default:
-        throw new UnSupportedDataTypeException(String.format(DATATYPE_UNSUPPORTED, dataType));
-    }
-  }
-
-  @Override
-  public void serializeToWAL(IWALByteBufferView buffer) {
-    serializeToWAL(buffer, 0, rowCount);
-  }
-
-  public void serializeToWAL(IWALByteBufferView buffer, int start, int end) {
-    int type = PhysicalPlanType.BATCHINSERT.ordinal();
-    buffer.put((byte) type);
-    subSerialize(buffer, start, end);
-  }
-
-  void subSerialize(IWALByteBufferView buffer, int start, int end) {
-    WALWriteUtils.write(devicePath.getFullPath(), buffer);
-    writeMeasurements(buffer);
-    writeDataTypes(buffer);
-    writeTimes(buffer, start, end);
-    writeBitMaps(buffer, start, end);
-    writeValues(buffer, start, end);
-    buffer.put((byte) (isAligned ? 1 : 0));
-  }
-
-  private void writeMeasurements(IWALByteBufferView buffer) {
-    buffer.putInt(measurements.length - getFailedMeasurementNumber());
-    for (String m : measurements) {
-      if (m != null) {
-        WALWriteUtils.write(m, buffer);
-      }
-    }
-  }
-
-  private void writeDataTypes(IWALByteBufferView buffer) {
-    buffer.putInt(dataTypes.length - getFailedMeasurementNumber());
-    for (int i = 0, dataTypesLength = dataTypes.length; i < dataTypesLength; i++) {
-      TSDataType dataType = dataTypes[i];
-      if (columns[i] == null) {
-        continue;
-      }
-      WALWriteUtils.write(dataType, buffer);
-    }
-  }
-
-  private void writeTimes(IWALByteBufferView buffer, int start, int end) {
-    buffer.putInt(end - start);
-    for (int i = start; i < end; i++) {
-      buffer.putLong(times[i]);
-    }
-  }
-
-  private void writeBitMaps(IWALByteBufferView buffer, int start, int end) {
-    buffer.put(BytesUtils.boolToByte(bitMaps != null));
-    if (bitMaps != null) {
-      for (int i = 0; i < bitMaps.length; i++) {
-        if (columns[i] == null) {
-          continue;
-        }
-
-        if (bitMaps[i] == null) {
-          buffer.put(BytesUtils.boolToByte(false));
-        } else {
-          buffer.put(BytesUtils.boolToByte(true));
-          int len = end - start;
-          BitMap partBitMap = new BitMap(len);
-          BitMap.copyOfRange(bitMaps[i], start, partBitMap, 0, len);
-          buffer.put(partBitMap.getByteArray());
-        }
-      }
-    }
-  }
-
-  private void writeValues(IWALByteBufferView buffer, int start, int end) {
-    serializeValues(buffer, start, end);
-    buffer.putLong(index);
-  }
-
-  private void serializeValues(IWALByteBufferView buffer, int start, int end) {
-    for (int i = 0; i < columns.length; i++) {
-      if (columns[i] == null) {
-        continue;
-      }
-      serializeColumn(dataTypes[i], columns[i], buffer, start, end);
-    }
-  }
-
-  private void serializeColumn(
-      TSDataType dataType, Object column, IWALByteBufferView buffer, int start, int end) {
-    switch (dataType) {
-      case INT32:
-        int[] intValues = (int[]) column;
-        for (int j = start; j < end; j++) {
-          buffer.putInt(intValues[j]);
-        }
-        break;
-      case INT64:
-        long[] longValues = (long[]) column;
-        for (int j = start; j < end; j++) {
-          buffer.putLong(longValues[j]);
-        }
-        break;
-      case FLOAT:
-        float[] floatValues = (float[]) column;
-        for (int j = start; j < end; j++) {
-          buffer.putFloat(floatValues[j]);
-        }
-        break;
-      case DOUBLE:
-        double[] doubleValues = (double[]) column;
-        for (int j = start; j < end; j++) {
-          buffer.putDouble(doubleValues[j]);
-        }
-        break;
-      case BOOLEAN:
-        boolean[] boolValues = (boolean[]) column;
-        for (int j = start; j < end; j++) {
-          buffer.put(BytesUtils.boolToByte(boolValues[j]));
-        }
-        break;
-      case TEXT:
-        Binary[] binaryValues = (Binary[]) column;
-        for (int j = start; j < end; j++) {
-          buffer.putInt(binaryValues[j].getLength());
-          buffer.put(binaryValues[j].getValues());
-        }
-        break;
-      default:
-        throw new UnSupportedDataTypeException(String.format(DATATYPE_UNSUPPORTED, dataType));
-    }
-  }
-
-  @Override
-  public void deserialize(DataInputStream stream) throws IOException, IllegalPathException {
-    this.devicePath = new PartialPath(ReadWriteIOUtils.readString(stream));
-
-    int measurementSize = stream.readInt();
-    this.measurements = new String[measurementSize];
-    for (int i = 0; i < measurementSize; i++) {
-      measurements[i] = ReadWriteIOUtils.readString(stream);
-    }
-
-    int dataTypeSize = stream.readInt();
-    this.dataTypes = new TSDataType[dataTypeSize];
-    for (int i = 0; i < dataTypeSize; i++) {
-      dataTypes[i] = TSDataType.deserialize(stream.readByte());
-    }
-
-    int rows = stream.readInt();
-    rowCount = rows;
-    this.times = new long[rows];
-    times = QueryDataSetUtils.readTimesFromStream(stream, rows);
-
-    boolean hasBitMaps = BytesUtils.byteToBool(stream.readByte());
-    if (hasBitMaps) {
-      bitMaps = QueryDataSetUtils.readBitMapsFromStream(stream, dataTypeSize, rows);
-    }
-    columns = QueryDataSetUtils.readTabletValuesFromStream(stream, dataTypes, dataTypeSize, rows);
-    this.index = stream.readLong();
-    this.isAligned = stream.readByte() == 1;
-  }
-
-  @Override
-  public void deserialize(ByteBuffer buffer) throws IllegalPathException {
-    this.devicePath = new PartialPath(readString(buffer));
-
-    int measurementSize = buffer.getInt();
-    this.measurements = new String[measurementSize];
-    for (int i = 0; i < measurementSize; i++) {
-      measurements[i] = readString(buffer);
-    }
-
-    int dataTypeSize = buffer.getInt();
-    this.dataTypes = new TSDataType[dataTypeSize];
-    for (int i = 0; i < dataTypeSize; i++) {
-      dataTypes[i] = TSDataType.deserialize(buffer.get());
-    }
-
-    int rows = buffer.getInt();
-    rowCount = rows;
-    this.times = new long[rows];
-    times = QueryDataSetUtils.readTimesFromBuffer(buffer, rows);
-
-    boolean hasBitMaps = BytesUtils.byteToBool(buffer.get());
-    if (hasBitMaps) {
-      bitMaps = QueryDataSetUtils.readBitMapsFromBuffer(buffer, dataTypeSize, rows);
-    }
-    columns = QueryDataSetUtils.readTabletValuesFromBuffer(buffer, dataTypes, dataTypeSize, rows);
-    this.index = buffer.getLong();
-    this.isAligned = buffer.get() == 1;
-  }
-
-  public void setDataTypes(List<Integer> dataTypes) {
-    this.dataTypes = new TSDataType[dataTypes.size()];
-    for (int i = 0; i < dataTypes.size(); i++) {
-      this.dataTypes[i] = TSDataType.values()[dataTypes.get(i)];
-    }
-  }
-
-  public Object[] getColumns() {
-    return columns;
-  }
-
-  public void setColumns(Object[] columns) {
-    this.columns = columns;
-  }
-
-  public void setColumn(int index, Object column) {
-    columns[index] = column;
-  }
-
-  public BitMap[] getBitMaps() {
-    return bitMaps;
-  }
-
-  public void setBitMaps(BitMap[] bitMaps) {
-    this.bitMaps = bitMaps;
-  }
-
-  @Override
-  public long getMinTime() {
-    return times.length != 0 ? times[0] : Long.MIN_VALUE;
-  }
-
-  public long getMaxTime() {
-    return times.length != 0 ? times[times.length - 1] : Long.MAX_VALUE;
-  }
-
-  @Override
-  public Object getFirstValueOfIndex(int index) {
-    Object value;
-    switch (dataTypes[index]) {
-      case INT32:
-        int[] intValues = (int[]) columns[index];
-        value = intValues[0];
-        break;
-      case INT64:
-        long[] longValues = (long[]) columns[index];
-        value = longValues[0];
-        break;
-      case FLOAT:
-        float[] floatValues = (float[]) columns[index];
-        value = floatValues[0];
-        break;
-      case DOUBLE:
-        double[] doubleValues = (double[]) columns[index];
-        value = doubleValues[0];
-        break;
-      case BOOLEAN:
-        boolean[] boolValues = (boolean[]) columns[index];
-        value = boolValues[0];
-        break;
-      case TEXT:
-        Binary[] binaryValues = (Binary[]) columns[index];
-        value = binaryValues[0];
-        break;
-      default:
-        throw new UnSupportedDataTypeException(
-            String.format(DATATYPE_UNSUPPORTED, dataTypes[index]));
-    }
-    return value;
-  }
-
-  public TimeValuePair composeLastTimeValuePair(int measurementIndex) {
-    if (measurementIndex >= columns.length) {
-      return null;
-    }
-
-    // get non-null value
-    int lastIdx = rowCount - 1;
-    if (bitMaps != null && bitMaps[measurementIndex] != null) {
-      BitMap bitMap = bitMaps[measurementIndex];
-      while (lastIdx >= 0) {
-        if (!bitMap.isMarked(lastIdx)) {
-          break;
-        }
-        lastIdx--;
-      }
-    }
-    if (lastIdx < 0) {
-      return null;
-    }
-
-    TsPrimitiveType value;
-    switch (dataTypes[measurementIndex]) {
-      case INT32:
-        int[] intValues = (int[]) columns[measurementIndex];
-        value = new TsInt(intValues[lastIdx]);
-        break;
-      case INT64:
-        long[] longValues = (long[]) columns[measurementIndex];
-        value = new TsLong(longValues[lastIdx]);
-        break;
-      case FLOAT:
-        float[] floatValues = (float[]) columns[measurementIndex];
-        value = new TsFloat(floatValues[lastIdx]);
-        break;
-      case DOUBLE:
-        double[] doubleValues = (double[]) columns[measurementIndex];
-        value = new TsDouble(doubleValues[lastIdx]);
-        break;
-      case BOOLEAN:
-        boolean[] boolValues = (boolean[]) columns[measurementIndex];
-        value = new TsBoolean(boolValues[lastIdx]);
-        break;
-      case TEXT:
-        Binary[] binaryValues = (Binary[]) columns[measurementIndex];
-        value = new TsBinary(binaryValues[lastIdx]);
-        break;
-      default:
-        throw new UnSupportedDataTypeException(
-            String.format(DATATYPE_UNSUPPORTED, dataTypes[measurementIndex]));
-    }
-    return new TimeValuePair(times[lastIdx], value);
-  }
-
-  public long[] getTimes() {
-    return times;
-  }
-
-  public void setTimes(long[] times) {
-    this.times = times;
-  }
-
-  public int getRowCount() {
-    return rowCount;
-  }
-
-  public void setRowCount(int size) {
-    this.rowCount = size;
-  }
-
-  @Override
-  public String toString() {
-    return "InsertTabletPlan {"
-        + "prefixPath:"
-        + devicePath
-        + ", timesRange["
-        + times[0]
-        + ","
-        + times[times.length - 1]
-        + "]"
-        + ", isAligned:"
-        + isAligned
-        + '}';
-  }
-
-  @Override
-  public void markFailedMeasurementInsertion(int index, Exception e) {
-    if (measurements[index] == null) {
-      return;
-    }
-    super.markFailedMeasurementInsertion(index, e);
-    if (failedColumns == null) {
-      failedColumns = new ArrayList<>();
-    }
-    failedColumns.add(columns[index]);
-    columns[index] = null;
-  }
-
-  @Override
-  public InsertPlan getPlanFromFailed() {
-    if (super.getPlanFromFailed() == null) {
-      return null;
-    }
-    // TODO anything else?
-    columns = failedColumns.toArray(new Object[0]);
-    failedColumns = null;
-    return this;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-    InsertTabletPlan that = (InsertTabletPlan) o;
-
-    return rowCount == that.rowCount
-        && Objects.equals(devicePath, that.devicePath)
-        && Arrays.equals(times, that.times)
-        && Objects.equals(paths, that.paths)
-        && Objects.equals(range, that.range)
-        && Objects.equals(isAligned, that.isAligned);
-  }
-
-  @Override
-  public int hashCode() {
-    int result = Objects.hash(rowCount, paths, range);
-    result = 31 * result + Arrays.hashCode(times);
-    return result;
-  }
-
-  @Override
-  public void recoverFromFailure() {
-    if (failedMeasurements == null) {
-      return;
-    }
-
-    for (int i = 0; i < failedMeasurements.size(); i++) {
-      int index = failedIndices.get(i);
-      columns[index] = failedColumns.get(i);
-    }
-    super.recoverFromFailure();
-
-    failedColumns = null;
-  }
-
-  @Override
-  public void checkIntegrity() throws QueryProcessException {
-    super.checkIntegrity();
-    if (columns == null || columns.length == 0) {
-      throw new QueryProcessException("Values are null");
-    }
-    if (dataTypes.length != columns.length) {
-      throw new QueryProcessException(
-          String.format(
-              "Measurements length [%d] does not match " + "columns length [%d]",
-              measurements.length, columns.length));
-    }
-    for (Object value : columns) {
-      if (value == null) {
-        throw new QueryProcessException("Columns contain null: " + Arrays.toString(columns));
-      }
-    }
-  }
-}
diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/ClientRPCServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/ClientRPCServiceImpl.java
index e002d89ad5..b712c13779 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/ClientRPCServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/ClientRPCServiceImpl.java
@@ -953,7 +953,7 @@ public class ClientRPCServiceImpl implements IClientRPCServiceWithHandler {
       // check whether measurement is legal according to syntax convention
       PathUtils.isLegalSingleMeasurementLists(req.getMeasurementsList());
 
-      // Step 1: TODO(INSERT) transfer from TSInsertTabletsReq to Statement
+      // Step 1:  transfer from TSInsertRecordsReq to Statement
       InsertRowsStatement statement = (InsertRowsStatement) StatementGenerator.createStatement(req);
       // return success when this statement is empty because server doesn't need to execute it
       if (statement.isEmpty()) {
@@ -1008,7 +1008,7 @@ public class ClientRPCServiceImpl implements IClientRPCServiceWithHandler {
       // check whether measurement is legal according to syntax convention
       PathUtils.isLegalSingleMeasurementLists(req.getMeasurementsList());
 
-      // Step 1: TODO(INSERT) transfer from TSInsertTabletsReq to Statement
+      // Step 1: transfer from TSInsertRecordsOfOneDeviceReq to Statement
       InsertRowsOfOneDeviceStatement statement =
           (InsertRowsOfOneDeviceStatement) StatementGenerator.createStatement(req);
       // return success when this statement is empty because server doesn't need to execute it
@@ -1064,7 +1064,7 @@ public class ClientRPCServiceImpl implements IClientRPCServiceWithHandler {
       // check whether measurement is legal according to syntax convention
       PathUtils.isLegalSingleMeasurementLists(req.getMeasurementsList());
 
-      // Step 1: TODO(INSERT) transfer from TSInsertTabletsReq to Statement
+      // Step 1: transfer from TSInsertStringRecordsOfOneDeviceReq to Statement
       InsertRowsOfOneDeviceStatement statement =
           (InsertRowsOfOneDeviceStatement) StatementGenerator.createStatement(req);
       // return success when this statement is empty because server doesn't need to execute it
@@ -1166,7 +1166,7 @@ public class ClientRPCServiceImpl implements IClientRPCServiceWithHandler {
 
       PathUtils.isLegalSingleMeasurementLists(req.getMeasurementsList());
 
-      // Step 1: TODO(INSERT) transfer from TSInsertTabletsReq to Statement
+      // Step 1: transfer from TSInsertTabletsReq to Statement
       InsertMultiTabletsStatement statement =
           (InsertMultiTabletsStatement) StatementGenerator.createStatement(req);
       // return success when this statement is empty because server doesn't need to execute it
@@ -1213,7 +1213,7 @@ public class ClientRPCServiceImpl implements IClientRPCServiceWithHandler {
 
       // check whether measurement is legal according to syntax convention
       PathUtils.isLegalSingleMeasurements(req.getMeasurements());
-      // Step 1: TODO(INSERT) transfer from TSInsertTabletReq to Statement
+      // Step 1: transfer from TSInsertTabletReq to Statement
       InsertTabletStatement statement =
           (InsertTabletStatement) StatementGenerator.createStatement(req);
       // return success when this statement is empty because server doesn't need to execute it
diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java
index 72f1112c71..bddca8c86c 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java
@@ -37,18 +37,11 @@ import org.apache.iotdb.db.auth.AuthorizerManager;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.conf.OperationType;
-import org.apache.iotdb.db.exception.QueryInBatchStatementException;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.template.TemplateQueryType;
 import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
 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.qp.physical.crud.QueryPlan;
 import org.apache.iotdb.db.qp.physical.crud.UDFPlan;
 import org.apache.iotdb.db.qp.physical.sys.AppendTemplatePlan;
@@ -366,25 +359,6 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
     return IoTDB.schemaProcessor.getSeriesType(path);
   }
 
-  private boolean executeInsertRowsPlan(InsertRowsPlan insertRowsPlan, List<TSStatus> result) {
-    long t1 = System.currentTimeMillis();
-    TSStatus tsStatus = executeNonQueryPlan(insertRowsPlan);
-    addOperationLatency(Operation.EXECUTE_ROWS_PLAN_IN_BATCH, t1);
-    int startIndex = result.size();
-    if (startIndex > 0) {
-      startIndex = startIndex - 1;
-    }
-    for (int i = 0; i < insertRowsPlan.getRowCount(); i++) {
-      result.add(RpcUtils.SUCCESS_STATUS);
-    }
-    if (tsStatus.subStatus != null) {
-      for (Entry<Integer, TSStatus> entry : insertRowsPlan.getResults().entrySet()) {
-        result.set(startIndex + entry.getKey(), entry.getValue());
-      }
-    }
-    return tsStatus.getCode() == RpcUtils.SUCCESS_STATUS.getCode();
-  }
-
   private boolean executeMultiTimeSeriesPlan(
       CreateMultiTimeSeriesPlan multiPlan, List<TSStatus> result) {
     long t1 = System.currentTimeMillis();
@@ -444,130 +418,9 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
     multiPlan.getAlias().add(alias);
   }
 
-  private boolean executeBatchList(List executeList, List<TSStatus> result) {
-    boolean isAllSuccessful = true;
-    for (int j = 0; j < executeList.size(); j++) {
-      Object planObject = executeList.get(j);
-      if (InsertRowsPlan.class.isInstance(planObject)) {
-        if (!executeInsertRowsPlan((InsertRowsPlan) planObject, result)) {
-          isAllSuccessful = false;
-        }
-      } else if (CreateMultiTimeSeriesPlan.class.isInstance(planObject)) {
-        if (!executeMultiTimeSeriesPlan((CreateMultiTimeSeriesPlan) planObject, result)) {
-          isAllSuccessful = false;
-        }
-      }
-    }
-    return isAllSuccessful;
-  }
-
   @Override
   public TSStatus executeBatchStatement(TSExecuteBatchStatementReq req) {
-    long t1 = System.currentTimeMillis();
-    List<TSStatus> result = new ArrayList<>();
-    boolean isAllSuccessful = true;
-    if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
-      return getNotLoggedInStatus();
-    }
-
-    InsertRowsPlan insertRowsPlan;
-    int index = 0;
-    List<Object> executeList = new ArrayList<>();
-    OperatorType lastOperatorType = null;
-    CreateMultiTimeSeriesPlan multiPlan;
-    for (int i = 0; i < req.getStatements().size(); i++) {
-      String statement = req.getStatements().get(i);
-      try {
-        PhysicalPlan physicalPlan =
-            serviceProvider
-                .getPlanner()
-                .parseSQLToPhysicalPlan(
-                    statement,
-                    SESSION_MANAGER.getSessionTimeZone().toZoneId(),
-                    SESSION_MANAGER.getCurrSession().getClientVersion());
-        if (physicalPlan.isQuery() || physicalPlan.isSelectInto()) {
-          throw new QueryInBatchStatementException(statement);
-        }
-
-        if (physicalPlan.getOperatorType().equals(OperatorType.INSERT)) {
-          if (OperatorType.INSERT == lastOperatorType) {
-            insertRowsPlan = (InsertRowsPlan) executeList.get(executeList.size() - 1);
-          } else {
-            insertRowsPlan = new InsertRowsPlan();
-            executeList.add(insertRowsPlan);
-            index = 0;
-          }
-
-          TSStatus status =
-              SESSION_MANAGER.checkAuthority(physicalPlan, SESSION_MANAGER.getCurrSession());
-          if (status != null) {
-            insertRowsPlan.getResults().put(index, status);
-            isAllSuccessful = false;
-          }
-
-          lastOperatorType = OperatorType.INSERT;
-          insertRowsPlan.addOneInsertRowPlan((InsertRowPlan) physicalPlan, index);
-          index++;
-
-          if (i == req.getStatements().size() - 1) {
-            if (!executeBatchList(executeList, result)) {
-              isAllSuccessful = false;
-            }
-          }
-        } else if (physicalPlan.getOperatorType().equals(OperatorType.CREATE_TIMESERIES)) {
-          if (OperatorType.CREATE_TIMESERIES == lastOperatorType) {
-            multiPlan = (CreateMultiTimeSeriesPlan) executeList.get(executeList.size() - 1);
-          } else {
-            multiPlan = new CreateMultiTimeSeriesPlan();
-            executeList.add(multiPlan);
-          }
-          TSStatus status =
-              SESSION_MANAGER.checkAuthority(physicalPlan, SESSION_MANAGER.getCurrSession());
-          if (status != null) {
-            multiPlan.getResults().put(i, status);
-            isAllSuccessful = false;
-          }
-
-          lastOperatorType = OperatorType.CREATE_TIMESERIES;
-          initMultiTimeSeriesPlan(multiPlan);
-
-          CreateTimeSeriesPlan createTimeSeriesPlan = (CreateTimeSeriesPlan) physicalPlan;
-          setMultiTimeSeriesPlan(multiPlan, createTimeSeriesPlan);
-          if (i == req.getStatements().size() - 1) {
-            if (!executeBatchList(executeList, result)) {
-              isAllSuccessful = false;
-            }
-          }
-        } else {
-          lastOperatorType = physicalPlan.getOperatorType();
-          if (!executeList.isEmpty()) {
-            if (!executeBatchList(executeList, result)) {
-              isAllSuccessful = false;
-            }
-            executeList.clear();
-          }
-          long t2 = System.currentTimeMillis();
-          TSExecuteStatementResp resp = executeNonQueryStatement(physicalPlan);
-          addOperationLatency(Operation.EXECUTE_ONE_SQL_IN_BATCH, t2);
-          result.add(resp.status);
-          if (resp.getStatus().code != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
-            isAllSuccessful = false;
-          }
-        }
-      } catch (Exception e) {
-        LOGGER.error("Error occurred when executing executeBatchStatement: ", e);
-        TSStatus status =
-            onQueryException(e, "\"" + statement + "\". " + OperationType.EXECUTE_BATCH_STATEMENT);
-        if (status.getCode() != TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()) {
-          isAllSuccessful = false;
-        }
-        result.add(status);
-      }
-    }
-    addOperationLatency(Operation.EXECUTE_JDBC_BATCH, t1);
-    return isAllSuccessful
-        ? RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS, "Execute batch statements successfully")
-        : RpcUtils.getStatus(result);
+    return null;
   }
 
   @Override
@@ -886,23 +739,6 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
     return resp;
   }
 
-  private TSStatus insertTabletsInternally(List<InsertTabletPlan> insertTabletPlans) {
-    InsertMultiTabletsPlan insertMultiTabletsPlan = new InsertMultiTabletsPlan();
-    for (int i = 0; i < insertTabletPlans.size(); i++) {
-      InsertTabletPlan insertTabletPlan = insertTabletPlans.get(i);
-      TSStatus status =
-          SESSION_MANAGER.checkAuthority(insertTabletPlan, SESSION_MANAGER.getCurrSession());
-
-      if (status != null) {
-        // not authorized
-        insertMultiTabletsPlan.getResults().put(i, status);
-      }
-    }
-    insertMultiTabletsPlan.setInsertTabletPlanList(insertTabletPlans);
-
-    return executeNonQueryPlan(insertMultiTabletsPlan);
-  }
-
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   @Override
   public TSFetchResultsResp fetchResults(TSFetchResultsReq req) {
@@ -1085,56 +921,7 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
 
   @Override
   public TSStatus insertRecords(TSInsertRecordsReq req) {
-    if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
-      return getNotLoggedInStatus();
-    }
-
-    if (AUDIT_LOGGER.isDebugEnabled()) {
-      AUDIT_LOGGER.debug(
-          "Session {} insertRecords, first device {}, first time {}",
-          SESSION_MANAGER.getCurrSession(),
-          req.prefixPaths.get(0),
-          req.getTimestamps().get(0));
-    }
-    boolean allCheckSuccess = true;
-
-    InsertRowsPlan insertRowsPlan = new InsertRowsPlan();
-    for (int i = 0; i < req.prefixPaths.size(); i++) {
-      try {
-        // check whether measurement is legal according to syntax convention
-        PathUtils.isLegalSingleMeasurements(req.getMeasurementsList().get(i));
-        InsertRowPlan plan =
-            new InsertRowPlan(
-                new PartialPath(req.getPrefixPaths().get(i)),
-                req.getTimestamps().get(i),
-                req.getMeasurementsList().get(i).toArray(new String[0]),
-                req.valuesList.get(i),
-                req.isAligned);
-        TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
-        if (status != null) {
-          insertRowsPlan.getResults().put(i, status);
-          allCheckSuccess = false;
-        }
-        insertRowsPlan.addOneInsertRowPlan(plan, i);
-      } catch (IoTDBException e) {
-        allCheckSuccess = false;
-        insertRowsPlan
-            .getResults()
-            .put(i, onIoTDBException(e, OperationType.INSERT_RECORDS, e.getErrorCode()));
-      } catch (Exception e) {
-        allCheckSuccess = false;
-        insertRowsPlan
-            .getResults()
-            .put(
-                i,
-                onNPEOrUnexpectedException(
-                    e, OperationType.INSERT_RECORDS, TSStatusCode.INTERNAL_SERVER_ERROR));
-      }
-    }
-    TSStatus tsStatus = executeNonQueryPlan(insertRowsPlan);
-
-    return judgeFinalTsStatus(
-        allCheckSuccess, tsStatus, insertRowsPlan.getResults(), req.prefixPaths.size());
+    return null;
   }
 
   private TSStatus judgeFinalTsStatus(
@@ -1160,183 +947,17 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
 
   @Override
   public TSStatus insertRecordsOfOneDevice(TSInsertRecordsOfOneDeviceReq req) {
-    if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
-      return getNotLoggedInStatus();
-    }
-
-    if (AUDIT_LOGGER.isDebugEnabled()) {
-      AUDIT_LOGGER.debug(
-          "Session {} insertRecords, device {}, first time {}",
-          SESSION_MANAGER.getCurrSession(),
-          req.prefixPath,
-          req.getTimestamps().get(0));
-    }
-
-    List<TSStatus> statusList = new ArrayList<>();
-    try {
-      // check whether measurement is legal according to syntax convention
-      PathUtils.isLegalSingleMeasurementLists(req.getMeasurementsList());
-      InsertRowsOfOneDevicePlan plan =
-          new InsertRowsOfOneDevicePlan(
-              new PartialPath(req.getPrefixPath()),
-              req.getTimestamps(),
-              req.getMeasurementsList(),
-              req.getValuesList(),
-              req.isAligned);
-      TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
-      statusList.add(status != null ? status : executeNonQueryPlan(plan));
-    } catch (IoTDBException e) {
-      statusList.add(
-          onIoTDBException(e, OperationType.INSERT_RECORDS_OF_ONE_DEVICE, e.getErrorCode()));
-    } catch (Exception e) {
-      statusList.add(
-          onNPEOrUnexpectedException(
-              e, OperationType.INSERT_RECORDS_OF_ONE_DEVICE, TSStatusCode.INTERNAL_SERVER_ERROR));
-    }
-
-    TSStatus resp = RpcUtils.getStatus(statusList);
-    for (TSStatus status : resp.subStatus) {
-      if (status.code != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
-        return resp;
-      }
-    }
-
-    resp.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode());
-
-    return resp;
+    return null;
   }
 
   @Override
   public TSStatus insertStringRecordsOfOneDevice(TSInsertStringRecordsOfOneDeviceReq req) {
-    if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
-      return getNotLoggedInStatus();
-    }
-
-    if (AUDIT_LOGGER.isDebugEnabled()) {
-      AUDIT_LOGGER.debug(
-          "Session {} insertRecords, device {}, first time {}",
-          SESSION_MANAGER.getCurrSession(),
-          req.prefixPath,
-          req.getTimestamps().get(0));
-    }
-
-    boolean allCheckSuccess = true;
-    InsertRowsPlan insertRowsPlan = new InsertRowsPlan();
-    for (int i = 0; i < req.timestamps.size(); i++) {
-      InsertRowPlan plan = new InsertRowPlan();
-      try {
-        // check whether measurement is legal according to syntax convention
-        PathUtils.isLegalSingleMeasurements(req.getMeasurementsList().get(i));
-        plan.setDevicePath(new PartialPath(req.getPrefixPath()));
-        plan.setTime(req.getTimestamps().get(i));
-        addMeasurementAndValue(plan, req.getMeasurementsList().get(i), req.getValuesList().get(i));
-        plan.setDataTypes(new TSDataType[plan.getMeasurements().length]);
-        plan.setNeedInferType(true);
-        plan.setAligned(req.isAligned);
-        TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
-
-        if (status != null) {
-          insertRowsPlan.getResults().put(i, status);
-          allCheckSuccess = false;
-        }
-        insertRowsPlan.addOneInsertRowPlan(plan, i);
-      } catch (IoTDBException e) {
-        insertRowsPlan
-            .getResults()
-            .put(
-                i,
-                onIoTDBException(
-                    e, OperationType.INSERT_STRING_RECORDS_OF_ONE_DEVICE, e.getErrorCode()));
-        allCheckSuccess = false;
-      } catch (Exception e) {
-        insertRowsPlan
-            .getResults()
-            .put(
-                i,
-                onNPEOrUnexpectedException(
-                    e,
-                    OperationType.INSERT_STRING_RECORDS_OF_ONE_DEVICE,
-                    TSStatusCode.INTERNAL_SERVER_ERROR));
-        allCheckSuccess = false;
-      }
-    }
-    TSStatus tsStatus = executeNonQueryPlan(insertRowsPlan);
-
-    return judgeFinalTsStatus(
-        allCheckSuccess, tsStatus, insertRowsPlan.getResults(), req.timestamps.size());
+    return null;
   }
 
   @Override
   public TSStatus insertStringRecords(TSInsertStringRecordsReq req) {
-    if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
-      return getNotLoggedInStatus();
-    }
-
-    if (AUDIT_LOGGER.isDebugEnabled()) {
-      AUDIT_LOGGER.debug(
-          "Session {} insertRecords, first device {}, first time {}",
-          SESSION_MANAGER.getCurrSession(),
-          req.prefixPaths.get(0),
-          req.getTimestamps().get(0));
-    }
-
-    boolean allCheckSuccess = true;
-    InsertRowsPlan insertRowsPlan = new InsertRowsPlan();
-    for (int i = 0; i < req.prefixPaths.size(); i++) {
-      InsertRowPlan plan = new InsertRowPlan();
-      try {
-        // check whether measurement is legal according to syntax convention
-        PathUtils.isLegalSingleMeasurements(req.getMeasurementsList().get(i));
-        plan.setDevicePath(new PartialPath(req.getPrefixPaths().get(i)));
-        plan.setTime(req.getTimestamps().get(i));
-        addMeasurementAndValue(plan, req.getMeasurementsList().get(i), req.getValuesList().get(i));
-        plan.setDataTypes(new TSDataType[plan.getMeasurements().length]);
-        plan.setNeedInferType(true);
-        plan.setAligned(req.isAligned);
-        TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
-
-        if (status != null) {
-          insertRowsPlan.getResults().put(i, status);
-          allCheckSuccess = false;
-        }
-        insertRowsPlan.addOneInsertRowPlan(plan, i);
-      } catch (IoTDBException e) {
-        insertRowsPlan
-            .getResults()
-            .put(i, onIoTDBException(e, OperationType.INSERT_STRING_RECORDS, e.getErrorCode()));
-        allCheckSuccess = false;
-      } catch (Exception e) {
-        insertRowsPlan
-            .getResults()
-            .put(
-                i,
-                onNPEOrUnexpectedException(
-                    e, OperationType.INSERT_STRING_RECORDS, TSStatusCode.INTERNAL_SERVER_ERROR));
-        allCheckSuccess = false;
-      }
-    }
-    TSStatus tsStatus = executeNonQueryPlan(insertRowsPlan);
-
-    return judgeFinalTsStatus(
-        allCheckSuccess, tsStatus, insertRowsPlan.getResults(), req.prefixPaths.size());
-  }
-
-  private void addMeasurementAndValue(
-      InsertRowPlan insertRowPlan, List<String> measurements, List<String> values) {
-    List<String> newMeasurements = new ArrayList<>(measurements.size());
-    List<Object> newValues = new ArrayList<>(values.size());
-
-    for (int i = 0; i < measurements.size(); ++i) {
-      String value = values.get(i);
-      if (value.isEmpty()) {
-        continue;
-      }
-      newMeasurements.add(measurements.get(i));
-      newValues.add(value);
-    }
-
-    insertRowPlan.setValues(newValues.toArray(new Object[0]));
-    insertRowPlan.setMeasurements(newMeasurements.toArray(new String[0]));
+    return null;
   }
 
   @Override
@@ -1383,133 +1004,22 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
 
   @Override
   public TSStatus insertRecord(TSInsertRecordReq req) {
-    try {
-      if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
-        return getNotLoggedInStatus();
-      }
-
-      AUDIT_LOGGER.debug(
-          "Session {} insertRecord, device {}, time {}",
-          SESSION_MANAGER.getCurrSession(),
-          req.getPrefixPath(),
-          req.getTimestamp());
-
-      // check whether measurement is legal according to syntax convention
-
-      PathUtils.isLegalSingleMeasurements(req.getMeasurements());
-
-      InsertRowPlan plan =
-          new InsertRowPlan(
-              new PartialPath(req.getPrefixPath()),
-              req.getTimestamp(),
-              req.getMeasurements().toArray(new String[0]),
-              req.values,
-              req.isAligned);
-      TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
-      return status != null ? status : executeNonQueryPlan(plan);
-    } catch (IoTDBException e) {
-      return onIoTDBException(e, OperationType.INSERT_RECORD, e.getErrorCode());
-    } catch (Exception e) {
-      return onNPEOrUnexpectedException(
-          e, OperationType.INSERT_RECORD, TSStatusCode.EXECUTE_STATEMENT_ERROR);
-    }
+    return null;
   }
 
   @Override
   public TSStatus insertStringRecord(TSInsertStringRecordReq req) {
-    try {
-      if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
-        return getNotLoggedInStatus();
-      }
-
-      AUDIT_LOGGER.debug(
-          "Session {} insertRecord, device {}, time {}",
-          SESSION_MANAGER.getCurrSession(),
-          req.getPrefixPath(),
-          req.getTimestamp());
-
-      // check whether measurement is legal according to syntax convention
-      PathUtils.isLegalSingleMeasurements(req.getMeasurements());
-
-      InsertRowPlan plan = new InsertRowPlan();
-      plan.setDevicePath(new PartialPath(req.getPrefixPath()));
-      plan.setTime(req.getTimestamp());
-      plan.setMeasurements(req.getMeasurements().toArray(new String[0]));
-      plan.setDataTypes(new TSDataType[plan.getMeasurements().length]);
-      plan.setValues(req.getValues().toArray(new Object[0]));
-      plan.setNeedInferType(true);
-      plan.setAligned(req.isAligned);
-      TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
-      return status != null ? status : executeNonQueryPlan(plan);
-    } catch (IoTDBException e) {
-      return onIoTDBException(e, OperationType.INSERT_STRING_RECORD, e.getErrorCode());
-    } catch (Exception e) {
-      return onNPEOrUnexpectedException(
-          e, OperationType.INSERT_STRING_RECORD, TSStatusCode.EXECUTE_STATEMENT_ERROR);
-    }
+    return null;
   }
 
   @Override
   public TSStatus deleteData(TSDeleteDataReq req) {
-    try {
-      if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
-        return getNotLoggedInStatus();
-      }
-
-      DeletePlan plan = new DeletePlan();
-      plan.setDeleteStartTime(req.getStartTime());
-      plan.setDeleteEndTime(req.getEndTime());
-      List<PartialPath> paths = new ArrayList<>();
-      for (String path : req.getPaths()) {
-        paths.add(new PartialPath(path));
-      }
-      plan.addPaths(paths);
-      TSStatus status = SESSION_MANAGER.checkAuthority(plan, SESSION_MANAGER.getCurrSession());
-
-      return status != null ? new TSStatus(status) : new TSStatus(executeNonQueryPlan(plan));
-    } catch (IoTDBException e) {
-      return onIoTDBException(e, OperationType.DELETE_DATA, e.getErrorCode());
-    } catch (Exception e) {
-      return onNPEOrUnexpectedException(
-          e, OperationType.DELETE_DATA, TSStatusCode.EXECUTE_STATEMENT_ERROR);
-    }
+    return null;
   }
 
   @Override
   public TSStatus insertTablet(TSInsertTabletReq req) {
-    long t1 = System.currentTimeMillis();
-    try {
-      if (!SESSION_MANAGER.checkLogin(SESSION_MANAGER.getCurrSession())) {
-        return getNotLoggedInStatus();
-      }
-
-      // check whether measurement is legal according to syntax convention
-
-      PathUtils.isLegalSingleMeasurements(req.getMeasurements());
-
-      InsertTabletPlan insertTabletPlan =
-          new InsertTabletPlan(new PartialPath(req.getPrefixPath()), req.measurements);
-      insertTabletPlan.setTimes(QueryDataSetUtils.readTimesFromBuffer(req.timestamps, req.size));
-      insertTabletPlan.setColumns(
-          QueryDataSetUtils.readTabletValuesFromBuffer(
-              req.values, req.types, req.types.size(), req.size));
-      insertTabletPlan.setBitMaps(
-          QueryDataSetUtils.readBitMapsFromBuffer(req.values, req.types.size(), req.size));
-      insertTabletPlan.setRowCount(req.size);
-      insertTabletPlan.setDataTypes(req.types);
-      insertTabletPlan.setAligned(req.isAligned);
-      TSStatus status =
-          SESSION_MANAGER.checkAuthority(insertTabletPlan, SESSION_MANAGER.getCurrSession());
-
-      return status != null ? status : executeNonQueryPlan(insertTabletPlan);
-    } catch (IoTDBException e) {
-      return onIoTDBException(e, OperationType.INSERT_TABLET, e.getErrorCode());
-    } catch (Exception e) {
-      return onNPEOrUnexpectedException(
-          e, OperationType.INSERT_TABLET, TSStatusCode.EXECUTE_STATEMENT_ERROR);
-    } finally {
-      addOperationLatency(Operation.EXECUTE_RPC_BATCH_INSERT, t1);
-    }
+    return null;
   }
 
   @Override
@@ -1534,46 +1044,8 @@ public class TSServiceImpl implements IClientRPCServiceWithHandler {
     }
   }
 
-  private InsertTabletPlan constructInsertTabletPlan(TSInsertTabletsReq req, int i)
-      throws MetadataException {
-    // check whether measurement is legal according to syntax convention
-    PathUtils.isLegalSingleMeasurementLists(req.getMeasurementsList());
-    InsertTabletPlan insertTabletPlan =
-        new InsertTabletPlan(new PartialPath(req.prefixPaths.get(i)), req.measurementsList.get(i));
-    insertTabletPlan.setTimes(
-        QueryDataSetUtils.readTimesFromBuffer(req.timestampsList.get(i), req.sizeList.get(i)));
-    insertTabletPlan.setColumns(
-        QueryDataSetUtils.readTabletValuesFromBuffer(
-            req.valuesList.get(i),
-            req.typesList.get(i),
-            req.measurementsList.get(i).size(),
-            req.sizeList.get(i)));
-    insertTabletPlan.setBitMaps(
-        QueryDataSetUtils.readBitMapsFromBuffer(
-            req.valuesList.get(i), req.measurementsList.get(i).size(), req.sizeList.get(i)));
-    insertTabletPlan.setRowCount(req.sizeList.get(i));
-    insertTabletPlan.setDataTypes(req.typesList.get(i));
-    insertTabletPlan.setAligned(req.isAligned);
-    return insertTabletPlan;
-  }
-
-  /** construct one InsertMultiTabletsPlan and process it */
   public TSStatus insertTabletsInternally(TSInsertTabletsReq req) throws MetadataException {
-    List<InsertTabletPlan> insertTabletPlanList = new ArrayList<>();
-    InsertMultiTabletsPlan insertMultiTabletsPlan = new InsertMultiTabletsPlan();
-    for (int i = 0; i < req.prefixPaths.size(); i++) {
-      InsertTabletPlan insertTabletPlan = constructInsertTabletPlan(req, i);
-      TSStatus status =
-          SESSION_MANAGER.checkAuthority(insertTabletPlan, SESSION_MANAGER.getCurrSession());
-      if (status != null) {
-        // not authorized
-        insertMultiTabletsPlan.getResults().put(i, status);
-      }
-      insertTabletPlanList.add(insertTabletPlan);
-    }
-
-    insertMultiTabletsPlan.setInsertTabletPlanList(insertTabletPlanList);
-    return executeNonQueryPlan(insertMultiTabletsPlan);
+    return null;
   }
 
   @Override
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/MemUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/MemUtils.java
index 61be14f55e..e879e3c78a 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/MemUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/MemUtils.java
@@ -20,7 +20,6 @@ package org.apache.iotdb.db.utils;
 
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 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.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.RamUsageEstimator;
@@ -135,59 +134,6 @@ public class MemUtils {
     return memSize;
   }
 
-  /**
-   * If mem control enabled, do not add text data size here, the size will be added to memtable
-   * before inserting.
-   */
-  public static long getTabletSize(
-      InsertTabletPlan insertTabletPlan, int start, int end, boolean addingTextDataSize) {
-    if (start >= end) {
-      return 0L;
-    }
-    long memSize = 0;
-    for (int i = 0; i < insertTabletPlan.getMeasurements().length; i++) {
-      if (insertTabletPlan.getMeasurements()[i] == null) {
-        continue;
-      }
-      // time column memSize
-      memSize += (end - start) * 8L;
-      if (insertTabletPlan.getDataTypes()[i] == TSDataType.TEXT && addingTextDataSize) {
-        for (int j = start; j < end; j++) {
-          memSize += getBinarySize(((Binary[]) insertTabletPlan.getColumns()[i])[j]);
-        }
-      } else {
-        memSize += (end - start) * insertTabletPlan.getDataTypes()[i].getDataTypeSize();
-      }
-    }
-    return memSize;
-  }
-
-  public static long getAlignedTabletSize(
-      InsertTabletPlan insertTabletPlan, int start, int end, boolean addingTextDataSize) {
-    if (start >= end) {
-      return 0L;
-    }
-    long memSize = 0;
-    for (int i = 0; i < insertTabletPlan.getMeasurements().length; i++) {
-      if (insertTabletPlan.getMeasurements()[i] == null) {
-        continue;
-      }
-      TSDataType valueType;
-      // value columns memSize
-      valueType = insertTabletPlan.getDataTypes()[i];
-      if (valueType == TSDataType.TEXT && addingTextDataSize) {
-        for (int j = start; j < end; j++) {
-          memSize += getBinarySize(((Binary[]) insertTabletPlan.getColumns()[i])[j]);
-        }
-      } else {
-        memSize += (long) (end - start) * valueType.getDataTypeSize();
-      }
-    }
-    // time and index column memSize for vector
-    memSize += (end - start) * (8L + 4L);
-    return memSize;
-  }
-
   public static long getAlignedTabletSize(
       InsertTabletNode insertTabletNode, int start, int end, boolean addingTextDataSize) {
     if (start >= end) {
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 ff1da0c37a..97214848de 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
@@ -21,16 +21,12 @@ package org.apache.iotdb.db.wal.buffer;
 /** Type of {@link WALEntry}, including info type and signal type */
 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),
   /** {@link org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertRowNode} */
   INSERT_ROW_NODE((byte) 4),
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 d4246616c7..6b8ebb4a70 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
@@ -30,7 +30,7 @@ public class WALInfoEntry extends WALEntry {
   /** wal entry type 1 byte, memTable id 8 bytes */
   public static final int FIXED_SERIALIZED_SIZE = Byte.BYTES + Long.BYTES;
 
-  /** extra info for InsertTabletPlan type value */
+  /** extra info for InsertTablet type value */
   private TabletInfo tabletInfo;
 
   public WALInfoEntry(long memTableId, WALEntryValue value, boolean wait) {
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 9f992a9442..b38b0cc6f4 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
@@ -18,7 +18,6 @@
  */
 package org.apache.iotdb.db.wal.recover.file;
 
-import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.memtable.IMemTable;
@@ -26,17 +25,13 @@ import org.apache.iotdb.db.engine.memtable.PrimitiveMemTable;
 import org.apache.iotdb.db.engine.modification.Deletion;
 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.metadata.idtable.IDTable;
 import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.mpp.plan.analyze.SchemaValidator;
 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.InsertPlan;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -137,29 +132,6 @@ public class TsFilePlanRedoer {
     }
   }
 
-  private void checkDataTypeAndMarkFailed(final IMeasurementMNode[] mNodes, InsertPlan tPlan) {
-    for (int i = 0; i < mNodes.length; i++) {
-      if (mNodes[i] == null) {
-        tPlan.markFailedMeasurementInsertion(
-            i,
-            new PathNotExistException(
-                tPlan.getDevicePath().getFullPath()
-                    + IoTDBConstant.PATH_SEPARATOR
-                    + tPlan.getMeasurements()[i]));
-      } else if (mNodes[i].getSchema().getType() != tPlan.getDataTypes()[i]) {
-        tPlan.markFailedMeasurementInsertion(
-            i,
-            new DataTypeMismatchException(
-                tPlan.getDevicePath().getFullPath(),
-                mNodes[i].getName(),
-                tPlan.getDataTypes()[i],
-                mNodes[i].getSchema().getType(),
-                tPlan.getMinTime(),
-                tPlan.getFirstValueOfIndex(i)));
-      }
-    }
-  }
-
   void resetRecoveryMemTable(IMemTable memTable) {
     this.recoveryMemTable = memTable;
   }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/SchemaBasicTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/SchemaBasicTest.java
index 5de4a0ac69..b1d5e2469a 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/SchemaBasicTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/SchemaBasicTest.java
@@ -30,8 +30,6 @@ 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.metadata.utils.MetaUtils;
-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.sys.ActivateTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.AppendTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
@@ -1957,161 +1955,6 @@ public abstract class SchemaBasicTest {
     assertTrue(schemaProcessor.isPathExist(new PartialPath("root.group_with_hyphen")));
   }
 
-  @Test
-  public void testCreateAlignedTimeseriesAndInsertWithMismatchDataType() {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-    try {
-      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
-      schemaProcessor.createAlignedTimeSeries(
-          new PartialPath("root.laptop.d1.aligned_device"),
-          Arrays.asList("s1", "s2", "s3"),
-          Arrays.asList(
-              TSDataType.valueOf("FLOAT"),
-              TSDataType.valueOf("INT64"),
-              TSDataType.valueOf("INT32")),
-          Arrays.asList(
-              TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE")),
-          Arrays.asList(compressionType, compressionType, compressionType));
-
-      // construct an insertRowPlan with mismatched data type
-      long time = 1L;
-      TSDataType[] dataTypes =
-          new TSDataType[] {TSDataType.FLOAT, TSDataType.DOUBLE, TSDataType.INT32};
-
-      String[] columns = new String[3];
-      columns[0] = 2.0 + "";
-      columns[1] = 10000 + "";
-      columns[2] = 100 + "";
-
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.aligned_device"),
-              time,
-              new String[] {"s1", "s2", "s3"},
-              dataTypes,
-              columns,
-              true);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      // call getSeriesSchemasAndReadLockDevice
-      IMNode node = schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
-      assertEquals(
-          3, schemaProcessor.getAllTimeseriesCount(node.getPartialPath().concatNode("**")));
-
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail(e.getMessage());
-    }
-  }
-
-  @Test
-  public void testCreateAlignedTimeseriesAndInsertWithNotAlignedData() {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-    try {
-      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
-      schemaProcessor.createAlignedTimeSeries(
-          new PartialPath("root.laptop.d1.aligned_device"),
-          Arrays.asList("s1", "s2", "s3"),
-          Arrays.asList(
-              TSDataType.valueOf("FLOAT"),
-              TSDataType.valueOf("INT64"),
-              TSDataType.valueOf("INT32")),
-          Arrays.asList(
-              TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE")),
-          Arrays.asList(compressionType, compressionType, compressionType));
-    } catch (Exception e) {
-      fail();
-    }
-
-    try {
-      schemaProcessor.createTimeseries(
-          new CreateTimeSeriesPlan(
-              new PartialPath("root.laptop.d1.aligned_device.s4"),
-              TSDataType.valueOf("FLOAT"),
-              TSEncoding.valueOf("RLE"),
-              compressionType,
-              null,
-              null,
-              null,
-              null));
-      fail();
-    } catch (Exception e) {
-      Assert.assertEquals(
-          "Timeseries under this entity is aligned, please use createAlignedTimeseries or change entity. (Path: root.laptop.d1.aligned_device)",
-          e.getMessage());
-    }
-
-    try {
-      // construct an insertRowPlan with mismatched data type
-      long time = 1L;
-      TSDataType[] dataTypes =
-          new TSDataType[] {TSDataType.FLOAT, TSDataType.INT64, TSDataType.INT32};
-
-      String[] columns = new String[3];
-      columns[0] = "1.0";
-      columns[1] = "2";
-      columns[2] = "3";
-
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.aligned_device"),
-              time,
-              new String[] {"s1", "s2", "s3"},
-              dataTypes,
-              columns,
-              false);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      // call getSeriesSchemasAndReadLockDevice
-      schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
-      fail();
-    } catch (Exception e) {
-      Assert.assertEquals(
-          "timeseries under this device are aligned, please use aligned interface (Path: root.laptop.d1.aligned_device)",
-          e.getMessage());
-    }
-  }
-
-  @Test
-  public void testCreateTimeseriesAndInsertWithMismatchDataType() {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-    try {
-      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
-      schemaProcessor.createTimeseries(
-          new PartialPath("root.laptop.d1.s0"),
-          TSDataType.valueOf("INT32"),
-          TSEncoding.valueOf("RLE"),
-          compressionType,
-          Collections.emptyMap());
-
-      // construct an insertRowPlan with mismatched data type
-      long time = 1L;
-      TSDataType[] dataTypes = new TSDataType[] {TSDataType.FLOAT};
-
-      String[] columns = new String[1];
-      columns[0] = 2.0 + "";
-
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1"), time, new String[] {"s0"}, dataTypes, columns);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      // call getSeriesSchemasAndReadLockDevice
-      IMNode node = schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
-      assertEquals(
-          1, schemaProcessor.getAllTimeseriesCount(node.getPartialPath().concatNode("**")));
-      assertNull(insertRowPlan.getMeasurementMNodes()[0]);
-      assertEquals(1, insertRowPlan.getFailedMeasurementNumber());
-
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail(e.getMessage());
-    }
-  }
-
   @Test
   public void testCreateTimeseriesAndInsertWithAlignedData() {
     LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
@@ -2150,35 +1993,6 @@ public abstract class SchemaBasicTest {
           "Timeseries under this entity is not aligned, please use createTimeseries or change entity. (Path: root.laptop.d1.aligned_device)",
           e.getMessage());
     }
-
-    try {
-      // construct an insertRowPlan with mismatched data type
-      long time = 1L;
-      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
-
-      String[] columns = new String[2];
-      columns[0] = "1";
-      columns[1] = "2";
-
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.aligned_device"),
-              time,
-              new String[] {"s1", "s2"},
-              dataTypes,
-              columns,
-              true);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      // call getSeriesSchemasAndReadLockDevice
-      schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
-      fail();
-    } catch (Exception e) {
-      Assert.assertEquals(
-          "timeseries under this device are not aligned, please use non-aligned interface (Path: root.laptop.d1.aligned_device)",
-          e.getMessage());
-    }
   }
 
   @Test
@@ -2300,51 +2114,6 @@ public abstract class SchemaBasicTest {
     Assert.assertEquals(tags, result.getTag());
   }
 
-  @Test
-  public void testAutoCreateAlignedTimeseriesWhileInsert() {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-
-    try {
-      long time = 1L;
-      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT32};
-
-      String[] columns = new String[2];
-      columns[0] = "1";
-      columns[1] = "2";
-
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.aligned_device"),
-              time,
-              new String[] {"s1", "s2"},
-              dataTypes,
-              columns,
-              true);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
-
-      assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.aligned_device.s1")));
-      assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.aligned_device.s2")));
-
-      insertRowPlan.setMeasurements(new String[] {"s3", "s4"});
-      schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
-      assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.aligned_device.s3")));
-      assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.aligned_device.s4")));
-
-      insertRowPlan.setMeasurements(new String[] {"s2", "s5"});
-      schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
-      assertTrue(schemaProcessor.isPathExist(new PartialPath("root.laptop.d1.aligned_device.s5")));
-
-      insertRowPlan.setMeasurements(new String[] {"s2", "s3"});
-      schemaProcessor.getSeriesSchemasAndReadLockDevice(insertRowPlan);
-
-    } catch (MetadataException | IOException e) {
-      fail();
-    }
-  }
-
   @Test
   public void testGetStorageGroupNodeByPath() {
     LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
@@ -2382,48 +2151,6 @@ public abstract class SchemaBasicTest {
     }
   }
 
-  @Test
-  public void testMeasurementIdWhileInsert() throws Exception {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-
-    PartialPath deviceId = new PartialPath("root.sg.d");
-    InsertPlan insertPlan;
-
-    insertPlan = getInsertPlan("`\"a+b\"`");
-    schemaProcessor.getSeriesSchemasAndReadLockDevice(insertPlan);
-    assertTrue(schemaProcessor.isPathExist(deviceId.concatNode("`\"a+b\"`")));
-
-    insertPlan = getInsertPlan("`\"a.b\"`");
-    schemaProcessor.getSeriesSchemasAndReadLockDevice(insertPlan);
-    assertTrue(schemaProcessor.isPathExist(deviceId.concatNode("`\"a.b\"`")));
-
-    insertPlan = getInsertPlan("`\"a“(Φ)”b\"`");
-    schemaProcessor.getSeriesSchemasAndReadLockDevice(insertPlan);
-    assertTrue(schemaProcessor.isPathExist(deviceId.concatNode("`\"a“(Φ)”b\"`")));
-
-    String[] illegalMeasurementIds = {"time", "timestamp", "TIME", "TIMESTAMP"};
-    for (String measurementId : illegalMeasurementIds) {
-      insertPlan = getInsertPlan(measurementId);
-      try {
-        schemaProcessor.getSeriesSchemasAndReadLockDevice(insertPlan);
-        assertFalse(schemaProcessor.isPathExist(deviceId.concatNode(measurementId)));
-      } catch (MetadataException e) {
-        e.printStackTrace();
-      }
-    }
-  }
-
-  private InsertPlan getInsertPlan(String measurementId) throws MetadataException {
-    PartialPath deviceId = new PartialPath("root.sg.d");
-    String[] measurementList = {measurementId};
-    String[] values = {"1"};
-    IMeasurementMNode[] measurementMNodes = new IMeasurementMNode[1];
-    InsertPlan insertPlan = new InsertRowPlan(deviceId, 1L, measurementList, values);
-    insertPlan.setMeasurementMNodes(measurementMNodes);
-    insertPlan.getDataTypes()[0] = TSDataType.INT32;
-    return insertPlan;
-  }
-
   @Test
   public void testTemplateSchemaNameCheckWhileCreate() {
     LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
@@ -2458,31 +2185,6 @@ public abstract class SchemaBasicTest {
         "template1", schemaNames, measurementList, dataTypeList, encodingList, compressionTypes);
   }
 
-  @Test
-  public void testDeviceNodeAfterAutoCreateTimeseriesFailure() throws Exception {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-
-    PartialPath sg1 = new PartialPath("root.a.sg");
-    schemaProcessor.setStorageGroup(sg1);
-
-    PartialPath deviceId = new PartialPath("root.a.d");
-    String[] measurementList = {"s"};
-    String[] values = {"1"};
-    IMeasurementMNode[] measurementMNodes = new IMeasurementMNode[1];
-    InsertPlan insertPlan = new InsertRowPlan(deviceId, 1L, measurementList, values);
-    insertPlan.setMeasurementMNodes(measurementMNodes);
-    insertPlan.getDataTypes()[0] = TSDataType.INT32;
-
-    try {
-      schemaProcessor.getSeriesSchemasAndReadLockDevice(insertPlan);
-      fail();
-    } catch (MetadataException e) {
-      Assert.assertEquals(
-          "some children of root.a have already been set to storage group", e.getMessage());
-      Assert.assertFalse(schemaProcessor.isPathExist(new PartialPath("root.a.d")));
-    }
-  }
-
   @Test
   @Ignore
   public void testTimeseriesDeletionWithEntityUsingTemplate() throws MetadataException {
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 819d8cdedb..41d8aff180 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
@@ -18,7 +18,6 @@
  */
 package org.apache.iotdb.db.metadata;
 
-import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.MeasurementPath;
 import org.apache.iotdb.commons.path.PartialPath;
@@ -27,7 +26,6 @@ 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.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
 import org.apache.iotdb.db.qp.physical.sys.ActivateTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.DropTemplatePlan;
@@ -476,16 +474,4 @@ public class TemplateTest {
           e.getMessage());
     }
   }
-
-  private InsertRowPlan getInsertRowPlan(String prefixPath, String measurement)
-      throws IllegalPathException {
-    long time = 110L;
-    TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT64};
-
-    String[] columns = new String[1];
-    columns[0] = "1";
-
-    return new InsertRowPlan(
-        new PartialPath(prefixPath), time, new String[] {measurement}, dataTypes, columns);
-  }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableLogFileTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableLogFileTest.java
index eb2b337518..cccadde68a 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableLogFileTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableLogFileTest.java
@@ -19,27 +19,6 @@
 
 package org.apache.iotdb.db.metadata.idtable;
 
-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.metadata.idtable.entry.DeviceIDFactory;
-import org.apache.iotdb.db.metadata.idtable.entry.DiskSchemaEntry;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 public class IDTableLogFileTest {
 
   private boolean isEnableIDTable = false;
@@ -48,113 +27,113 @@ public class IDTableLogFileTest {
 
   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 testInsertAndAutoCreate() {
-    try {
-      // construct an insertRowPlan with mismatched data type
-      long time = 1L;
-      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
-
-      String[] columns = new String[2];
-      columns[0] = "1";
-      columns[1] = "2";
-
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.non_aligned_device1"),
-              time,
-              new String[] {"s1", "s2"},
-              dataTypes,
-              columns,
-              false);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      // call getSeriesSchemasAndReadLockDevice
-      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
-      idTable.getSeriesSchemas(insertRowPlan);
-
-      insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d2.non_aligned_device2"),
-              time,
-              new String[] {"s3", "s4"},
-              dataTypes,
-              columns,
-              false);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      idTable.getSeriesSchemas(insertRowPlan);
-
-      // test part
-      HashMap<String, HashSet<String>> deviceIdToNameMap = new HashMap<>();
-      HashSet<String> d1Set = new HashSet<>();
-      d1Set.add("root.laptop.d1.non_aligned_device1.s1");
-      d1Set.add("root.laptop.d1.non_aligned_device1.s2");
-      HashSet<String> d2Set = new HashSet<>();
-      d2Set.add("root.laptop.d2.non_aligned_device2.s3");
-      d2Set.add("root.laptop.d2.non_aligned_device2.s4");
-
-      HashSet<String> resSet = new HashSet<>();
-
-      deviceIdToNameMap.put(
-          DeviceIDFactory.getInstance()
-              .getDeviceID(new PartialPath("root.laptop.d1.non_aligned_device1"))
-              .toStringID(),
-          d1Set);
-      deviceIdToNameMap.put(
-          DeviceIDFactory.getInstance()
-              .getDeviceID(new PartialPath("root.laptop.d2.non_aligned_device2"))
-              .toStringID(),
-          d2Set);
-
-      int count = 0;
-      for (DiskSchemaEntry entry : idTable.getIDiskSchemaManager().getAllSchemaEntry()) {
-        String deviceID = entry.deviceID;
-        HashSet<String> set = deviceIdToNameMap.get(deviceID);
-
-        if (set == null) {
-          fail("device path is not correct " + deviceID);
-        }
-
-        if (!set.contains(entry.seriesKey)) {
-          fail("series path is not correct " + entry.seriesKey);
-        }
-        count++;
-        resSet.add(entry.seriesKey);
-      }
-
-      assertEquals(4, count);
-      assertEquals(4, resSet.size());
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail("throw exception");
-    }
-  }
+  //  @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 testInsertAndAutoCreate() {
+  //    try {
+  //      // construct an insertRowPlan with mismatched data type
+  //      long time = 1L;
+  //      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
+  //
+  //      String[] columns = new String[2];
+  //      columns[0] = "1";
+  //      columns[1] = "2";
+  //
+  //      InsertRowPlan insertRowPlan =
+  //          new InsertRowPlan(
+  //              new PartialPath("root.laptop.d1.non_aligned_device1"),
+  //              time,
+  //              new String[] {"s1", "s2"},
+  //              dataTypes,
+  //              columns,
+  //              false);
+  //      insertRowPlan.setMeasurementMNodes(
+  //          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+  //
+  //      // call getSeriesSchemasAndReadLockDevice
+  //      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+  //
+  //      idTable.getSeriesSchemas(insertRowPlan);
+  //
+  //      insertRowPlan =
+  //          new InsertRowPlan(
+  //              new PartialPath("root.laptop.d2.non_aligned_device2"),
+  //              time,
+  //              new String[] {"s3", "s4"},
+  //              dataTypes,
+  //              columns,
+  //              false);
+  //      insertRowPlan.setMeasurementMNodes(
+  //          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+  //
+  //      idTable.getSeriesSchemas(insertRowPlan);
+  //
+  //      // test part
+  //      HashMap<String, HashSet<String>> deviceIdToNameMap = new HashMap<>();
+  //      HashSet<String> d1Set = new HashSet<>();
+  //      d1Set.add("root.laptop.d1.non_aligned_device1.s1");
+  //      d1Set.add("root.laptop.d1.non_aligned_device1.s2");
+  //      HashSet<String> d2Set = new HashSet<>();
+  //      d2Set.add("root.laptop.d2.non_aligned_device2.s3");
+  //      d2Set.add("root.laptop.d2.non_aligned_device2.s4");
+  //
+  //      HashSet<String> resSet = new HashSet<>();
+  //
+  //      deviceIdToNameMap.put(
+  //          DeviceIDFactory.getInstance()
+  //              .getDeviceID(new PartialPath("root.laptop.d1.non_aligned_device1"))
+  //              .toStringID(),
+  //          d1Set);
+  //      deviceIdToNameMap.put(
+  //          DeviceIDFactory.getInstance()
+  //              .getDeviceID(new PartialPath("root.laptop.d2.non_aligned_device2"))
+  //              .toStringID(),
+  //          d2Set);
+  //
+  //      int count = 0;
+  //      for (DiskSchemaEntry entry : idTable.getIDiskSchemaManager().getAllSchemaEntry()) {
+  //        String deviceID = entry.deviceID;
+  //        HashSet<String> set = deviceIdToNameMap.get(deviceID);
+  //
+  //        if (set == null) {
+  //          fail("device path is not correct " + deviceID);
+  //        }
+  //
+  //        if (!set.contains(entry.seriesKey)) {
+  //          fail("series path is not correct " + entry.seriesKey);
+  //        }
+  //        count++;
+  //        resSet.add(entry.seriesKey);
+  //      }
+  //
+  //      assertEquals(4, count);
+  //      assertEquals(4, resSet.size());
+  //    } catch (Exception e) {
+  //      e.printStackTrace();
+  //      fail("throw exception");
+  //    }
+  //  }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableTest.java
index 2ea9f8cd35..836fc89092 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableTest.java
@@ -18,737 +18,748 @@
  */
 
 package org.apache.iotdb.db.metadata.idtable;
-
-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.trigger.service.TriggerRegistrationService;
-import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
-import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
-import org.apache.iotdb.db.metadata.idtable.entry.DiskSchemaEntry;
-import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
-import org.apache.iotdb.db.metadata.idtable.entry.SchemaEntry;
-import org.apache.iotdb.db.metadata.lastCache.container.ILastCacheContainer;
-import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
-import org.apache.iotdb.db.qp.Planner;
-import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateTriggerPlan;
-import org.apache.iotdb.db.qp.physical.sys.DropTriggerPlan;
-import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
-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.utils.Pair;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class IDTableTest {
-
-  private CompressionType compressionType;
-
-  private boolean isEnableIDTable = false;
-
-  private String originalDeviceIDTransformationMethod = null;
-
-  private boolean isEnableIDTableLogFile = false;
-
-  @Before
-  public void before() {
-    compressionType = TSFileDescriptor.getInstance().getConfig().getCompressor();
-    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 testCreateAlignedTimeseriesAndInsert() {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-
-    try {
-      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
-      CreateAlignedTimeSeriesPlan plan =
-          new CreateAlignedTimeSeriesPlan(
-              new PartialPath("root.laptop.d1.aligned_device"),
-              Arrays.asList("s1", "s2", "s3"),
-              Arrays.asList(
-                  TSDataType.valueOf("FLOAT"),
-                  TSDataType.valueOf("INT64"),
-                  TSDataType.valueOf("INT32")),
-              Arrays.asList(
-                  TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE")),
-              Arrays.asList(compressionType, compressionType, compressionType),
-              null,
-              null,
-              null);
-
-      schemaProcessor.createAlignedTimeSeries(plan);
-
-      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
-      // construct an insertRowPlan with mismatched data type
-      long time = 1L;
-      TSDataType[] dataTypes =
-          new TSDataType[] {TSDataType.FLOAT, TSDataType.INT64, TSDataType.INT32};
-
-      String[] columns = new String[3];
-      columns[0] = 2.0 + "";
-      columns[1] = 10000 + "";
-      columns[2] = 100 + "";
-
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.aligned_device"),
-              time,
-              new String[] {"s1", "s2", "s3"},
-              dataTypes,
-              columns,
-              true);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      idTable.getSeriesSchemas(insertRowPlan);
-
-      // with type mismatch
-      dataTypes = new TSDataType[] {TSDataType.FLOAT, TSDataType.DOUBLE, TSDataType.INT32};
-      InsertRowPlan insertRowPlan2 =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.aligned_device"),
-              time,
-              new String[] {"s1", "s2", "s3"},
-              dataTypes,
-              columns,
-              true);
-      insertRowPlan2.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      // we should throw type mismatch exception here
-      try {
-        IoTDBDescriptor.getInstance().getConfig().setEnablePartialInsert(false);
-        idTable.getSeriesSchemas(insertRowPlan2);
-        fail("should throw exception");
-      } catch (DataTypeMismatchException e) {
-        assertEquals(
-            "data type of root.laptop.d1.aligned_device.s2 is not consistent, registered type INT64, inserting type DOUBLE, timestamp 1, value 10000.0",
-            e.getMessage());
-      } catch (Exception e2) {
-        fail("throw wrong exception");
-      }
-
-      IoTDBDescriptor.getInstance().getConfig().setEnablePartialInsert(true);
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail(e.getMessage());
-    }
-  }
-
-  @Test
-  public void testCreateAlignedTimeseriesAndInsertNotAlignedData() {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-
-    try {
-      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
-      CreateAlignedTimeSeriesPlan plan =
-          new CreateAlignedTimeSeriesPlan(
-              new PartialPath("root.laptop.d1.aligned_device"),
-              Arrays.asList("s1", "s2", "s3"),
-              Arrays.asList(
-                  TSDataType.valueOf("FLOAT"),
-                  TSDataType.valueOf("INT64"),
-                  TSDataType.valueOf("INT32")),
-              Arrays.asList(
-                  TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE")),
-              Arrays.asList(compressionType, compressionType, compressionType),
-              null,
-              null,
-              null);
-
-      schemaProcessor.createAlignedTimeSeries(plan);
-
-      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
-      // construct an insertRowPlan with mismatched data type
-      long time = 1L;
-      TSDataType[] dataTypes =
-          new TSDataType[] {TSDataType.FLOAT, TSDataType.INT64, TSDataType.INT32};
-
-      String[] columns = new String[3];
-      columns[0] = 2.0 + "";
-      columns[1] = 10000 + "";
-      columns[2] = 100 + "";
-
-      // non aligned plan
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.aligned_device"),
-              time,
-              new String[] {"s1", "s2", "s3"},
-              dataTypes,
-              columns,
-              false);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      // call getSeriesSchemasAndReadLockDevice
-      try {
-        idTable.getSeriesSchemas(insertRowPlan);
-        fail("should throw exception");
-      } catch (MetadataException e) {
-        assertEquals(
-            "Timeseries under path [root.laptop.d1.aligned_device]'s align value is [true], which is not consistent with insert plan",
-            e.getMessage());
-      } catch (Exception e2) {
-        fail("throw wrong exception");
-      }
-
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail(e.getMessage());
-    }
-  }
-
-  @Test
-  public void testCreateTimeseriesAndInsert() {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-    try {
-      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
-      schemaProcessor.createTimeseries(
-          new PartialPath("root.laptop.d1.s0"),
-          TSDataType.valueOf("INT32"),
-          TSEncoding.valueOf("RLE"),
-          compressionType,
-          Collections.emptyMap());
-
-      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
-      long time = 1L;
-      String[] columns = new String[1];
-      columns[0] = 2 + "";
-
-      // correct insert plan
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1"),
-              time,
-              new String[] {"s0"},
-              new TSDataType[] {TSDataType.INT32},
-              columns);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      idTable.getSeriesSchemas(insertRowPlan);
-      assertEquals(insertRowPlan.getMeasurementMNodes()[0].getSchema().getType(), TSDataType.INT32);
-      assertEquals(0, insertRowPlan.getFailedMeasurementNumber());
-
-      // construct an insertRowPlan with mismatched data type
-      InsertRowPlan insertRowPlan2 =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1"),
-              time,
-              new String[] {"s0"},
-              new TSDataType[] {TSDataType.FLOAT},
-              columns);
-      insertRowPlan2.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      // get series schema
-      idTable.getSeriesSchemas(insertRowPlan2);
-      assertNull(insertRowPlan2.getMeasurementMNodes()[0]);
-      assertEquals(1, insertRowPlan2.getFailedMeasurementNumber());
-
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail(e.getMessage());
-    }
-  }
-
-  @Test
-  public void testCreateTimeseriesAndInsertWithAlignedData() {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-    try {
-      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
-      schemaProcessor.createTimeseries(
-          new PartialPath("root.laptop.d1.non_aligned_device.s1"),
-          TSDataType.valueOf("INT32"),
-          TSEncoding.valueOf("RLE"),
-          compressionType,
-          Collections.emptyMap());
-      schemaProcessor.createTimeseries(
-          new PartialPath("root.laptop.d1.non_aligned_device.s2"),
-          TSDataType.valueOf("INT64"),
-          TSEncoding.valueOf("RLE"),
-          compressionType,
-          Collections.emptyMap());
-
-      // construct an insertRowPlan with mismatched data type
-      long time = 1L;
-      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
-
-      String[] columns = new String[2];
-      columns[0] = "1";
-      columns[1] = "2";
-
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.non_aligned_device"),
-              time,
-              new String[] {"s1", "s2"},
-              dataTypes,
-              columns,
-              true);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      // call getSeriesSchemasAndReadLockDevice
-      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
-      try {
-        idTable.getSeriesSchemas(insertRowPlan);
-        fail("should throw exception");
-      } catch (MetadataException e) {
-        assertEquals(
-            "Timeseries under path [root.laptop.d1.non_aligned_device]'s align value is [false], which is not consistent with insert plan",
-            e.getMessage());
-      }
-    } catch (Exception e) {
-      fail("throw wrong exception");
-    }
-  }
-
-  @Test
-  public void testInsertAndAutoCreate() {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-    try {
-      // construct an insertRowPlan with mismatched data type
-      long time = 1L;
-      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
-
-      String[] columns = new String[2];
-      columns[0] = "1";
-      columns[1] = "2";
-
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.non_aligned_device"),
-              time,
-              new String[] {"s1", "s2"},
-              dataTypes,
-              columns,
-              false);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      // call getSeriesSchemasAndReadLockDevice
-      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
-      idTable.getSeriesSchemas(insertRowPlan);
-
-      // check SchemaProcessor
-      IMeasurementMNode s1Node =
-          schemaProcessor.getMeasurementMNode(
-              new PartialPath("root.laptop.d1.non_aligned_device.s1"));
-      assertEquals("s1", s1Node.getName());
-      assertEquals(TSDataType.INT32, s1Node.getSchema().getType());
-      IMeasurementMNode s2Node =
-          schemaProcessor.getMeasurementMNode(
-              new PartialPath("root.laptop.d1.non_aligned_device.s2"));
-      assertEquals("s2", s2Node.getName());
-      assertEquals(TSDataType.INT64, s2Node.getSchema().getType());
-
-      // insert type mismatch data
-      InsertRowPlan insertRowPlan2 =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.non_aligned_device"),
-              time,
-              new String[] {"s1", "s2"},
-              new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
-              columns,
-              false);
-      insertRowPlan2.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      idTable.getSeriesSchemas(insertRowPlan2);
-
-      assertNull(insertRowPlan2.getMeasurementMNodes()[0]);
-      assertEquals(insertRowPlan.getMeasurementMNodes()[1].getSchema().getType(), TSDataType.INT64);
-      assertEquals(1, insertRowPlan2.getFailedMeasurementNumber());
-
-      // insert aligned data
-      InsertRowPlan insertRowPlan3 =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.non_aligned_device"),
-              time,
-              new String[] {"s1", "s2"},
-              new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
-              columns,
-              true);
-      insertRowPlan3.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      try {
-        idTable.getSeriesSchemas(insertRowPlan3);
-        fail("should throw exception");
-      } catch (MetadataException e) {
-        assertEquals(
-            "Timeseries under path [root.laptop.d1.non_aligned_device]'s align value is [false], which is not consistent with insert plan",
-            e.getMessage());
-      } catch (Exception e) {
-        fail("throw wrong exception");
-      }
-    } catch (MetadataException e) {
-      e.printStackTrace();
-      fail("throw exception");
-    }
-  }
-
-  @Test
-  public void testAlignedInsertAndAutoCreate() {
-    LocalSchemaProcessor processor = IoTDB.schemaProcessor;
-    try {
-      // construct an insertRowPlan with mismatched data type
-      long time = 1L;
-      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
-
-      String[] columns = new String[2];
-      columns[0] = "1";
-      columns[1] = "2";
-
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.aligned_device"),
-              time,
-              new String[] {"s1", "s2"},
-              dataTypes,
-              columns,
-              true);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      // call getSeriesSchemasAndReadLockDevice
-      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
-      idTable.getSeriesSchemas(insertRowPlan);
-
-      // check SchemaProcessor
-      IMeasurementMNode s1Node =
-          processor.getMeasurementMNode(new PartialPath("root.laptop.d1.aligned_device.s1"));
-      assertEquals("s1", s1Node.getName());
-      assertEquals(TSDataType.INT32, s1Node.getSchema().getType());
-      IMeasurementMNode s2Node =
-          processor.getMeasurementMNode(new PartialPath("root.laptop.d1.aligned_device.s2"));
-      assertEquals("s2", s2Node.getName());
-      assertEquals(TSDataType.INT64, s2Node.getSchema().getType());
-      assertTrue(s2Node.getParent().isAligned());
-
-      // insert type mismatch data
-      InsertRowPlan insertRowPlan2 =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.aligned_device"),
-              time,
-              new String[] {"s1", "s2"},
-              new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
-              columns,
-              true);
-      insertRowPlan2.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      idTable.getSeriesSchemas(insertRowPlan2);
-
-      assertNull(insertRowPlan2.getMeasurementMNodes()[0]);
-      assertEquals(insertRowPlan.getMeasurementMNodes()[1].getSchema().getType(), TSDataType.INT64);
-      assertEquals(1, insertRowPlan2.getFailedMeasurementNumber());
-
-      // insert non-aligned data
-      InsertRowPlan insertRowPlan3 =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.aligned_device"),
-              time,
-              new String[] {"s1", "s2"},
-              new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
-              columns,
-              false);
-      insertRowPlan3.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      try {
-        idTable.getSeriesSchemas(insertRowPlan3);
-        fail("should throw exception");
-      } catch (MetadataException e) {
-        assertEquals(
-            "Timeseries under path [root.laptop.d1.aligned_device]'s align value is [true], which is not consistent with insert plan",
-            e.getMessage());
-      } catch (Exception e) {
-        fail("throw wrong exception");
-      }
-    } catch (MetadataException e) {
-      e.printStackTrace();
-      fail("throw exception");
-    }
-  }
-
-  @Test
-  public void testTriggerAndInsert() {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-    try {
-      long time = 1L;
-
-      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
-      schemaProcessor.createTimeseries(
-          new PartialPath("root.laptop.d1.non_aligned_device.s1"),
-          TSDataType.valueOf("INT32"),
-          TSEncoding.valueOf("RLE"),
-          compressionType,
-          Collections.emptyMap());
-      schemaProcessor.createTimeseries(
-          new PartialPath("root.laptop.d1.non_aligned_device.s2"),
-          TSDataType.valueOf("INT64"),
-          TSEncoding.valueOf("RLE"),
-          compressionType,
-          Collections.emptyMap());
-
-      Planner processor = new Planner();
-
-      String sql =
-          "CREATE TRIGGER trigger1 BEFORE INSERT ON root.laptop.d1.non_aligned_device.s1 AS 'org.apache.iotdb.db.metadata.idtable.trigger_example.Counter'";
-
-      CreateTriggerPlan plan = (CreateTriggerPlan) processor.parseSQLToPhysicalPlan(sql);
-
-      TriggerRegistrationService.getInstance().register(plan);
-
-      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
-      String[] columns = new String[2];
-      columns[0] = "1";
-      columns[1] = "2";
-
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.non_aligned_device"),
-              time,
-              new String[] {"s1", "s2"},
-              dataTypes,
-              columns,
-              false);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      // call getSeriesSchemasAndReadLockDevice
-      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
-      idTable.getSeriesSchemas(insertRowPlan);
-
-      // check SchemaProcessor
-      IMeasurementMNode s1Node =
-          schemaProcessor.getMeasurementMNode(
-              new PartialPath("root.laptop.d1.non_aligned_device.s1"));
-      assertEquals("s1", s1Node.getName());
-      assertEquals(TSDataType.INT32, s1Node.getSchema().getType());
-      assertNotNull(s1Node.getTriggerExecutor());
-
-      IMeasurementMNode s2Node =
-          schemaProcessor.getMeasurementMNode(
-              new PartialPath("root.laptop.d1.non_aligned_device.s2"));
-      assertEquals("s2", s2Node.getName());
-      assertEquals(TSDataType.INT64, s2Node.getSchema().getType());
-      assertNull(s2Node.getTriggerExecutor());
-
-      // drop trigger
-      String sql2 = "Drop trigger trigger1";
-
-      DropTriggerPlan plan2 = (DropTriggerPlan) processor.parseSQLToPhysicalPlan(sql2);
-      TriggerRegistrationService.getInstance().deregister(plan2);
-
-      idTable.getSeriesSchemas(insertRowPlan);
-      assertNull(s1Node.getTriggerExecutor());
-    } catch (MetadataException | StorageEngineException | QueryProcessException e) {
-      e.printStackTrace();
-      fail("throw exception");
-    }
-  }
-
-  @Test
-  public void testGetDiskSchemaEntries() {
-    try {
-      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-      String sgPath = "root.laptop";
-      for (int i = 0; i < 10; i++) {
-        String devicePath = sgPath + ".d" + i;
-        IDeviceID iDeviceID = DeviceIDFactory.getInstance().getDeviceID(devicePath);
-        String measurement = "s" + i;
-        idTable.putSchemaEntry(
-            devicePath,
-            measurement,
-            new SchemaEntry(
-                TSDataType.BOOLEAN,
-                TSEncoding.BITMAP,
-                CompressionType.UNCOMPRESSED,
-                iDeviceID,
-                new PartialPath(devicePath + "." + measurement),
-                false,
-                idTable.getIDiskSchemaManager()),
-            false);
-        SchemaEntry schemaEntry =
-            idTable.getDeviceEntry(iDeviceID.toStringID()).getSchemaEntry(measurement);
-        List<SchemaEntry> schemaEntries = new ArrayList<>();
-        schemaEntries.add(schemaEntry);
-        List<DiskSchemaEntry> diskSchemaEntries = idTable.getDiskSchemaEntries(schemaEntries);
-        assertNotNull(diskSchemaEntries);
-        assertEquals(diskSchemaEntries.size(), 1);
-        assertEquals(diskSchemaEntries.get(0).seriesKey, devicePath + "." + measurement);
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail("throw exception");
-    }
-  }
-
-  @Test
-  public void testDeleteTimeseries() {
-    try {
-      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-      String sgPath = "root.laptop";
-      for (int i = 0; i < 10; i++) {
-        String devicePath = sgPath + ".d" + i;
-        IDeviceID iDeviceID = DeviceIDFactory.getInstance().getDeviceID(devicePath);
-        String measurement = "s" + i;
-        SchemaEntry schemaEntry =
-            new SchemaEntry(
-                TSDataType.BOOLEAN,
-                TSEncoding.BITMAP,
-                CompressionType.UNCOMPRESSED,
-                iDeviceID,
-                new PartialPath(devicePath + "." + measurement),
-                false,
-                idTable.getIDiskSchemaManager());
-        idTable.putSchemaEntry(devicePath, measurement, schemaEntry, false);
-      }
-      List<PartialPath> partialPaths = new ArrayList<>();
-      partialPaths.add(new PartialPath("root.laptop.d0.s0"));
-      partialPaths.add(new PartialPath("root.laptop.d8.s8"));
-      partialPaths.add(new PartialPath("root.laptop.d2.s3"));
-      Pair<Integer, Set<String>> pairs = idTable.deleteTimeseries(partialPaths);
-      assertNotNull(pairs);
-      assertEquals((int) pairs.left, 2);
-      assertTrue(pairs.right.contains("root.laptop.d2.s3"));
-      assertFalse(pairs.right.contains("root.laptop.d0.s0"));
-      assertFalse(pairs.right.contains("root.laptop.d8.s8"));
-      Collection<DiskSchemaEntry> diskSchemaEntries =
-          idTable.getIDiskSchemaManager().getAllSchemaEntry();
-      for (DiskSchemaEntry diskSchemaEntry : diskSchemaEntries) {
-        assertNotEquals("root.laptop.d0.s0", diskSchemaEntry.seriesKey);
-        assertNotEquals("root.laptop.d8.s8", diskSchemaEntry.seriesKey);
-      }
-      assertNull(idTable.getDeviceEntry("root.laptop.d0").getMeasurementMap().get("s0"));
-      assertNull(idTable.getDeviceEntry("root.laptop.d8").getMeasurementMap().get("s1"));
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail("throw exception");
-    }
-  }
-
-  @Test
-  public void testFlushTimeAndLastCache() {
-    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
-    try {
-      long time = 1L;
-
-      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
-      schemaProcessor.createTimeseries(
-          new PartialPath("root.laptop.d1.non_aligned_device.s1"),
-          TSDataType.valueOf("INT32"),
-          TSEncoding.valueOf("RLE"),
-          compressionType,
-          Collections.emptyMap());
-      schemaProcessor.createTimeseries(
-          new PartialPath("root.laptop.d1.non_aligned_device.s2"),
-          TSDataType.valueOf("INT64"),
-          TSEncoding.valueOf("RLE"),
-          compressionType,
-          Collections.emptyMap());
-
-      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
-      String[] columns = new String[2];
-      columns[0] = "1";
-      columns[1] = "2";
-
-      InsertRowPlan insertRowPlan =
-          new InsertRowPlan(
-              new PartialPath("root.laptop.d1.non_aligned_device"),
-              time,
-              new String[] {"s1", "s2"},
-              dataTypes,
-              columns,
-              false);
-      insertRowPlan.setMeasurementMNodes(
-          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
-
-      // call getSeriesSchemasAndReadLockDevice
-      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
-
-      idTable.getSeriesSchemas(insertRowPlan);
-
-      IMeasurementMNode s2Node = insertRowPlan.getMeasurementMNodes()[1];
-      ILastCacheContainer cacheContainer = s2Node.getLastCacheContainer();
-      // last cache
-      cacheContainer.updateCachedLast(
-          new TimeValuePair(100L, new TsPrimitiveType.TsLong(1L)), false, 0L);
-      assertEquals(new TsPrimitiveType.TsLong(1L), cacheContainer.getCachedLast().getValue());
-      assertEquals(100L, cacheContainer.getCachedLast().getTimestamp());
-
-      cacheContainer.updateCachedLast(
-          new TimeValuePair(90L, new TsPrimitiveType.TsLong(2L)), false, 0L);
-      assertEquals(new TsPrimitiveType.TsLong(1L), cacheContainer.getCachedLast().getValue());
-      assertEquals(100L, cacheContainer.getCachedLast().getTimestamp());
-
-      cacheContainer.updateCachedLast(
-          new TimeValuePair(110L, new TsPrimitiveType.TsLong(2L)), false, 0L);
-      assertEquals(new TsPrimitiveType.TsLong(2L), cacheContainer.getCachedLast().getValue());
-      assertEquals(110L, cacheContainer.getCachedLast().getTimestamp());
-
-    } catch (MetadataException e) {
-      e.printStackTrace();
-      fail("throw exception");
-    }
-  }
-}
+//
+// 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.trigger.service.TriggerRegistrationService;
+// import org.apache.iotdb.db.exception.StorageEngineException;
+// import org.apache.iotdb.db.exception.metadata.DataTypeMismatchException;
+// import org.apache.iotdb.db.exception.query.QueryProcessException;
+// import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
+// import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
+// import org.apache.iotdb.db.metadata.idtable.entry.DiskSchemaEntry;
+// import org.apache.iotdb.db.metadata.idtable.entry.IDeviceID;
+// import org.apache.iotdb.db.metadata.idtable.entry.SchemaEntry;
+// import org.apache.iotdb.db.metadata.lastCache.container.ILastCacheContainer;
+// import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+// import org.apache.iotdb.db.qp.Planner;
+// import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
+// import org.apache.iotdb.db.qp.physical.sys.CreateAlignedTimeSeriesPlan;
+// import org.apache.iotdb.db.qp.physical.sys.CreateTriggerPlan;
+// import org.apache.iotdb.db.qp.physical.sys.DropTriggerPlan;
+// import org.apache.iotdb.db.service.IoTDB;
+// import org.apache.iotdb.db.utils.EnvironmentUtils;
+// import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+// 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.utils.Pair;
+// import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+//
+// import org.junit.After;
+// import org.junit.Before;
+// import org.junit.Test;
+//
+// import java.io.IOException;
+// import java.util.ArrayList;
+// import java.util.Arrays;
+// import java.util.Collection;
+// import java.util.Collections;
+// import java.util.List;
+// import java.util.Set;
+//
+// import static org.junit.Assert.assertEquals;
+// import static org.junit.Assert.assertFalse;
+// import static org.junit.Assert.assertNotEquals;
+// import static org.junit.Assert.assertNotNull;
+// import static org.junit.Assert.assertNull;
+// import static org.junit.Assert.assertTrue;
+// import static org.junit.Assert.fail;
+//
+// public class IDTableTest {
+//
+//  private CompressionType compressionType;
+//
+//  private boolean isEnableIDTable = false;
+//
+//  private String originalDeviceIDTransformationMethod = null;
+//
+//  private boolean isEnableIDTableLogFile = false;
+//
+//  @Before
+//  public void before() {
+//    compressionType = TSFileDescriptor.getInstance().getConfig().getCompressor();
+//    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 testCreateAlignedTimeseriesAndInsert() {
+//    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+//
+//    try {
+//      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
+//      CreateAlignedTimeSeriesPlan plan =
+//          new CreateAlignedTimeSeriesPlan(
+//              new PartialPath("root.laptop.d1.aligned_device"),
+//              Arrays.asList("s1", "s2", "s3"),
+//              Arrays.asList(
+//                  TSDataType.valueOf("FLOAT"),
+//                  TSDataType.valueOf("INT64"),
+//                  TSDataType.valueOf("INT32")),
+//              Arrays.asList(
+//                  TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"),
+// TSEncoding.valueOf("RLE")),
+//              Arrays.asList(compressionType, compressionType, compressionType),
+//              null,
+//              null,
+//              null);
+//
+//      schemaProcessor.createAlignedTimeSeries(plan);
+//
+//      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+//      // construct an insertRowPlan with mismatched data type
+//      long time = 1L;
+//      TSDataType[] dataTypes =
+//          new TSDataType[] {TSDataType.FLOAT, TSDataType.INT64, TSDataType.INT32};
+//
+//      String[] columns = new String[3];
+//      columns[0] = 2.0 + "";
+//      columns[1] = 10000 + "";
+//      columns[2] = 100 + "";
+//
+//      InsertRowPlan insertRowPlan =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1.aligned_device"),
+//              time,
+//              new String[] {"s1", "s2", "s3"},
+//              dataTypes,
+//              columns,
+//              true);
+//      insertRowPlan.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      idTable.getSeriesSchemas(insertRowPlan);
+//
+//      // with type mismatch
+//      dataTypes = new TSDataType[] {TSDataType.FLOAT, TSDataType.DOUBLE, TSDataType.INT32};
+//      InsertRowPlan insertRowPlan2 =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1.aligned_device"),
+//              time,
+//              new String[] {"s1", "s2", "s3"},
+//              dataTypes,
+//              columns,
+//              true);
+//      insertRowPlan2.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      // we should throw type mismatch exception here
+//      try {
+//        IoTDBDescriptor.getInstance().getConfig().setEnablePartialInsert(false);
+//        idTable.getSeriesSchemas(insertRowPlan2);
+//        fail("should throw exception");
+//      } catch (DataTypeMismatchException e) {
+//        assertEquals(
+//            "data type of root.laptop.d1.aligned_device.s2 is not consistent, registered type
+// INT64, inserting type DOUBLE, timestamp 1, value 10000.0",
+//            e.getMessage());
+//      } catch (Exception e2) {
+//        fail("throw wrong exception");
+//      }
+//
+//      IoTDBDescriptor.getInstance().getConfig().setEnablePartialInsert(true);
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//      fail(e.getMessage());
+//    }
+//  }
+//
+//  @Test
+//  public void testCreateAlignedTimeseriesAndInsertNotAlignedData() {
+//    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+//
+//    try {
+//      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
+//      CreateAlignedTimeSeriesPlan plan =
+//          new CreateAlignedTimeSeriesPlan(
+//              new PartialPath("root.laptop.d1.aligned_device"),
+//              Arrays.asList("s1", "s2", "s3"),
+//              Arrays.asList(
+//                  TSDataType.valueOf("FLOAT"),
+//                  TSDataType.valueOf("INT64"),
+//                  TSDataType.valueOf("INT32")),
+//              Arrays.asList(
+//                  TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"),
+// TSEncoding.valueOf("RLE")),
+//              Arrays.asList(compressionType, compressionType, compressionType),
+//              null,
+//              null,
+//              null);
+//
+//      schemaProcessor.createAlignedTimeSeries(plan);
+//
+//      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+//      // construct an insertRowPlan with mismatched data type
+//      long time = 1L;
+//      TSDataType[] dataTypes =
+//          new TSDataType[] {TSDataType.FLOAT, TSDataType.INT64, TSDataType.INT32};
+//
+//      String[] columns = new String[3];
+//      columns[0] = 2.0 + "";
+//      columns[1] = 10000 + "";
+//      columns[2] = 100 + "";
+//
+//      // non aligned plan
+//      InsertRowPlan insertRowPlan =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1.aligned_device"),
+//              time,
+//              new String[] {"s1", "s2", "s3"},
+//              dataTypes,
+//              columns,
+//              false);
+//      insertRowPlan.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      // call getSeriesSchemasAndReadLockDevice
+//      try {
+//        idTable.getSeriesSchemas(insertRowPlan);
+//        fail("should throw exception");
+//      } catch (MetadataException e) {
+//        assertEquals(
+//            "Timeseries under path [root.laptop.d1.aligned_device]'s align value is [true], which
+// is not consistent with insert plan",
+//            e.getMessage());
+//      } catch (Exception e2) {
+//        fail("throw wrong exception");
+//      }
+//
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//      fail(e.getMessage());
+//    }
+//  }
+//
+//  @Test
+//  public void testCreateTimeseriesAndInsert() {
+//    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+//    try {
+//      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
+//      schemaProcessor.createTimeseries(
+//          new PartialPath("root.laptop.d1.s0"),
+//          TSDataType.valueOf("INT32"),
+//          TSEncoding.valueOf("RLE"),
+//          compressionType,
+//          Collections.emptyMap());
+//
+//      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+//      long time = 1L;
+//      String[] columns = new String[1];
+//      columns[0] = 2 + "";
+//
+//      // correct insert plan
+//      InsertRowPlan insertRowPlan =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1"),
+//              time,
+//              new String[] {"s0"},
+//              new TSDataType[] {TSDataType.INT32},
+//              columns);
+//      insertRowPlan.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      idTable.getSeriesSchemas(insertRowPlan);
+//      assertEquals(insertRowPlan.getMeasurementMNodes()[0].getSchema().getType(),
+// TSDataType.INT32);
+//      assertEquals(0, insertRowPlan.getFailedMeasurementNumber());
+//
+//      // construct an insertRowPlan with mismatched data type
+//      InsertRowPlan insertRowPlan2 =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1"),
+//              time,
+//              new String[] {"s0"},
+//              new TSDataType[] {TSDataType.FLOAT},
+//              columns);
+//      insertRowPlan2.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      // get series schema
+//      idTable.getSeriesSchemas(insertRowPlan2);
+//      assertNull(insertRowPlan2.getMeasurementMNodes()[0]);
+//      assertEquals(1, insertRowPlan2.getFailedMeasurementNumber());
+//
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//      fail(e.getMessage());
+//    }
+//  }
+//
+//  @Test
+//  public void testCreateTimeseriesAndInsertWithAlignedData() {
+//    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+//    try {
+//      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
+//      schemaProcessor.createTimeseries(
+//          new PartialPath("root.laptop.d1.non_aligned_device.s1"),
+//          TSDataType.valueOf("INT32"),
+//          TSEncoding.valueOf("RLE"),
+//          compressionType,
+//          Collections.emptyMap());
+//      schemaProcessor.createTimeseries(
+//          new PartialPath("root.laptop.d1.non_aligned_device.s2"),
+//          TSDataType.valueOf("INT64"),
+//          TSEncoding.valueOf("RLE"),
+//          compressionType,
+//          Collections.emptyMap());
+//
+//      // construct an insertRowPlan with mismatched data type
+//      long time = 1L;
+//      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
+//
+//      String[] columns = new String[2];
+//      columns[0] = "1";
+//      columns[1] = "2";
+//
+//      InsertRowPlan insertRowPlan =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1.non_aligned_device"),
+//              time,
+//              new String[] {"s1", "s2"},
+//              dataTypes,
+//              columns,
+//              true);
+//      insertRowPlan.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      // call getSeriesSchemasAndReadLockDevice
+//      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+//      try {
+//        idTable.getSeriesSchemas(insertRowPlan);
+//        fail("should throw exception");
+//      } catch (MetadataException e) {
+//        assertEquals(
+//            "Timeseries under path [root.laptop.d1.non_aligned_device]'s align value is [false],
+// which is not consistent with insert plan",
+//            e.getMessage());
+//      }
+//    } catch (Exception e) {
+//      fail("throw wrong exception");
+//    }
+//  }
+//
+//  @Test
+//  public void testInsertAndAutoCreate() {
+//    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+//    try {
+//      // construct an insertRowPlan with mismatched data type
+//      long time = 1L;
+//      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
+//
+//      String[] columns = new String[2];
+//      columns[0] = "1";
+//      columns[1] = "2";
+//
+//      InsertRowPlan insertRowPlan =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1.non_aligned_device"),
+//              time,
+//              new String[] {"s1", "s2"},
+//              dataTypes,
+//              columns,
+//              false);
+//      insertRowPlan.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      // call getSeriesSchemasAndReadLockDevice
+//      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+//      idTable.getSeriesSchemas(insertRowPlan);
+//
+//      // check SchemaProcessor
+//      IMeasurementMNode s1Node =
+//          schemaProcessor.getMeasurementMNode(
+//              new PartialPath("root.laptop.d1.non_aligned_device.s1"));
+//      assertEquals("s1", s1Node.getName());
+//      assertEquals(TSDataType.INT32, s1Node.getSchema().getType());
+//      IMeasurementMNode s2Node =
+//          schemaProcessor.getMeasurementMNode(
+//              new PartialPath("root.laptop.d1.non_aligned_device.s2"));
+//      assertEquals("s2", s2Node.getName());
+//      assertEquals(TSDataType.INT64, s2Node.getSchema().getType());
+//
+//      // insert type mismatch data
+//      InsertRowPlan insertRowPlan2 =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1.non_aligned_device"),
+//              time,
+//              new String[] {"s1", "s2"},
+//              new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
+//              columns,
+//              false);
+//      insertRowPlan2.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      idTable.getSeriesSchemas(insertRowPlan2);
+//
+//      assertNull(insertRowPlan2.getMeasurementMNodes()[0]);
+//      assertEquals(insertRowPlan.getMeasurementMNodes()[1].getSchema().getType(),
+// TSDataType.INT64);
+//      assertEquals(1, insertRowPlan2.getFailedMeasurementNumber());
+//
+//      // insert aligned data
+//      InsertRowPlan insertRowPlan3 =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1.non_aligned_device"),
+//              time,
+//              new String[] {"s1", "s2"},
+//              new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
+//              columns,
+//              true);
+//      insertRowPlan3.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      try {
+//        idTable.getSeriesSchemas(insertRowPlan3);
+//        fail("should throw exception");
+//      } catch (MetadataException e) {
+//        assertEquals(
+//            "Timeseries under path [root.laptop.d1.non_aligned_device]'s align value is [false],
+// which is not consistent with insert plan",
+//            e.getMessage());
+//      } catch (Exception e) {
+//        fail("throw wrong exception");
+//      }
+//    } catch (MetadataException e) {
+//      e.printStackTrace();
+//      fail("throw exception");
+//    }
+//  }
+//
+//  @Test
+//  public void testAlignedInsertAndAutoCreate() {
+//    LocalSchemaProcessor processor = IoTDB.schemaProcessor;
+//    try {
+//      // construct an insertRowPlan with mismatched data type
+//      long time = 1L;
+//      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
+//
+//      String[] columns = new String[2];
+//      columns[0] = "1";
+//      columns[1] = "2";
+//
+//      InsertRowPlan insertRowPlan =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1.aligned_device"),
+//              time,
+//              new String[] {"s1", "s2"},
+//              dataTypes,
+//              columns,
+//              true);
+//      insertRowPlan.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      // call getSeriesSchemasAndReadLockDevice
+//      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+//      idTable.getSeriesSchemas(insertRowPlan);
+//
+//      // check SchemaProcessor
+//      IMeasurementMNode s1Node =
+//          processor.getMeasurementMNode(new PartialPath("root.laptop.d1.aligned_device.s1"));
+//      assertEquals("s1", s1Node.getName());
+//      assertEquals(TSDataType.INT32, s1Node.getSchema().getType());
+//      IMeasurementMNode s2Node =
+//          processor.getMeasurementMNode(new PartialPath("root.laptop.d1.aligned_device.s2"));
+//      assertEquals("s2", s2Node.getName());
+//      assertEquals(TSDataType.INT64, s2Node.getSchema().getType());
+//      assertTrue(s2Node.getParent().isAligned());
+//
+//      // insert type mismatch data
+//      InsertRowPlan insertRowPlan2 =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1.aligned_device"),
+//              time,
+//              new String[] {"s1", "s2"},
+//              new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
+//              columns,
+//              true);
+//      insertRowPlan2.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      idTable.getSeriesSchemas(insertRowPlan2);
+//
+//      assertNull(insertRowPlan2.getMeasurementMNodes()[0]);
+//      assertEquals(insertRowPlan.getMeasurementMNodes()[1].getSchema().getType(),
+// TSDataType.INT64);
+//      assertEquals(1, insertRowPlan2.getFailedMeasurementNumber());
+//
+//      // insert non-aligned data
+//      InsertRowPlan insertRowPlan3 =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1.aligned_device"),
+//              time,
+//              new String[] {"s1", "s2"},
+//              new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
+//              columns,
+//              false);
+//      insertRowPlan3.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      try {
+//        idTable.getSeriesSchemas(insertRowPlan3);
+//        fail("should throw exception");
+//      } catch (MetadataException e) {
+//        assertEquals(
+//            "Timeseries under path [root.laptop.d1.aligned_device]'s align value is [true], which
+// is not consistent with insert plan",
+//            e.getMessage());
+//      } catch (Exception e) {
+//        fail("throw wrong exception");
+//      }
+//    } catch (MetadataException e) {
+//      e.printStackTrace();
+//      fail("throw exception");
+//    }
+//  }
+//
+//  @Test
+//  public void testTriggerAndInsert() {
+//    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+//    try {
+//      long time = 1L;
+//
+//      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
+//      schemaProcessor.createTimeseries(
+//          new PartialPath("root.laptop.d1.non_aligned_device.s1"),
+//          TSDataType.valueOf("INT32"),
+//          TSEncoding.valueOf("RLE"),
+//          compressionType,
+//          Collections.emptyMap());
+//      schemaProcessor.createTimeseries(
+//          new PartialPath("root.laptop.d1.non_aligned_device.s2"),
+//          TSDataType.valueOf("INT64"),
+//          TSEncoding.valueOf("RLE"),
+//          compressionType,
+//          Collections.emptyMap());
+//
+//      Planner processor = new Planner();
+//
+//      String sql =
+//          "CREATE TRIGGER trigger1 BEFORE INSERT ON root.laptop.d1.non_aligned_device.s1 AS
+// 'org.apache.iotdb.db.metadata.idtable.trigger_example.Counter'";
+//
+//      CreateTriggerPlan plan = (CreateTriggerPlan) processor.parseSQLToPhysicalPlan(sql);
+//
+//      TriggerRegistrationService.getInstance().register(plan);
+//
+//      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
+//      String[] columns = new String[2];
+//      columns[0] = "1";
+//      columns[1] = "2";
+//
+//      InsertRowPlan insertRowPlan =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1.non_aligned_device"),
+//              time,
+//              new String[] {"s1", "s2"},
+//              dataTypes,
+//              columns,
+//              false);
+//      insertRowPlan.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      // call getSeriesSchemasAndReadLockDevice
+//      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+//      idTable.getSeriesSchemas(insertRowPlan);
+//
+//      // check SchemaProcessor
+//      IMeasurementMNode s1Node =
+//          schemaProcessor.getMeasurementMNode(
+//              new PartialPath("root.laptop.d1.non_aligned_device.s1"));
+//      assertEquals("s1", s1Node.getName());
+//      assertEquals(TSDataType.INT32, s1Node.getSchema().getType());
+//      assertNotNull(s1Node.getTriggerExecutor());
+//
+//      IMeasurementMNode s2Node =
+//          schemaProcessor.getMeasurementMNode(
+//              new PartialPath("root.laptop.d1.non_aligned_device.s2"));
+//      assertEquals("s2", s2Node.getName());
+//      assertEquals(TSDataType.INT64, s2Node.getSchema().getType());
+//      assertNull(s2Node.getTriggerExecutor());
+//
+//      // drop trigger
+//      String sql2 = "Drop trigger trigger1";
+//
+//      DropTriggerPlan plan2 = (DropTriggerPlan) processor.parseSQLToPhysicalPlan(sql2);
+//      TriggerRegistrationService.getInstance().deregister(plan2);
+//
+//      idTable.getSeriesSchemas(insertRowPlan);
+//      assertNull(s1Node.getTriggerExecutor());
+//    } catch (MetadataException | StorageEngineException | QueryProcessException e) {
+//      e.printStackTrace();
+//      fail("throw exception");
+//    }
+//  }
+//
+//  @Test
+//  public void testGetDiskSchemaEntries() {
+//    try {
+//      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//      String sgPath = "root.laptop";
+//      for (int i = 0; i < 10; i++) {
+//        String devicePath = sgPath + ".d" + i;
+//        IDeviceID iDeviceID = DeviceIDFactory.getInstance().getDeviceID(devicePath);
+//        String measurement = "s" + i;
+//        idTable.putSchemaEntry(
+//            devicePath,
+//            measurement,
+//            new SchemaEntry(
+//                TSDataType.BOOLEAN,
+//                TSEncoding.BITMAP,
+//                CompressionType.UNCOMPRESSED,
+//                iDeviceID,
+//                new PartialPath(devicePath + "." + measurement),
+//                false,
+//                idTable.getIDiskSchemaManager()),
+//            false);
+//        SchemaEntry schemaEntry =
+//            idTable.getDeviceEntry(iDeviceID.toStringID()).getSchemaEntry(measurement);
+//        List<SchemaEntry> schemaEntries = new ArrayList<>();
+//        schemaEntries.add(schemaEntry);
+//        List<DiskSchemaEntry> diskSchemaEntries = idTable.getDiskSchemaEntries(schemaEntries);
+//        assertNotNull(diskSchemaEntries);
+//        assertEquals(diskSchemaEntries.size(), 1);
+//        assertEquals(diskSchemaEntries.get(0).seriesKey, devicePath + "." + measurement);
+//      }
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//      fail("throw exception");
+//    }
+//  }
+//
+//  @Test
+//  public void testDeleteTimeseries() {
+//    try {
+//      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//      String sgPath = "root.laptop";
+//      for (int i = 0; i < 10; i++) {
+//        String devicePath = sgPath + ".d" + i;
+//        IDeviceID iDeviceID = DeviceIDFactory.getInstance().getDeviceID(devicePath);
+//        String measurement = "s" + i;
+//        SchemaEntry schemaEntry =
+//            new SchemaEntry(
+//                TSDataType.BOOLEAN,
+//                TSEncoding.BITMAP,
+//                CompressionType.UNCOMPRESSED,
+//                iDeviceID,
+//                new PartialPath(devicePath + "." + measurement),
+//                false,
+//                idTable.getIDiskSchemaManager());
+//        idTable.putSchemaEntry(devicePath, measurement, schemaEntry, false);
+//      }
+//      List<PartialPath> partialPaths = new ArrayList<>();
+//      partialPaths.add(new PartialPath("root.laptop.d0.s0"));
+//      partialPaths.add(new PartialPath("root.laptop.d8.s8"));
+//      partialPaths.add(new PartialPath("root.laptop.d2.s3"));
+//      Pair<Integer, Set<String>> pairs = idTable.deleteTimeseries(partialPaths);
+//      assertNotNull(pairs);
+//      assertEquals((int) pairs.left, 2);
+//      assertTrue(pairs.right.contains("root.laptop.d2.s3"));
+//      assertFalse(pairs.right.contains("root.laptop.d0.s0"));
+//      assertFalse(pairs.right.contains("root.laptop.d8.s8"));
+//      Collection<DiskSchemaEntry> diskSchemaEntries =
+//          idTable.getIDiskSchemaManager().getAllSchemaEntry();
+//      for (DiskSchemaEntry diskSchemaEntry : diskSchemaEntries) {
+//        assertNotEquals("root.laptop.d0.s0", diskSchemaEntry.seriesKey);
+//        assertNotEquals("root.laptop.d8.s8", diskSchemaEntry.seriesKey);
+//      }
+//      assertNull(idTable.getDeviceEntry("root.laptop.d0").getMeasurementMap().get("s0"));
+//      assertNull(idTable.getDeviceEntry("root.laptop.d8").getMeasurementMap().get("s1"));
+//    } catch (Exception e) {
+//      e.printStackTrace();
+//      fail("throw exception");
+//    }
+//  }
+//
+//  @Test
+//  public void testFlushTimeAndLastCache() {
+//    LocalSchemaProcessor schemaProcessor = IoTDB.schemaProcessor;
+//    try {
+//      long time = 1L;
+//
+//      schemaProcessor.setStorageGroup(new PartialPath("root.laptop"));
+//      schemaProcessor.createTimeseries(
+//          new PartialPath("root.laptop.d1.non_aligned_device.s1"),
+//          TSDataType.valueOf("INT32"),
+//          TSEncoding.valueOf("RLE"),
+//          compressionType,
+//          Collections.emptyMap());
+//      schemaProcessor.createTimeseries(
+//          new PartialPath("root.laptop.d1.non_aligned_device.s2"),
+//          TSDataType.valueOf("INT64"),
+//          TSEncoding.valueOf("RLE"),
+//          compressionType,
+//          Collections.emptyMap());
+//
+//      TSDataType[] dataTypes = new TSDataType[] {TSDataType.INT32, TSDataType.INT64};
+//      String[] columns = new String[2];
+//      columns[0] = "1";
+//      columns[1] = "2";
+//
+//      InsertRowPlan insertRowPlan =
+//          new InsertRowPlan(
+//              new PartialPath("root.laptop.d1.non_aligned_device"),
+//              time,
+//              new String[] {"s1", "s2"},
+//              dataTypes,
+//              columns,
+//              false);
+//      insertRowPlan.setMeasurementMNodes(
+//          new IMeasurementMNode[insertRowPlan.getMeasurements().length]);
+//
+//      // call getSeriesSchemasAndReadLockDevice
+//      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath("root.laptop"));
+//
+//      idTable.getSeriesSchemas(insertRowPlan);
+//
+//      IMeasurementMNode s2Node = insertRowPlan.getMeasurementMNodes()[1];
+//      ILastCacheContainer cacheContainer = s2Node.getLastCacheContainer();
+//      // last cache
+//      cacheContainer.updateCachedLast(
+//          new TimeValuePair(100L, new TsPrimitiveType.TsLong(1L)), false, 0L);
+//      assertEquals(new TsPrimitiveType.TsLong(1L), cacheContainer.getCachedLast().getValue());
+//      assertEquals(100L, cacheContainer.getCachedLast().getTimestamp());
+//
+//      cacheContainer.updateCachedLast(
+//          new TimeValuePair(90L, new TsPrimitiveType.TsLong(2L)), false, 0L);
+//      assertEquals(new TsPrimitiveType.TsLong(1L), cacheContainer.getCachedLast().getValue());
+//      assertEquals(100L, cacheContainer.getCachedLast().getTimestamp());
+//
+//      cacheContainer.updateCachedLast(
+//          new TimeValuePair(110L, new TsPrimitiveType.TsLong(2L)), false, 0L);
+//      assertEquals(new TsPrimitiveType.TsLong(2L), cacheContainer.getCachedLast().getValue());
+//      assertEquals(110L, cacheContainer.getCachedLast().getTimestamp());
+//
+//    } catch (MetadataException e) {
+//      e.printStackTrace();
+//      fail("throw exception");
+//    }
+//  }
+// }
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/PlannerTest.java b/server/src/test/java/org/apache/iotdb/db/qp/PlannerTest.java
index 9a621011f8..f77621ba52 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/PlannerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/PlannerTest.java
@@ -30,7 +30,6 @@ import org.apache.iotdb.db.metadata.LocalSchemaProcessor;
 import org.apache.iotdb.db.qp.executor.PlanExecutor;
 import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-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.service.rpc.thrift.TSLastDataQueryReq;
@@ -54,7 +53,6 @@ import java.util.Collections;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 public class PlannerTest {
 
@@ -235,28 +233,6 @@ public class PlannerTest {
     processor.parseSQLToPhysicalPlan(createTSStatement);
   }
 
-  @Test
-  public void insertStatementWithNullValue() throws QueryProcessException {
-    String createTSStatement = "insert into root.vehicle.d0(time,s0) values(10,NaN)";
-    PhysicalPlan physicalPlan = processor.parseSQLToPhysicalPlan(createTSStatement);
-
-    assertTrue(physicalPlan instanceof InsertRowPlan);
-    assertEquals("NaN", ((InsertRowPlan) physicalPlan).getValues()[0]);
-    // Later we will use Double.parseDouble so we have to ensure that it is parsed right
-    assertEquals(Double.NaN, Double.parseDouble("NaN"), 1e-15);
-  }
-
-  @Test
-  public void insertStatementWithNegativeTimeStamp()
-      throws QueryProcessException, MetadataException {
-    String createTSStatement = "insert into root.vehicle.d0(time,s0) values(-1000, 111)";
-    PhysicalPlan physicalPlan = processor.parseSQLToPhysicalPlan(createTSStatement);
-
-    assertTrue(physicalPlan instanceof InsertRowPlan);
-    assertEquals(-1000, ((InsertRowPlan) physicalPlan).getTime());
-    assertEquals("111", ((InsertRowPlan) physicalPlan).getValues()[0]);
-  }
-
   @Test
   public void rawDataQueryReqToPhysicalPlanTest()
       throws QueryProcessException, IllegalPathException {
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/writelog/LogWriterReaderTest.java b/server/src/test/java/org/apache/iotdb/db/utils/writelog/LogWriterReaderTest.java
index fa2969c3ff..71e48b6183 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/writelog/LogWriterReaderTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/writelog/LogWriterReaderTest.java
@@ -22,9 +22,10 @@ package org.apache.iotdb.db.utils.writelog;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
 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.sys.CreateTimeSeriesPlan;
+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.junit.Before;
 import org.junit.Test;
@@ -51,24 +52,33 @@ public class LogWriterReaderTest {
     if (new File(filePath).exists()) {
       new File(filePath).delete();
     }
-    InsertRowPlan insertRowPlan1 =
-        new InsertRowPlan(
-            new PartialPath("d1"),
-            10L,
-            new String[] {"s1", "s2"},
-            new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
-            new String[] {"1", "2"});
-    InsertRowPlan insertRowPlan2 =
-        new InsertRowPlan(
-            new PartialPath("d1"),
-            10L,
-            new String[] {"s1", "s2"},
-            new TSDataType[] {TSDataType.INT64, TSDataType.INT64},
-            new String[] {"1", "2"});
-    DeletePlan deletePlan = new DeletePlan(Long.MIN_VALUE, 10L, new PartialPath("root.d1.s1"));
-    plans.add(insertRowPlan1);
-    plans.add(insertRowPlan2);
-    plans.add(deletePlan);
+    CreateTimeSeriesPlan plan1 =
+        new CreateTimeSeriesPlan(
+            new PartialPath("d1.s1"),
+            TSDataType.INT64,
+            TSEncoding.PLAIN,
+            CompressionType.SNAPPY,
+            null,
+            null,
+            null,
+            null);
+
+    CreateTimeSeriesPlan plan2 =
+        new CreateTimeSeriesPlan(
+            new PartialPath("d1.s2"),
+            TSDataType.INT64,
+            TSEncoding.PLAIN,
+            CompressionType.SNAPPY,
+            null,
+            null,
+            null,
+            null);
+
+    List<PartialPath> paths = new ArrayList<>();
+    paths.add(new PartialPath("d1.s1"));
+    paths.add(new PartialPath("d1.s2"));
+    plans.add(plan1);
+    plans.add(plan2);
     for (PhysicalPlan plan : plans) {
       plan.serialize(logsBuffer);
     }
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/writelog/MultiFileLogReaderTest.java b/server/src/test/java/org/apache/iotdb/db/utils/writelog/MultiFileLogReaderTest.java
index 5e7af41382..14b0d6f377 100644
--- a/server/src/test/java/org/apache/iotdb/db/utils/writelog/MultiFileLogReaderTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/utils/writelog/MultiFileLogReaderTest.java
@@ -22,7 +22,10 @@ package org.apache.iotdb.db.utils.writelog;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
+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.commons.io.FileUtils;
 import org.junit.After;
@@ -50,7 +53,15 @@ public class MultiFileLogReaderTest {
       logFiles[i] = new File(i + ".log");
       for (int j = 0; j < logsPerFile; j++) {
         fileLogs[i][j] =
-            new DeletePlan(Long.MIN_VALUE, i * logsPerFile + j, new PartialPath("path" + j));
+            new CreateTimeSeriesPlan(
+                new PartialPath("root.sg.d1.s" + j),
+                TSDataType.INT32,
+                TSEncoding.PLAIN,
+                CompressionType.SNAPPY,
+                null,
+                null,
+                null,
+                null);
       }
 
       ByteBuffer buffer = ByteBuffer.allocate(64 * 1024);
diff --git a/server/src/test/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoerTest.java b/server/src/test/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoerTest.java
index b0197a7a3b..cfbd8e095e 100644
--- a/server/src/test/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/wal/recover/file/TsFilePlanRedoerTest.java
@@ -616,7 +616,7 @@ public class TsFilePlanRedoerTest {
     tsFileResource.updateStartTime(DEVICE2_NAME, 3);
     tsFileResource.updateEndTime(DEVICE2_NAME, 4);
 
-    // generate DeletePlan
+    // generate DeleteDataNode
     DeleteDataNode deleteDataNode =
         new DeleteDataNode(
             new PlanNodeId(""),
@@ -624,7 +624,7 @@ public class TsFilePlanRedoerTest {
             Long.MIN_VALUE,
             Long.MAX_VALUE);
 
-    // redo DeletePlan, vsg processor is used to test IdTable, don't test IdTable here
+    // redo DeleteDataNode, vsg processor is used to test IdTable, don't test IdTable here
     File modsFile = new File(FILE_NAME.concat(ModificationFile.FILE_SUFFIX));
     assertFalse(modsFile.exists());
     TsFilePlanRedoer planRedoer = new TsFilePlanRedoer(tsFileResource, false, null);