You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2020/09/27 05:41:27 UTC

[GitHub] [incubator-iotdb] mychaow opened a new pull request #1771: [IOTDB-884] batch create timeSeriesPlan

mychaow opened a new pull request #1771:
URL: https://github.com/apache/incubator-iotdb/pull/1771


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-iotdb] mychaow commented on a change in pull request #1771: [IOTDB-884] batch create timeSeriesPlan

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #1771:
URL: https://github.com/apache/incubator-iotdb/pull/1771#discussion_r496487653



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -1040,6 +1029,31 @@ private boolean createTimeSeries(CreateTimeSeriesPlan createTimeSeriesPlan)
     return true;
   }
 
+  private boolean createMultiTimeSeries(CreateMultiTimeSeriesPlan createMultiTimeSeriesPlan) {
+    Map<Integer, Boolean> results = new HashMap<>(createMultiTimeSeriesPlan.getPaths().size());
+    for (int i = 0; i < createMultiTimeSeriesPlan.getPaths().size(); i++) {
+      CreateTimeSeriesPlan plan = new CreateTimeSeriesPlan(createMultiTimeSeriesPlan.getPaths().get(i),
+        createMultiTimeSeriesPlan.getDataTypes().get(i), createMultiTimeSeriesPlan.getEncodings().get(i),
+        createMultiTimeSeriesPlan.getCompressors().get(i),
+        createMultiTimeSeriesPlan.getProps() == null ? null : createMultiTimeSeriesPlan.getProps().get(i),
+        createMultiTimeSeriesPlan.getTags() == null ? null : createMultiTimeSeriesPlan.getTags().get(i),
+        createMultiTimeSeriesPlan.getAttributes() == null ? null : createMultiTimeSeriesPlan.getAttributes().get(i),
+        createMultiTimeSeriesPlan.getAlias() == null ? null : createMultiTimeSeriesPlan.getAlias().get(i));
+
+      boolean success = false;
+      try {
+        success = createTimeSeries(plan);
+      } catch (QueryProcessException e) {
+        // do nothing
+        logger.debug("meet error while processing create timeseries. ", e);
+      } finally {
+        results.put(createMultiTimeSeriesPlan.getIndexes().get(i), success);

Review comment:
       ok




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-iotdb] mychaow commented on a change in pull request #1771: [IOTDB-884] batch create timeSeriesPlan

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #1771:
URL: https://github.com/apache/incubator-iotdb/pull/1771#discussion_r496489240



##########
File path: server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
##########
@@ -1533,14 +1528,35 @@ public TSStatus createMultiTimeseries(TSCreateMultiTimeseriesReq req) {
             req.getPaths().size(), req.getPaths().get(0));
       }
       List<TSStatus> statusList = new ArrayList<>(req.paths.size());
+      CreateTimeSeriesPlan plan = new CreateTimeSeriesPlan();

Review comment:
       Just to do some check, like checkAuthority before execute the CreateMultiTimeSeriesPlan.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-iotdb] jt2594838 commented on a change in pull request #1771: [IOTDB-884] batch create timeSeriesPlan

Posted by GitBox <gi...@apache.org>.
jt2594838 commented on a change in pull request #1771:
URL: https://github.com/apache/incubator-iotdb/pull/1771#discussion_r496444420



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -1040,6 +1029,31 @@ private boolean createTimeSeries(CreateTimeSeriesPlan createTimeSeriesPlan)
     return true;
   }
 
+  private boolean createMultiTimeSeries(CreateMultiTimeSeriesPlan createMultiTimeSeriesPlan) {
+    Map<Integer, Boolean> results = new HashMap<>(createMultiTimeSeriesPlan.getPaths().size());
+    for (int i = 0; i < createMultiTimeSeriesPlan.getPaths().size(); i++) {
+      CreateTimeSeriesPlan plan = new CreateTimeSeriesPlan(createMultiTimeSeriesPlan.getPaths().get(i),
+        createMultiTimeSeriesPlan.getDataTypes().get(i), createMultiTimeSeriesPlan.getEncodings().get(i),
+        createMultiTimeSeriesPlan.getCompressors().get(i),
+        createMultiTimeSeriesPlan.getProps() == null ? null : createMultiTimeSeriesPlan.getProps().get(i),
+        createMultiTimeSeriesPlan.getTags() == null ? null : createMultiTimeSeriesPlan.getTags().get(i),
+        createMultiTimeSeriesPlan.getAttributes() == null ? null : createMultiTimeSeriesPlan.getAttributes().get(i),
+        createMultiTimeSeriesPlan.getAlias() == null ? null : createMultiTimeSeriesPlan.getAlias().get(i));
+
+      boolean success = false;
+      try {
+        success = createTimeSeries(plan);
+      } catch (QueryProcessException e) {
+        // do nothing
+        logger.debug("meet error while processing create timeseries. ", e);
+      } finally {
+        results.put(createMultiTimeSeriesPlan.getIndexes().get(i), success);

Review comment:
       I recommend that you straightly make` results` a `Map<Integer, Exception>` and record the potential `QueryProcessException` into it since `createTimeSeries` always returns true if there is no exception. And when changed this way, you would be able to tell the user what are the reasons behind the failures, instead of a simple failure.

##########
File path: server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
##########
@@ -1533,14 +1528,35 @@ public TSStatus createMultiTimeseries(TSCreateMultiTimeseriesReq req) {
             req.getPaths().size(), req.getPaths().get(0));
       }
       List<TSStatus> statusList = new ArrayList<>(req.paths.size());
+      CreateTimeSeriesPlan plan = new CreateTimeSeriesPlan();

Review comment:
       What is this plan for?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-iotdb] mychaow commented on a change in pull request #1771: [IOTDB-884] batch create timeSeriesPlan

Posted by GitBox <gi...@apache.org>.
mychaow commented on a change in pull request #1771:
URL: https://github.com/apache/incubator-iotdb/pull/1771#discussion_r496352092



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateMultiTimeSeriesPlan.java
##########
@@ -0,0 +1,284 @@
+/*
+ * 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.sys;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+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.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * create multiple timeSeries, could be split to several sub Plans to execute in different DataGroup
+ */
+public class CreateMultiTimeSeriesPlan extends PhysicalPlan {
+  private List<PartialPath> paths;
+  private List<TSDataType> dataTypes;
+  private List<TSEncoding> encodings;
+  private List<CompressionType> compressors;
+  private List<String> alias;
+  private List<Map<String, String>> props = null;
+  private List<Map<String, String>> tags = null;
+  private List<Map<String, String>> attributes = null;
+
+  /*
+   ** record the result of creation of time series
+   */
+  private Map<Integer, Boolean> results = new HashMap<>();
+  private List<Integer> indexes;
+
+  public CreateMultiTimeSeriesPlan() {
+    super(false, Operator.OperatorType.CREATE_MULTI_TIMESERIES);
+  }
+
+  @Override
+  public List<PartialPath> getPaths() {
+    return paths;
+  }
+
+  public void setPaths(List<PartialPath> paths) {
+    this.paths = paths;
+  }
+
+  public List<TSDataType> getDataTypes() {
+    return dataTypes;
+  }
+
+  public void setDataTypes(List<TSDataType> dataTypes) {
+    this.dataTypes = dataTypes;
+  }
+
+  public List<TSEncoding> getEncodings() {
+    return encodings;
+  }
+
+  public void setEncodings(List<TSEncoding> encodings) {
+    this.encodings = encodings;
+  }
+
+  public List<CompressionType> getCompressors() {
+    return compressors;
+  }
+
+  public void setCompressors(List<CompressionType> compressors) {
+    this.compressors = compressors;
+  }
+
+  public List<String> getAlias() {
+    return alias;
+  }
+
+  public void setAlias(List<String> alias) {
+    this.alias = alias;
+  }
+
+  public List<Map<String, String>> getProps() {
+    return props;
+  }
+
+  public void setProps(List<Map<String, String>> props) {
+    this.props = props;
+  }
+
+  public List<Map<String, String>> getTags() {
+    return tags;
+  }
+
+  public void setTags(List<Map<String, String>> tags) {
+    this.tags = tags;
+  }
+
+  public List<Map<String, String>> getAttributes() {
+    return attributes;
+  }
+
+  public void setAttributes(List<Map<String, String>> attributes) {
+    this.attributes = attributes;
+  }
+
+  public List<Integer> getIndexes() {
+    return indexes;
+  }
+
+  public void setIndexes(List<Integer> indexes) {
+    this.indexes = indexes;
+  }
+
+  public Map<Integer, Boolean> getResults() {
+    return results;
+  }
+
+  public void setResults(Map<Integer, Boolean> results) {
+    this.results = results;
+  }
+
+  @Override
+  public void serialize(DataOutputStream stream) throws IOException {
+    int type = PhysicalPlanType.MULTI_CREATE_TIMESERIES.ordinal();
+    stream.write(type);
+    stream.writeInt(paths.size());
+
+    for (PartialPath path : paths) {
+      putString(stream, path.getFullPath());
+    }
+
+    for (TSDataType dataType : dataTypes) {
+      stream.write(dataType.ordinal());
+    }
+
+    for (TSEncoding encoding : encodings) {
+      stream.write(encoding.ordinal());
+    }
+
+    for (CompressionType compressor : compressors) {
+      stream.write(compressor.ordinal());
+    }
+
+    for (String name : alias) {

Review comment:
       ok




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-iotdb] Alima777 commented on a change in pull request #1771: [IOTDB-884] batch create timeSeriesPlan

Posted by GitBox <gi...@apache.org>.
Alima777 commented on a change in pull request #1771:
URL: https://github.com/apache/incubator-iotdb/pull/1771#discussion_r496346212



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateMultiTimeSeriesPlan.java
##########
@@ -0,0 +1,284 @@
+/*
+ * 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.sys;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+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.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * create multiple timeSeries, could be split to several sub Plans to execute in different DataGroup
+ */
+public class CreateMultiTimeSeriesPlan extends PhysicalPlan {
+  private List<PartialPath> paths;
+  private List<TSDataType> dataTypes;
+  private List<TSEncoding> encodings;
+  private List<CompressionType> compressors;
+  private List<String> alias;
+  private List<Map<String, String>> props = null;
+  private List<Map<String, String>> tags = null;
+  private List<Map<String, String>> attributes = null;
+
+  /*
+   ** record the result of creation of time series
+   */
+  private Map<Integer, Boolean> results = new HashMap<>();
+  private List<Integer> indexes;
+
+  public CreateMultiTimeSeriesPlan() {
+    super(false, Operator.OperatorType.CREATE_MULTI_TIMESERIES);
+  }
+
+  @Override
+  public List<PartialPath> getPaths() {
+    return paths;
+  }
+
+  public void setPaths(List<PartialPath> paths) {
+    this.paths = paths;
+  }
+
+  public List<TSDataType> getDataTypes() {
+    return dataTypes;
+  }
+
+  public void setDataTypes(List<TSDataType> dataTypes) {
+    this.dataTypes = dataTypes;
+  }
+
+  public List<TSEncoding> getEncodings() {
+    return encodings;
+  }
+
+  public void setEncodings(List<TSEncoding> encodings) {
+    this.encodings = encodings;
+  }
+
+  public List<CompressionType> getCompressors() {
+    return compressors;
+  }
+
+  public void setCompressors(List<CompressionType> compressors) {
+    this.compressors = compressors;
+  }
+
+  public List<String> getAlias() {
+    return alias;
+  }
+
+  public void setAlias(List<String> alias) {
+    this.alias = alias;
+  }
+
+  public List<Map<String, String>> getProps() {
+    return props;
+  }
+
+  public void setProps(List<Map<String, String>> props) {
+    this.props = props;
+  }
+
+  public List<Map<String, String>> getTags() {
+    return tags;
+  }
+
+  public void setTags(List<Map<String, String>> tags) {
+    this.tags = tags;
+  }
+
+  public List<Map<String, String>> getAttributes() {
+    return attributes;
+  }
+
+  public void setAttributes(List<Map<String, String>> attributes) {
+    this.attributes = attributes;
+  }
+
+  public List<Integer> getIndexes() {
+    return indexes;
+  }
+
+  public void setIndexes(List<Integer> indexes) {
+    this.indexes = indexes;
+  }
+
+  public Map<Integer, Boolean> getResults() {
+    return results;
+  }
+
+  public void setResults(Map<Integer, Boolean> results) {
+    this.results = results;
+  }
+
+  @Override
+  public void serialize(DataOutputStream stream) throws IOException {
+    int type = PhysicalPlanType.MULTI_CREATE_TIMESERIES.ordinal();
+    stream.write(type);
+    stream.writeInt(paths.size());
+
+    for (PartialPath path : paths) {
+      putString(stream, path.getFullPath());
+    }
+
+    for (TSDataType dataType : dataTypes) {
+      stream.write(dataType.ordinal());
+    }
+
+    for (TSEncoding encoding : encodings) {
+      stream.write(encoding.ordinal());
+    }
+
+    for (CompressionType compressor : compressors) {
+      stream.write(compressor.ordinal());
+    }
+
+    for (String name : alias) {
+      putString(stream, name);
+    }
+
+    if (props != null) {
+      stream.write(1);
+      for (Map<String, String> prop : props) {
+        ReadWriteIOUtils.write(prop, stream);
+      }
+    } else {
+      stream.write(0);
+    }
+
+    if (tags != null) {
+      stream.write(1);
+      for (Map<String, String> tag : tags) {
+        ReadWriteIOUtils.write(tag, stream);
+      }
+    } else {
+      stream.write(0);
+    }
+
+    if (attributes != null) {
+      stream.write(1);
+      for (Map<String, String> attribute : attributes) {
+        ReadWriteIOUtils.write(attribute, stream);
+      }
+    } else {
+      stream.write(0);
+    }
+  }
+
+  @Override
+  public void serialize(ByteBuffer buffer) {
+    int type = PhysicalPlanType.MULTI_CREATE_TIMESERIES.ordinal();
+    buffer.put((byte) type);
+    buffer.putInt(paths.size());
+
+    for (PartialPath path : paths) {
+      putString(buffer, path.getFullPath());
+    }
+
+    for (TSDataType dataType : dataTypes) {
+      buffer.put((byte) dataType.ordinal());
+    }
+
+    for (TSEncoding encoding : encodings) {
+      buffer.put((byte) encoding.ordinal());
+    }
+
+    for (CompressionType compressor : compressors) {
+      buffer.put((byte) compressor.ordinal());
+    }
+
+    for (String name : alias) {
+      putString(buffer, name);
+    }
+
+    if (props != null) {
+      buffer.put((byte) 1);
+      for (Map<String, String> prop : props) {
+        ReadWriteIOUtils.write(prop, buffer);
+      }
+    } else {
+      buffer.put((byte) 0);
+    }
+
+    if (tags != null) {
+      buffer.put((byte) 1);
+      for (Map<String, String> tag : tags) {
+        ReadWriteIOUtils.write(tag, buffer);
+      }
+    } else {
+      buffer.put((byte) 0);
+    }
+
+    if (attributes != null) {
+      buffer.put((byte) 1);
+      for (Map<String, String> attribute : attributes) {
+        ReadWriteIOUtils.write(attribute, buffer);
+      }
+    } else {
+      buffer.put((byte) 0);
+    }
+  }
+
+  @Override
+  public void deserialize(ByteBuffer buffer) throws IllegalPathException {
+    int totalSize = buffer.getInt();
+    for (int i = 0; i < totalSize; i++) {
+      paths.add(new PartialPath(readString(buffer)));
+    }
+    for (int i = 0; i < totalSize; i++) {
+      dataTypes.add(TSDataType.values()[buffer.get()]);
+    }
+    for (int i = 0; i < totalSize; i++) {
+      encodings.add(TSEncoding.values()[buffer.get()]);
+    }
+
+    for (int i = 0; i < totalSize; i++) {
+      alias.add(readString(buffer));
+    }

Review comment:
       Same.

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateMultiTimeSeriesPlan.java
##########
@@ -0,0 +1,284 @@
+/*
+ * 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.sys;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+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.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * create multiple timeSeries, could be split to several sub Plans to execute in different DataGroup
+ */
+public class CreateMultiTimeSeriesPlan extends PhysicalPlan {
+  private List<PartialPath> paths;
+  private List<TSDataType> dataTypes;
+  private List<TSEncoding> encodings;
+  private List<CompressionType> compressors;
+  private List<String> alias;
+  private List<Map<String, String>> props = null;
+  private List<Map<String, String>> tags = null;
+  private List<Map<String, String>> attributes = null;
+
+  /*
+   ** record the result of creation of time series
+   */
+  private Map<Integer, Boolean> results = new HashMap<>();
+  private List<Integer> indexes;
+
+  public CreateMultiTimeSeriesPlan() {
+    super(false, Operator.OperatorType.CREATE_MULTI_TIMESERIES);
+  }
+
+  @Override
+  public List<PartialPath> getPaths() {
+    return paths;
+  }
+
+  public void setPaths(List<PartialPath> paths) {
+    this.paths = paths;
+  }
+
+  public List<TSDataType> getDataTypes() {
+    return dataTypes;
+  }
+
+  public void setDataTypes(List<TSDataType> dataTypes) {
+    this.dataTypes = dataTypes;
+  }
+
+  public List<TSEncoding> getEncodings() {
+    return encodings;
+  }
+
+  public void setEncodings(List<TSEncoding> encodings) {
+    this.encodings = encodings;
+  }
+
+  public List<CompressionType> getCompressors() {
+    return compressors;
+  }
+
+  public void setCompressors(List<CompressionType> compressors) {
+    this.compressors = compressors;
+  }
+
+  public List<String> getAlias() {
+    return alias;
+  }
+
+  public void setAlias(List<String> alias) {
+    this.alias = alias;
+  }
+
+  public List<Map<String, String>> getProps() {
+    return props;
+  }
+
+  public void setProps(List<Map<String, String>> props) {
+    this.props = props;
+  }
+
+  public List<Map<String, String>> getTags() {
+    return tags;
+  }
+
+  public void setTags(List<Map<String, String>> tags) {
+    this.tags = tags;
+  }
+
+  public List<Map<String, String>> getAttributes() {
+    return attributes;
+  }
+
+  public void setAttributes(List<Map<String, String>> attributes) {
+    this.attributes = attributes;
+  }
+
+  public List<Integer> getIndexes() {
+    return indexes;
+  }
+
+  public void setIndexes(List<Integer> indexes) {
+    this.indexes = indexes;
+  }
+
+  public Map<Integer, Boolean> getResults() {
+    return results;
+  }
+
+  public void setResults(Map<Integer, Boolean> results) {
+    this.results = results;
+  }
+
+  @Override
+  public void serialize(DataOutputStream stream) throws IOException {
+    int type = PhysicalPlanType.MULTI_CREATE_TIMESERIES.ordinal();
+    stream.write(type);
+    stream.writeInt(paths.size());
+
+    for (PartialPath path : paths) {
+      putString(stream, path.getFullPath());
+    }
+
+    for (TSDataType dataType : dataTypes) {
+      stream.write(dataType.ordinal());
+    }
+
+    for (TSEncoding encoding : encodings) {
+      stream.write(encoding.ordinal());
+    }
+
+    for (CompressionType compressor : compressors) {
+      stream.write(compressor.ordinal());
+    }
+
+    for (String name : alias) {

Review comment:
       alias maybe null, please change the logic here.

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateMultiTimeSeriesPlan.java
##########
@@ -0,0 +1,284 @@
+/*
+ * 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.sys;
+
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+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.utils.ReadWriteIOUtils;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * create multiple timeSeries, could be split to several sub Plans to execute in different DataGroup
+ */
+public class CreateMultiTimeSeriesPlan extends PhysicalPlan {
+  private List<PartialPath> paths;
+  private List<TSDataType> dataTypes;
+  private List<TSEncoding> encodings;
+  private List<CompressionType> compressors;
+  private List<String> alias;
+  private List<Map<String, String>> props = null;
+  private List<Map<String, String>> tags = null;
+  private List<Map<String, String>> attributes = null;
+
+  /*
+   ** record the result of creation of time series
+   */
+  private Map<Integer, Boolean> results = new HashMap<>();
+  private List<Integer> indexes;
+
+  public CreateMultiTimeSeriesPlan() {
+    super(false, Operator.OperatorType.CREATE_MULTI_TIMESERIES);
+  }
+
+  @Override
+  public List<PartialPath> getPaths() {
+    return paths;
+  }
+
+  public void setPaths(List<PartialPath> paths) {
+    this.paths = paths;
+  }
+
+  public List<TSDataType> getDataTypes() {
+    return dataTypes;
+  }
+
+  public void setDataTypes(List<TSDataType> dataTypes) {
+    this.dataTypes = dataTypes;
+  }
+
+  public List<TSEncoding> getEncodings() {
+    return encodings;
+  }
+
+  public void setEncodings(List<TSEncoding> encodings) {
+    this.encodings = encodings;
+  }
+
+  public List<CompressionType> getCompressors() {
+    return compressors;
+  }
+
+  public void setCompressors(List<CompressionType> compressors) {
+    this.compressors = compressors;
+  }
+
+  public List<String> getAlias() {
+    return alias;
+  }
+
+  public void setAlias(List<String> alias) {
+    this.alias = alias;
+  }
+
+  public List<Map<String, String>> getProps() {
+    return props;
+  }
+
+  public void setProps(List<Map<String, String>> props) {
+    this.props = props;
+  }
+
+  public List<Map<String, String>> getTags() {
+    return tags;
+  }
+
+  public void setTags(List<Map<String, String>> tags) {
+    this.tags = tags;
+  }
+
+  public List<Map<String, String>> getAttributes() {
+    return attributes;
+  }
+
+  public void setAttributes(List<Map<String, String>> attributes) {
+    this.attributes = attributes;
+  }
+
+  public List<Integer> getIndexes() {
+    return indexes;
+  }
+
+  public void setIndexes(List<Integer> indexes) {
+    this.indexes = indexes;
+  }
+
+  public Map<Integer, Boolean> getResults() {
+    return results;
+  }
+
+  public void setResults(Map<Integer, Boolean> results) {
+    this.results = results;
+  }
+
+  @Override
+  public void serialize(DataOutputStream stream) throws IOException {
+    int type = PhysicalPlanType.MULTI_CREATE_TIMESERIES.ordinal();
+    stream.write(type);
+    stream.writeInt(paths.size());
+
+    for (PartialPath path : paths) {
+      putString(stream, path.getFullPath());
+    }
+
+    for (TSDataType dataType : dataTypes) {
+      stream.write(dataType.ordinal());
+    }
+
+    for (TSEncoding encoding : encodings) {
+      stream.write(encoding.ordinal());
+    }
+
+    for (CompressionType compressor : compressors) {
+      stream.write(compressor.ordinal());
+    }
+
+    for (String name : alias) {
+      putString(stream, name);
+    }
+
+    if (props != null) {
+      stream.write(1);
+      for (Map<String, String> prop : props) {
+        ReadWriteIOUtils.write(prop, stream);
+      }
+    } else {
+      stream.write(0);
+    }
+
+    if (tags != null) {
+      stream.write(1);
+      for (Map<String, String> tag : tags) {
+        ReadWriteIOUtils.write(tag, stream);
+      }
+    } else {
+      stream.write(0);
+    }
+
+    if (attributes != null) {
+      stream.write(1);
+      for (Map<String, String> attribute : attributes) {
+        ReadWriteIOUtils.write(attribute, stream);
+      }
+    } else {
+      stream.write(0);
+    }
+  }
+
+  @Override
+  public void serialize(ByteBuffer buffer) {
+    int type = PhysicalPlanType.MULTI_CREATE_TIMESERIES.ordinal();
+    buffer.put((byte) type);
+    buffer.putInt(paths.size());
+
+    for (PartialPath path : paths) {
+      putString(buffer, path.getFullPath());
+    }
+
+    for (TSDataType dataType : dataTypes) {
+      buffer.put((byte) dataType.ordinal());
+    }
+
+    for (TSEncoding encoding : encodings) {
+      buffer.put((byte) encoding.ordinal());
+    }
+
+    for (CompressionType compressor : compressors) {
+      buffer.put((byte) compressor.ordinal());
+    }
+
+    for (String name : alias) {
+      putString(buffer, name);
+    }

Review comment:
       Same.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org