You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by su...@apache.org on 2021/03/13 15:38:17 UTC

[iotdb] 01/02: [IOTDB-1225] Support CreateAlignedTimeseriesPlan in MManager

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

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

commit 3f6b91559bfc93e73aa8738d7da5a93ce47a6759
Author: samperson1997 <sz...@mails.tsinghua.edu.cn>
AuthorDate: Sat Mar 13 23:24:40 2021 +0800

    [IOTDB-1225] Support CreateAlignedTimeseriesPlan in MManager
---
 .../metadata/AlignedTimeseriesException.java       |  31 ++++
 .../org/apache/iotdb/db/metadata/MManager.java     |  79 +++++++++-
 .../java/org/apache/iotdb/db/metadata/MTree.java   | 167 ++++++++++++++++++---
 .../iotdb/db/metadata/MManagerBasicTest.java       |  67 ++++++++-
 .../db/qp/physical/PhysicalPlanSerializeTest.java  |  20 +--
 .../java/org/apache/iotdb/rpc/TSStatusCode.java    |   1 +
 .../write/writer/VectorChunkWriterImplTest.java    |   8 -
 7 files changed, 317 insertions(+), 56 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/AlignedTimeseriesException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/AlignedTimeseriesException.java
new file mode 100644
index 0000000..ac95856
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/AlignedTimeseriesException.java
@@ -0,0 +1,31 @@
+/*
+ * 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.exception.metadata;
+
+import org.apache.iotdb.rpc.TSStatusCode;
+
+public class AlignedTimeseriesException extends MetadataException {
+
+  public AlignedTimeseriesException(String message, String path) {
+    super(String.format("%s (Timeseries: %s)", message, path));
+    errorCode = TSStatusCode.ALIGNED_TIMESERIES_ERROR.getStatusCode();
+    this.isUserException = true;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index 76703a8..60819b3 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@ -73,6 +73,7 @@ import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -485,8 +486,63 @@ public class MManager {
     }
   }
 
-  // TODO
-  public void createAlignedTimeSeries(CreateAlignedTimeSeriesPlan plan) throws MetadataException {}
+  public void createAlignedTimeSeries(
+      PartialPath devicePath,
+      List<String> measurements,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings,
+      CompressionType compressor)
+      throws MetadataException {
+    createAlignedTimeSeries(
+        new CreateAlignedTimeSeriesPlan(
+            devicePath, measurements, dataTypes, encodings, compressor, null));
+  }
+
+  /**
+   * create aligned timeseries
+   *
+   * @param plan CreateAlignedTimeSeriesPlan
+   */
+  public void createAlignedTimeSeries(CreateAlignedTimeSeriesPlan plan) throws MetadataException {
+    if (!allowToCreateNewSeries) {
+      throw new MetadataException(
+          "IoTDB system load is too large to create timeseries, "
+              + "please increase MAX_HEAP_SIZE in iotdb-env.sh/bat and restart");
+    }
+    try {
+      PartialPath devicePath = plan.getDevicePath();
+      List<String> measurements = plan.getMeasurements();
+      int alignedSize = measurements.size();
+      List<TSDataType> dataTypes = plan.getDataTypes();
+      List<TSEncoding> encodings = plan.getEncodings();
+
+      for (int i = 0; i < alignedSize; i++) {
+        SchemaUtils.checkDataTypeWithEncoding(dataTypes.get(i), encodings.get(i));
+      }
+
+      ensureStorageGroup(devicePath);
+
+      // create time series in MTree
+      mtree.createAlignedTimeseries(
+          devicePath, measurements, plan.getDataTypes(), plan.getEncodings(), plan.getCompressor());
+
+      // update statistics and schemaDataTypeNumMap
+      totalSeriesNumber.addAndGet(measurements.size());
+      if (totalSeriesNumber.get() * ESTIMATED_SERIES_SIZE >= MTREE_SIZE_THRESHOLD) {
+        logger.warn("Current series number {} is too large...", totalSeriesNumber);
+        allowToCreateNewSeries = false;
+      }
+      for (TSDataType type : dataTypes) {
+        updateSchemaDataTypeNumMap(type, 1);
+      }
+      // write log
+      if (!isRecovering) {
+        logWriter.createAlignedTimeseries(plan);
+      }
+    } catch (IOException e) {
+      throw new MetadataException(e);
+    }
+  }
 
   /**
    * Delete all timeseries under the given path, may cross different storage group
@@ -586,15 +642,24 @@ public class MManager {
       throws MetadataException, IOException {
     Pair<PartialPath, MeasurementMNode> pair =
         mtree.deleteTimeseriesAndReturnEmptyStorageGroup(path);
-    removeFromTagInvertedIndex(pair.right);
+    // if one of the aligned timeseries is deleted, pair.right could be null
+    IMeasurementSchema schema = pair.right.getSchema();
+    int timeseriesNum = 0;
+    if (schema instanceof MeasurementSchema) {
+      removeFromTagInvertedIndex(pair.right);
+      updateSchemaDataTypeNumMap(schema.getType(), -1);
+      timeseriesNum = 1;
+    } else if (schema instanceof VectorMeasurementSchema) {
+      for (TSDataType dataType : schema.getValueTSDataTypeList()) {
+        updateSchemaDataTypeNumMap(dataType, -1);
+        timeseriesNum++;
+      }
+    }
     PartialPath storageGroupPath = pair.left;
 
-    // update statistics in schemaDataTypeNumMap
-    updateSchemaDataTypeNumMap(pair.right.getSchema().getType(), -1);
-
     // TODO: delete the path node and all its ancestors
     mNodeCache.clear();
-    totalSeriesNumber.addAndGet(-1);
+    totalSeriesNumber.addAndGet(-timeseriesNum);
     if (!allowToCreateNewSeries
         && totalSeriesNumber.get() * ESTIMATED_SERIES_SIZE < MTREE_SIZE_THRESHOLD) {
       logger.info("Current series number {} come back to normal level", totalSeriesNumber);
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
index 762e2a8..7b4110e 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
@@ -22,7 +22,14 @@ import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
-import org.apache.iotdb.db.exception.metadata.*;
+import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.IllegalParameterOfPathException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.metadata.MManager.StorageGroupFilter;
 import org.apache.iotdb.db.metadata.logfile.MLogReader;
 import org.apache.iotdb.db.metadata.logfile.MLogWriter;
@@ -30,7 +37,11 @@ import org.apache.iotdb.db.metadata.mnode.MNode;
 import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.sys.*;
+import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.StorageGroupMNodePlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.query.dataset.ShowDevicesResult;
@@ -42,6 +53,8 @@ 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.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
 
 import com.google.gson.Gson;
 import com.google.gson.GsonBuilder;
@@ -53,14 +66,31 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.*;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Queue;
+import java.util.Set;
+import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.regex.Pattern;
 import java.util.stream.Stream;
 
 import static java.util.stream.Collectors.toList;
-import static org.apache.iotdb.db.conf.IoTDBConstant.*;
+import static org.apache.iotdb.db.conf.IoTDBConstant.LOSS;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_WILDCARD;
+import static org.apache.iotdb.db.conf.IoTDBConstant.SDT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.SDT_COMP_DEV;
+import static org.apache.iotdb.db.conf.IoTDBConstant.SDT_COMP_MAX_TIME;
+import static org.apache.iotdb.db.conf.IoTDBConstant.SDT_COMP_MIN_TIME;
 
 /** The hierarchical struct of the Metadata Tree is implemented in this class. */
 public class MTree implements Serializable {
@@ -165,7 +195,7 @@ public class MTree implements Serializable {
   }
 
   /**
-   * Create a timeseries with a full path from root to leaf node Before creating a timeseries, the
+   * Create a timeseries with a full path from root to leaf node. Before creating a timeseries, the
    * storage group should be set first, throw exception otherwise
    *
    * @param path timeseries path
@@ -244,6 +274,81 @@ public class MTree implements Serializable {
     }
   }
 
+  /**
+   * Create aligned timeseries with full paths from root to one leaf node. Before creating
+   * timeseries, the * storage group should be set first, throw exception otherwise
+   *
+   * @param devicePath device path
+   * @param measurements measurements list
+   * @param dataTypes data types list
+   * @param encodings encodings list
+   * @param compressor compressor
+   */
+  void createAlignedTimeseries(
+      PartialPath devicePath,
+      List<String> measurements,
+      List<TSDataType> dataTypes,
+      List<TSEncoding> encodings,
+      CompressionType compressor)
+      throws MetadataException {
+    String[] deviceNodeNames = devicePath.getNodes();
+    if (deviceNodeNames.length <= 1 || !deviceNodeNames[0].equals(root.getName())) {
+      throw new IllegalPathException(devicePath.getFullPath());
+    }
+    checkTimeseries(devicePath.getFullPath());
+    for (String measurement : measurements) {
+      checkTimeseries(measurement);
+    }
+    MNode cur = root;
+    boolean hasSetStorageGroup = false;
+    // e.g, devicePath = root.sg.d1, create internal nodes and set cur to d1 node
+    for (int i = 1; i < deviceNodeNames.length; i++) {
+      String nodeName = deviceNodeNames[i];
+      if (cur instanceof StorageGroupMNode) {
+        hasSetStorageGroup = true;
+      }
+      if (!cur.hasChild(nodeName)) {
+        if (!hasSetStorageGroup) {
+          throw new StorageGroupNotSetException("Storage group should be created first");
+        }
+        cur.addChild(nodeName, new MNode(cur, nodeName));
+      }
+      cur = cur.getChild(nodeName);
+    }
+
+    String leafName = measurements.get(0) + ".align";
+
+    // synchronize check and add, we need addChild and add Alias become atomic operation
+    // only write on mtree will be synchronized
+    synchronized (this) {
+      MNode child = cur.getChild(leafName);
+      if (child instanceof MeasurementMNode || child instanceof StorageGroupMNode) {
+        throw new PathAlreadyExistException(devicePath.getFullPath() + "." + leafName);
+      }
+
+      int measurementsSize = measurements.size();
+
+      // this measurementMNode could be a leaf or not.
+      MeasurementMNode measurementMNode =
+          new MeasurementMNode(
+              cur,
+              leafName,
+              new VectorMeasurementSchema(
+                  measurements.toArray(new String[measurementsSize]),
+                  dataTypes.toArray(new TSDataType[measurementsSize]),
+                  encodings.toArray(new TSEncoding[measurementsSize]),
+                  compressor),
+              null);
+      for (String measurement : measurements) {
+        if (child != null) {
+          cur.replaceChild(measurementMNode.getName(), measurementMNode);
+        } else {
+          cur.addChild(measurement, measurementMNode);
+        }
+      }
+    }
+  }
+
   private void checkTimeseries(String timeseries) throws IllegalPathException {
     if (!IoTDBConfig.NODE_PATTERN.matcher(timeseries).matches()) {
       throw new IllegalPathException(
@@ -473,7 +578,7 @@ public class MTree implements Serializable {
       throw new IllegalPathException(path.getFullPath());
     }
     // delete the last node of path
-    curNode.getParent().deleteChild(curNode.getName());
+    curNode.getParent().deleteChild(path.getMeasurement());
     MeasurementMNode deletedNode = (MeasurementMNode) curNode;
     if (deletedNode.getAlias() != null) {
       curNode.getParent().deleteAliasChild(((MeasurementMNode) curNode).getAlias());
@@ -1078,21 +1183,43 @@ public class MTree implements Serializable {
           return;
         }
       }
-
-      PartialPath nodePath = node.getPartialPath();
-      String[] tsRow = new String[7];
-      tsRow[0] = ((MeasurementMNode) node).getAlias();
       IMeasurementSchema measurementSchema = ((MeasurementMNode) node).getSchema();
-      tsRow[1] = getStorageGroupPath(nodePath).getFullPath();
-      tsRow[2] = measurementSchema.getType().toString();
-      tsRow[3] = measurementSchema.getEncodingType().toString();
-      tsRow[4] = measurementSchema.getCompressor().toString();
-      tsRow[5] = String.valueOf(((MeasurementMNode) node).getOffset());
-      tsRow[6] =
-          needLast ? String.valueOf(getLastTimeStamp((MeasurementMNode) node, queryContext)) : null;
-      Pair<PartialPath, String[]> temp = new Pair<>(nodePath, tsRow);
-      timeseriesSchemaList.add(temp);
-
+      if (measurementSchema instanceof MeasurementSchema) {
+        PartialPath nodePath = node.getPartialPath();
+        String[] tsRow = new String[7];
+        tsRow[0] = ((MeasurementMNode) node).getAlias();
+        tsRow[1] = getStorageGroupPath(nodePath).getFullPath();
+        tsRow[2] = measurementSchema.getType().toString();
+        tsRow[3] = measurementSchema.getEncodingType().toString();
+        tsRow[4] = measurementSchema.getCompressor().toString();
+        tsRow[5] = String.valueOf(((MeasurementMNode) node).getOffset());
+        tsRow[6] =
+            needLast
+                ? String.valueOf(getLastTimeStamp((MeasurementMNode) node, queryContext))
+                : null;
+        Pair<PartialPath, String[]> temp = new Pair<>(nodePath, tsRow);
+        timeseriesSchemaList.add(temp);
+      } else if (measurementSchema instanceof VectorMeasurementSchema) {
+        List<String> measurements = measurementSchema.getValueMeasurementIdList();
+        int measurementSize = measurements.size();
+        for (int i = 0; i < measurementSize; i++) {
+          PartialPath devicePath = node.getPartialPath().getDevicePath();
+          String[] tsRow = new String[7];
+          tsRow[0] = null;
+          tsRow[1] = getStorageGroupPath(devicePath).getFullPath();
+          tsRow[2] = measurementSchema.getValueTSDataTypeList().get(i).toString();
+          tsRow[3] = measurementSchema.getValueTSEncodingList().get(i).toString();
+          tsRow[4] = measurementSchema.getCompressor().toString();
+          tsRow[5] = "0";
+          tsRow[6] =
+              needLast
+                  ? String.valueOf(getLastTimeStamp((MeasurementMNode) node, queryContext))
+                  : null;
+          Pair<PartialPath, String[]> temp =
+              new Pair<>(new PartialPath(devicePath.getFullPath(), measurements.get(i)), tsRow);
+          timeseriesSchemaList.add(temp);
+        }
+      }
       if (hasLimit) {
         count.set(count.get() + 1);
       }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
index 624f441..1f4bbe4 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
@@ -64,7 +64,6 @@ public class MManagerBasicTest {
 
   @Test
   public void testAddPathAndExist() throws IllegalPathException {
-
     MManager manager = IoTDB.metaManager;
     assertTrue(manager.isPathExist(new PartialPath("root")));
 
@@ -114,19 +113,19 @@ public class MManagerBasicTest {
           TSDataType.INT32,
           TSEncoding.RLE,
           TSFileDescriptor.getInstance().getConfig().getCompressor(),
-          Collections.EMPTY_MAP);
+          Collections.emptyMap());
       manager.createTimeseries(
           new PartialPath("root.laptop.d1.\"1.2.3\""),
           TSDataType.INT32,
           TSEncoding.RLE,
           TSFileDescriptor.getInstance().getConfig().getCompressor(),
-          Collections.EMPTY_MAP);
+          Collections.emptyMap());
       manager.createTimeseries(
           new PartialPath("root.1.2.3"),
           TSDataType.INT32,
           TSEncoding.RLE,
           TSFileDescriptor.getInstance().getConfig().getCompressor(),
-          Collections.EMPTY_MAP);
+          Collections.emptyMap());
 
       assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s1")));
       assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.1_2")));
@@ -231,6 +230,66 @@ public class MManagerBasicTest {
   }
 
   @Test
+  public void testCreateAlignedTimeseries() throws IllegalPathException {
+    MManager manager = IoTDB.metaManager;
+    try {
+      manager.setStorageGroup(new PartialPath("root.laptop"));
+    } catch (MetadataException e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+
+    try {
+      manager.createTimeseries(
+          new PartialPath("root.laptop.d1.s0"),
+          TSDataType.valueOf("INT32"),
+          TSEncoding.valueOf("RLE"),
+          compressionType,
+          Collections.emptyMap());
+      manager.createAlignedTimeSeries(
+          new PartialPath("root.laptop.d1"),
+          Arrays.asList("s1", "s2", "s3"),
+          Arrays.asList(
+              TSDataType.valueOf("INT32"),
+              TSDataType.valueOf("FLOAT"),
+              TSDataType.valueOf("INT32")),
+          Arrays.asList(
+              TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE")),
+          compressionType);
+    } catch (MetadataException e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+
+    assertTrue(manager.isPathExist(new PartialPath("root.laptop")));
+    assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1")));
+    assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s0")));
+    assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s1")));
+    assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s2")));
+    assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s3")));
+
+    try {
+      manager.deleteTimeseries(new PartialPath("root.laptop.d1.s2"));
+    } catch (MetadataException e) {
+      e.printStackTrace();
+    }
+
+    assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1")));
+    assertTrue(manager.isPathExist(new PartialPath("root.laptop.d1.s0")));
+    assertFalse(manager.isPathExist(new PartialPath("root.laptop.d1.s1")));
+    assertFalse(manager.isPathExist(new PartialPath("root.laptop.d1.s2")));
+    assertFalse(manager.isPathExist(new PartialPath("root.laptop.d1.s3")));
+
+    try {
+      manager.deleteTimeseries(new PartialPath("root.laptop.d1.s0"));
+    } catch (MetadataException e) {
+      e.printStackTrace();
+    }
+    assertFalse(manager.isPathExist(new PartialPath("root.laptop.d1")));
+    assertFalse(manager.isPathExist(new PartialPath("root.laptop.d1.s0")));
+  }
+
+  @Test
   public void testGetAllTimeseriesCount() {
     MManager manager = IoTDB.metaManager;
 
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanSerializeTest.java b/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanSerializeTest.java
index e7d1b25..9a3eee3 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanSerializeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanSerializeTest.java
@@ -51,10 +51,8 @@ import java.io.ByteArrayOutputStream;
 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.List;
 import java.util.Properties;
 
 public class PhysicalPlanSerializeTest {
@@ -194,24 +192,12 @@ public class PhysicalPlanSerializeTest {
 
   @Test
   public void createAlignedTimeSeriesPlanSerializeTest() throws IOException, IllegalPathException {
-    List<String> measurements = new ArrayList<>();
-    measurements.add("s1");
-    measurements.add("s2");
-
-    List<TSDataType> dataTypes = new ArrayList<>();
-    dataTypes.add(TSDataType.DOUBLE);
-    dataTypes.add(TSDataType.INT32);
-
-    List<TSEncoding> encodings = new ArrayList<>();
-    encodings.add(TSEncoding.RLE);
-    encodings.add(TSEncoding.RLE);
-
     CreateAlignedTimeSeriesPlan createAlignedTimeSeriesPlan =
         new CreateAlignedTimeSeriesPlan(
             new PartialPath("root.sg.d1"),
-            measurements,
-            dataTypes,
-            encodings,
+            Arrays.asList("s1", "s2"),
+            Arrays.asList(TSDataType.DOUBLE, TSDataType.INT32),
+            Arrays.asList(TSEncoding.RLE, TSEncoding.RLE),
             CompressionType.SNAPPY,
             null);
 
diff --git a/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
index f2fd3c1..fd76bfc 100644
--- a/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
+++ b/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java
@@ -46,6 +46,7 @@ public enum TSStatusCode {
   LOAD_FILE_ERROR(316),
   STORAGE_GROUP_NOT_READY(317),
   ILLEGAL_PARAMETER(318),
+  ALIGNED_TIMESERIES_ERROR(319),
 
   EXECUTE_STATEMENT_ERROR(400),
   SQL_PARSE_ERROR(401),
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/VectorChunkWriterImplTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/VectorChunkWriterImplTest.java
index 7866f48..3ca81b1 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/VectorChunkWriterImplTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/writer/VectorChunkWriterImplTest.java
@@ -65,11 +65,7 @@ public class VectorChunkWriterImplTest {
           (byte) (0x80 | MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER), ReadWriteIOUtils.readByte(buffer));
       assertEquals("s1.time", ReadWriteIOUtils.readVarIntString(buffer));
       assertEquals(164, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
-<<<<<<< HEAD
       assertEquals(TSDataType.VECTOR.serialize(), ReadWriteIOUtils.readByte(buffer));
-=======
-      assertEquals(TSDataType.Vector.serialize(), ReadWriteIOUtils.readByte(buffer));
->>>>>>> origin/Vector
       assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
       assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
       buffer.position(buffer.position() + 164);
@@ -142,11 +138,7 @@ public class VectorChunkWriterImplTest {
       assertEquals((byte) (0x80 | MetaMarker.CHUNK_HEADER), ReadWriteIOUtils.readByte(buffer));
       assertEquals("s1.time", ReadWriteIOUtils.readVarIntString(buffer));
       assertEquals(362, ReadWriteForEncodingUtils.readUnsignedVarInt(buffer));
-<<<<<<< HEAD
       assertEquals(TSDataType.VECTOR.serialize(), ReadWriteIOUtils.readByte(buffer));
-=======
-      assertEquals(TSDataType.Vector.serialize(), ReadWriteIOUtils.readByte(buffer));
->>>>>>> origin/Vector
       assertEquals(CompressionType.UNCOMPRESSED.serialize(), ReadWriteIOUtils.readByte(buffer));
       assertEquals(TSEncoding.PLAIN.serialize(), ReadWriteIOUtils.readByte(buffer));
       buffer.position(buffer.position() + 362);