You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2021/08/14 07:08:13 UTC

[iotdb] branch master updated: [IOTDB-1475][IOTDB-1556] Path name check and Abort auto create device while setStorageGroup exception (#3712)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new a43045b  [IOTDB-1475][IOTDB-1556] Path name check and Abort auto create device while setStorageGroup exception (#3712)
a43045b is described below

commit a43045b32929b4f47c80320264a0b8987b734f34
Author: zyk990424 <38...@users.noreply.github.com>
AuthorDate: Sat Aug 14 15:07:46 2021 +0800

    [IOTDB-1475][IOTDB-1556] Path name check and Abort auto create device while setStorageGroup exception (#3712)
---
 .../apache/iotdb/cluster/metadata/CMManager.java   |   2 +-
 .../iotdb/db/engine/memtable/AbstractMemTable.java |   2 +-
 .../exception/metadata/IllegalPathException.java   |   6 +
 .../metadata/StorageGroupAlreadySetException.java  |   7 +
 .../org/apache/iotdb/db/metadata/MManager.java     |   6 +
 .../java/org/apache/iotdb/db/metadata/MTree.java   |  87 +----------
 .../org/apache/iotdb/db/metadata/PartialPath.java  |   1 +
 .../iotdb/db/metadata/mnode/InternalMNode.java     |   2 +-
 .../db/metadata/template/TemplateManager.java      |  10 ++
 .../iotdb/db/metadata/utils/MetaFormatUtils.java   | 169 +++++++++++++++++++++
 .../iotdb/db/metadata/{ => utils}/MetaUtils.java   |   5 +-
 .../org/apache/iotdb/db/utils/FilePathUtils.java   |   2 +-
 .../iotdb/db/metadata/MManagerBasicTest.java       | 144 ++++++++++++++++++
 .../apache/iotdb/db/metadata/MetaUtilsTest.java    |   1 +
 .../apache/iotdb/db/metadata/mnode/MNodeTest.java  |   2 +-
 15 files changed, 359 insertions(+), 87 deletions(-)

diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
index 95f085c..27325af 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/metadata/CMManager.java
@@ -41,13 +41,13 @@ import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.metadata.PathNotExistException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.metadata.MManager;
-import org.apache.iotdb.db.metadata.MetaUtils;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.metadata.VectorPartialPath;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
 import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.InternalMNode;
 import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
+import org.apache.iotdb.db.metadata.utils.MetaUtils;
 import org.apache.iotdb.db.qp.constant.SQLConstant;
 import org.apache.iotdb.db.qp.physical.BatchPlan;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
index e2b7638..984ce32 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
@@ -22,9 +22,9 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 import org.apache.iotdb.db.exception.WriteProcessException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.MetaUtils;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.utils.MetaUtils;
 import org.apache.iotdb.db.qp.physical.crud.InsertRowPlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertTabletPlan;
 import org.apache.iotdb.db.rescon.TVListAllocator;
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/IllegalPathException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/IllegalPathException.java
index 4b7ddfb..218aa56 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/metadata/IllegalPathException.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/IllegalPathException.java
@@ -29,4 +29,10 @@ public class IllegalPathException extends MetadataException {
     errorCode = TSStatusCode.PATH_ILLEGAL.getStatusCode();
     this.isUserException = true;
   }
+
+  public IllegalPathException(String path, String reason) {
+    super(String.format("%s is not a legal path, because %s", path, reason));
+    errorCode = TSStatusCode.PATH_ILLEGAL.getStatusCode();
+    this.isUserException = true;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupAlreadySetException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupAlreadySetException.java
index 4222489..bcf8970 100644
--- a/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupAlreadySetException.java
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupAlreadySetException.java
@@ -25,12 +25,19 @@ public class StorageGroupAlreadySetException extends MetadataException {
 
   private static final long serialVersionUID = 9110669164701929779L;
 
+  private boolean hasChild = false;
+
   public StorageGroupAlreadySetException(String path) {
     super(getMessage(path, false), TSStatusCode.PATH_ALREADY_EXIST_ERROR.getStatusCode());
   }
 
   public StorageGroupAlreadySetException(String path, boolean hasChild) {
     super(getMessage(path, hasChild), TSStatusCode.PATH_ALREADY_EXIST_ERROR.getStatusCode());
+    this.hasChild = hasChild;
+  }
+
+  public boolean isHasChild() {
+    return hasChild;
   }
 
   private static String getMessage(String path, boolean hasChild) {
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 6e585cf..71045db 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
@@ -30,6 +30,7 @@ import org.apache.iotdb.db.metadata.mnode.*;
 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.metadata.utils.MetaUtils;
 import org.apache.iotdb.db.monitor.MonitorConstants;
 import org.apache.iotdb.db.qp.constant.SQLConstant;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
@@ -1248,6 +1249,11 @@ public class MManager {
       }
       return node;
     } catch (StorageGroupAlreadySetException e) {
+      if (e.isHasChild()) {
+        // if setStorageGroup failure is because of child, the deviceNode should not be created.
+        // Timeseries can't be create under a deviceNode without storageGroup.
+        throw e;
+      }
       // ignore set storage group concurrently
       node = mtree.getDeviceNodeWithAutoCreating(path, sgLevel);
       if (!(node.isStorageGroup())) {
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 76709a4..755605f 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
@@ -18,13 +18,11 @@
  */
 package org.apache.iotdb.db.metadata;
 
-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.fileSystem.SystemFileFactory;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 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;
@@ -42,6 +40,8 @@ import org.apache.iotdb.db.metadata.mnode.InternalMNode;
 import org.apache.iotdb.db.metadata.mnode.MeasurementMNode;
 import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
 import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
+import org.apache.iotdb.db.metadata.utils.MetaUtils;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
 import org.apache.iotdb.db.qp.physical.sys.MeasurementMNodePlan;
@@ -91,13 +91,8 @@ 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.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 {
@@ -285,7 +280,7 @@ public class MTree implements Serializable {
     if (nodeNames.length <= 2 || !nodeNames[0].equals(root.getName())) {
       throw new IllegalPathException(path.getFullPath());
     }
-    checkTimeseries(path.getFullPath());
+    MetaFormatUtils.checkTimeseries(path);
     IMNode cur = root;
     boolean hasSetStorageGroup = false;
     Template upperTemplate = cur.getSchemaTemplate();
@@ -324,9 +319,7 @@ public class MTree implements Serializable {
           path.getFullPath() + " ( which is incompatible with template )");
     }
 
-    if (props != null && props.containsKey(LOSS) && props.get(LOSS).equals(SDT)) {
-      checkSDTFormat(path.getFullPath(), props);
-    }
+    MetaFormatUtils.checkTimeseriesProps(path.getFullPath(), props);
 
     String leafName = nodeNames[nodeNames.length - 1];
 
@@ -375,10 +368,8 @@ public class MTree implements Serializable {
     if (deviceNodeNames.length <= 1 || !deviceNodeNames[0].equals(root.getName())) {
       throw new IllegalPathException(devicePath.getFullPath());
     }
-    checkTimeseries(devicePath.getFullPath());
-    for (String measurement : measurements) {
-      checkTimeseries(measurement);
-    }
+    MetaFormatUtils.checkTimeseries(devicePath);
+    MetaFormatUtils.checkSchemaMeasurementNames(measurements);
     IMNode cur = root;
     boolean hasSetStorageGroup = false;
     // e.g, devicePath = root.sg.d1, create internal nodes and set cur to d1 node
@@ -432,61 +423,6 @@ public class MTree implements Serializable {
     }
   }
 
-  private void checkTimeseries(String timeseries) throws IllegalPathException {
-    if (!IoTDBConfig.NODE_PATTERN.matcher(timeseries).matches()) {
-      throw new IllegalPathException(
-          String.format("The timeseries name contains unsupported character. %s", timeseries));
-    }
-  }
-
-  // check if sdt parameters are valid
-  private void checkSDTFormat(String path, Map<String, String> props)
-      throws IllegalParameterOfPathException {
-    if (!props.containsKey(SDT_COMP_DEV)) {
-      throw new IllegalParameterOfPathException("SDT compression deviation is required", path);
-    }
-
-    try {
-      double d = Double.parseDouble(props.get(SDT_COMP_DEV));
-      if (d < 0) {
-        throw new IllegalParameterOfPathException(
-            "SDT compression deviation cannot be negative", path);
-      }
-    } catch (NumberFormatException e) {
-      throw new IllegalParameterOfPathException("SDT compression deviation formatting error", path);
-    }
-
-    long compMinTime = sdtCompressionTimeFormat(SDT_COMP_MIN_TIME, props, path);
-    long compMaxTime = sdtCompressionTimeFormat(SDT_COMP_MAX_TIME, props, path);
-
-    if (compMaxTime <= compMinTime) {
-      throw new IllegalParameterOfPathException(
-          "SDT compression maximum time needs to be greater than compression minimum time", path);
-    }
-  }
-
-  private long sdtCompressionTimeFormat(String compTime, Map<String, String> props, String path)
-      throws IllegalParameterOfPathException {
-    boolean isCompMaxTime = compTime.equals(SDT_COMP_MAX_TIME);
-    long time = isCompMaxTime ? Long.MAX_VALUE : 0;
-    String s = isCompMaxTime ? "maximum" : "minimum";
-    if (props.containsKey(compTime)) {
-      try {
-        time = Long.parseLong(props.get(compTime));
-        if (time < 0) {
-          throw new IllegalParameterOfPathException(
-              String.format("SDT compression %s time cannot be negative", s), path);
-        }
-      } catch (IllegalParameterOfPathException e) {
-        throw new IllegalParameterOfPathException(
-            String.format("SDT compression %s time formatting error", s), path);
-      }
-    } else {
-      logger.info("{} enabled SDT but did not set compression {} time", path, s);
-    }
-    return time;
-  }
-
   /**
    * Add an interval path to MTree. This is only used for automatically creating schema
    *
@@ -562,7 +498,7 @@ public class MTree implements Serializable {
    */
   void setStorageGroup(PartialPath path) throws MetadataException {
     String[] nodeNames = path.getNodes();
-    checkStorageGroup(path.getFullPath());
+    MetaFormatUtils.checkStorageGroup(path.getFullPath());
     if (nodeNames.length <= 1 || !nodeNames[0].equals(root.getName())) {
       throw new IllegalPathException(path.getFullPath());
     }
@@ -610,15 +546,6 @@ public class MTree implements Serializable {
     }
   }
 
-  private void checkStorageGroup(String storageGroup) throws IllegalPathException {
-    if (!IoTDBConfig.STORAGE_GROUP_PATTERN.matcher(storageGroup).matches()) {
-      throw new IllegalPathException(
-          String.format(
-              "The storage group name can only be characters, numbers and underscores. %s",
-              storageGroup));
-    }
-  }
-
   /** Delete a storage group */
   List<IMeasurementMNode> deleteStorageGroup(PartialPath path) throws MetadataException {
     IMNode cur = getNodeByPath(path);
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java b/server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
index d95221b..97b3afd 100755
--- a/server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/PartialPath.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.metadata;
 
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.utils.MetaUtils;
 import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 import org.apache.iotdb.tsfile.read.common.Path;
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
index 72493a1..fabfda6 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/InternalMNode.java
@@ -20,10 +20,10 @@ package org.apache.iotdb.db.metadata.mnode;
 
 import org.apache.iotdb.db.exception.metadata.AlignedTimeseriesException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.metadata.MetaUtils;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.metadata.logfile.MLogWriter;
 import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.metadata.utils.MetaUtils;
 import org.apache.iotdb.db.qp.physical.sys.MNodePlan;
 
 import java.io.IOException;
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
index 9fdffef..90419b8 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/template/TemplateManager.java
@@ -22,9 +22,11 @@ import org.apache.iotdb.db.exception.metadata.DuplicatedTemplateException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.metadata.UndefinedTemplateException;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.utils.MetaFormatUtils;
 import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
 import org.apache.iotdb.db.utils.TestOnly;
 
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -54,6 +56,14 @@ public class TemplateManager {
   private TemplateManager() {}
 
   public void createSchemaTemplate(CreateTemplatePlan plan) throws MetadataException {
+    // check schema and measurement name before create template
+    for (String schemaNames : plan.getSchemaNames()) {
+      MetaFormatUtils.checkNodeName(schemaNames);
+    }
+    for (List<String> measurements : plan.getMeasurements()) {
+      MetaFormatUtils.checkSchemaMeasurementNames(measurements);
+    }
+
     Template template = new Template(plan);
     if (templateMap.putIfAbsent(plan.getName(), template) != null) {
       // already have template
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaFormatUtils.java b/server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaFormatUtils.java
new file mode 100644
index 0000000..a6288d4
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaFormatUtils.java
@@ -0,0 +1,169 @@
+/*
+ * 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.metadata.utils;
+
+import org.apache.iotdb.db.conf.IoTDBConfig;
+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.metadata.PartialPath;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.LOSS;
+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;
+
+public class MetaFormatUtils {
+
+  private static final Logger logger = LoggerFactory.getLogger(MetaFormatUtils.class);
+
+  public static String[] RESERVED_NODE_NAMES = {"time", "timestamp"};
+
+  /** check whether the given path is well formatted */
+  public static void checkTimeseries(PartialPath timeseries) throws IllegalPathException {
+    try {
+      checkCharacters(timeseries.getFullPath());
+    } catch (MetadataException e) {
+      throw new IllegalPathException(timeseries.getFullPath(), e.getMessage());
+    }
+    for (String name : timeseries.getNodes()) {
+      try {
+        checkReservedNames(name);
+        checkNameFormat(name);
+      } catch (MetadataException e) {
+        throw new IllegalPathException(timeseries.getFullPath(), e.getMessage());
+      }
+    }
+  }
+
+  /** check the characters in path or single node */
+  private static void checkCharacters(String timeseries) throws MetadataException {
+    if (!IoTDBConfig.NODE_PATTERN.matcher(timeseries).matches()) {
+      throw new MetadataException(
+          String.format("The name, %s, contains unsupported character.", timeseries));
+    }
+  }
+
+  /** check whether the node name uses the reserved name */
+  private static void checkReservedNames(String name) throws MetadataException {
+    String processedName = name.trim().toLowerCase(Locale.ENGLISH);
+    for (String reservedName : RESERVED_NODE_NAMES) {
+      if (reservedName.equals(processedName)) {
+        throw new MetadataException(String.format("%s is an illegal name.", name));
+      }
+    }
+  }
+
+  /** check whether the node name uses "." correctly */
+  private static void checkNameFormat(String name) throws MetadataException {
+    if (name.contains(".") && !(name.startsWith("\"") && name.endsWith("\""))) {
+      throw new MetadataException(String.format("%s is an illegal name.", name));
+    }
+  }
+
+  /** check whether the node name is well formatted */
+  public static void checkNodeName(String name) throws MetadataException {
+    checkCharacters(name);
+    checkReservedNames(name);
+    checkNameFormat(name);
+  }
+
+  /** check whether the measurement ids in schema is well formatted */
+  public static void checkSchemaMeasurementNames(List<String> measurements)
+      throws MetadataException {
+    for (String measurement : measurements) {
+      checkNodeName(measurement);
+    }
+  }
+
+  /** check whether the storageGroup name uses illegal characters */
+  public static void checkStorageGroup(String storageGroup) throws IllegalPathException {
+    if (!IoTDBConfig.STORAGE_GROUP_PATTERN.matcher(storageGroup).matches()) {
+      throw new IllegalPathException(
+          String.format(
+              "The storage group name can only be characters, numbers and underscores. %s",
+              storageGroup));
+    }
+  }
+
+  /** check props when creating timeseries */
+  public static void checkTimeseriesProps(String path, Map<String, String> props)
+      throws IllegalParameterOfPathException {
+    if (props != null && props.containsKey(LOSS) && props.get(LOSS).equals(SDT)) {
+      checkSDTFormat(path, props);
+    }
+  }
+
+  // check if sdt parameters are valid
+  private static void checkSDTFormat(String path, Map<String, String> props)
+      throws IllegalParameterOfPathException {
+    if (!props.containsKey(SDT_COMP_DEV)) {
+      throw new IllegalParameterOfPathException("SDT compression deviation is required", path);
+    }
+
+    try {
+      double d = Double.parseDouble(props.get(SDT_COMP_DEV));
+      if (d < 0) {
+        throw new IllegalParameterOfPathException(
+            "SDT compression deviation cannot be negative", path);
+      }
+    } catch (NumberFormatException e) {
+      throw new IllegalParameterOfPathException("SDT compression deviation formatting error", path);
+    }
+
+    long compMinTime = sdtCompressionTimeFormat(SDT_COMP_MIN_TIME, props, path);
+    long compMaxTime = sdtCompressionTimeFormat(SDT_COMP_MAX_TIME, props, path);
+
+    if (compMaxTime <= compMinTime) {
+      throw new IllegalParameterOfPathException(
+          "SDT compression maximum time needs to be greater than compression minimum time", path);
+    }
+  }
+
+  private static long sdtCompressionTimeFormat(
+      String compTime, Map<String, String> props, String path)
+      throws IllegalParameterOfPathException {
+    boolean isCompMaxTime = compTime.equals(SDT_COMP_MAX_TIME);
+    long time = isCompMaxTime ? Long.MAX_VALUE : 0;
+    String s = isCompMaxTime ? "maximum" : "minimum";
+    if (props.containsKey(compTime)) {
+      try {
+        time = Long.parseLong(props.get(compTime));
+        if (time < 0) {
+          throw new IllegalParameterOfPathException(
+              String.format("SDT compression %s time cannot be negative", s), path);
+        }
+      } catch (IllegalParameterOfPathException e) {
+        throw new IllegalParameterOfPathException(
+            String.format("SDT compression %s time formatting error", s), path);
+      }
+    } else {
+      logger.info("{} enabled SDT but did not set compression {} time", path, s);
+    }
+    return time;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MetaUtils.java b/server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaUtils.java
similarity index 96%
rename from server/src/main/java/org/apache/iotdb/db/metadata/MetaUtils.java
rename to server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaUtils.java
index e70848c..7ee8ee1 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MetaUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/utils/MetaUtils.java
@@ -16,11 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.metadata;
+package org.apache.iotdb.db.metadata.utils;
 
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
 import org.apache.iotdb.db.utils.TestOnly;
 
@@ -37,7 +38,7 @@ public class MetaUtils {
 
   private MetaUtils() {}
 
-  static String getNodeRegByIdx(int idx, String[] nodes) {
+  public static String getNodeRegByIdx(int idx, String[] nodes) {
     return idx >= nodes.length ? PATH_WILDCARD : nodes[idx];
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/FilePathUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/FilePathUtils.java
index e7038c3..e5baeab 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/FilePathUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/FilePathUtils.java
@@ -22,8 +22,8 @@ import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.metadata.MetaUtils;
 import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.utils.MetaUtils;
 import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
 import org.apache.iotdb.db.query.aggregation.AggregateResult;
 import org.apache.iotdb.db.query.aggregation.AggregationType;
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 d424e07..a121683 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
@@ -24,7 +24,9 @@ import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.metadata.mnode.IMNode;
 import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.utils.MetaUtils;
 import org.apache.iotdb.db.qp.physical.crud.CreateTemplatePlan;
+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.SetSchemaTemplatePlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
@@ -1692,6 +1694,55 @@ public class MManagerBasicTest {
   }
 
   @Test
+  public void testCreateAlignedTimeseriesWithIllegalNames() throws Exception {
+    MManager manager = IoTDB.metaManager;
+    manager.setStorageGroup(new PartialPath("root.laptop"));
+    PartialPath deviceId = new PartialPath("root.laptop.d1");
+    String[] measurementIds = {"a.b", "time", "timestamp", "TIME", "TIMESTAMP"};
+    for (String measurementId : measurementIds) {
+      PartialPath path = deviceId.concatNode(measurementId);
+      try {
+        manager.createAlignedTimeSeries(
+            path,
+            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")),
+            compressionType);
+        fail();
+      } catch (Exception e) {
+        Assert.assertEquals(
+            String.format(
+                "%s is not a legal path, because %s",
+                path.getFullPath(), String.format("%s is an illegal name.", measurementId)),
+            e.getMessage());
+      }
+    }
+
+    PartialPath path = deviceId.concatNode("t1");
+    for (String measurementId : measurementIds) {
+      try {
+        manager.createAlignedTimeSeries(
+            path,
+            Arrays.asList(measurementId, "s2", "s3"),
+            Arrays.asList(
+                TSDataType.valueOf("FLOAT"),
+                TSDataType.valueOf("INT64"),
+                TSDataType.valueOf("INT32")),
+            Arrays.asList(
+                TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE"), TSEncoding.valueOf("RLE")),
+            compressionType);
+        fail();
+      } catch (Exception e) {
+        Assert.assertEquals(String.format("%s is an illegal name.", measurementId), e.getMessage());
+      }
+    }
+  }
+
+  @Test
   public void testGetStorageGroupNodeByPath() {
     MManager manager = IoTDB.metaManager;
     PartialPath partialPath = null;
@@ -1724,4 +1775,97 @@ public class MManagerBasicTest {
       fail(e.getMessage());
     }
   }
+
+  @Test
+  public void testMeasurementIdWhileInsert() throws Exception {
+    MManager manager = IoTDB.metaManager;
+
+    PartialPath deviceId = new PartialPath("root.sg.d");
+    InsertPlan insertPlan;
+
+    insertPlan = getInsertPlan("\"a.b\"");
+    manager.getSeriesSchemasAndReadLockDevice(insertPlan);
+    assertTrue(manager.isPathExist(deviceId.concatNode("\"a.b\"")));
+
+    String[] illegalMeasurementIds = {"a.b", "time", "timestamp", "TIME", "TIMESTAMP"};
+    for (String measurementId : illegalMeasurementIds) {
+      insertPlan = getInsertPlan(measurementId);
+      try {
+        manager.getSeriesSchemasAndReadLockDevice(insertPlan);
+        assertFalse(manager.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() {
+    MManager manager = IoTDB.metaManager;
+    String[] illegalSchemaNames = {"a.b", "time", "timestamp", "TIME", "TIMESTAMP"};
+    for (String schemaName : illegalSchemaNames) {
+      CreateTemplatePlan plan = getCreateTemplatePlan(schemaName);
+      try {
+        manager.createSchemaTemplate(plan);
+      } catch (MetadataException e) {
+        Assert.assertEquals(String.format("%s is an illegal name.", schemaName), e.getMessage());
+      }
+    }
+  }
+
+  private CreateTemplatePlan getCreateTemplatePlan(String schemaName) {
+    List<List<String>> measurementList = new ArrayList<>();
+    measurementList.add(Collections.singletonList("s0"));
+
+    List<List<TSDataType>> dataTypeList = new ArrayList<>();
+    dataTypeList.add(Collections.singletonList(TSDataType.INT32));
+
+    List<List<TSEncoding>> encodingList = new ArrayList<>();
+    encodingList.add(Collections.singletonList(TSEncoding.RLE));
+
+    List<CompressionType> compressionTypes = new ArrayList<>();
+    compressionTypes.add(compressionType);
+
+    List<String> schemaNames = new ArrayList<>();
+    schemaNames.add(schemaName);
+
+    return new CreateTemplatePlan(
+        "template1", schemaNames, measurementList, dataTypeList, encodingList, compressionTypes);
+  }
+
+  @Test
+  public void testDeviceNodeAfterAutoCreateTimeseriesFailure() throws Exception {
+    MManager manager = IoTDB.metaManager;
+
+    PartialPath sg1 = new PartialPath("root.a.sg");
+    manager.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 {
+      manager.getSeriesSchemasAndReadLockDevice(insertPlan);
+      fail();
+    } catch (MetadataException e) {
+      Assert.assertEquals(
+          "some children of root.a have already been set to storage group", e.getMessage());
+      Assert.assertFalse(manager.isPathExist(new PartialPath("root.a.d")));
+    }
+  }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MetaUtilsTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MetaUtilsTest.java
index 61e72b5..cc5085b 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MetaUtilsTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MetaUtilsTest.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.metadata;
 
 import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.mnode.InternalMNode;
+import org.apache.iotdb.db.metadata.utils.MetaUtils;
 
 import org.junit.Assert;
 import org.junit.Test;
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/mnode/MNodeTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/mnode/MNodeTest.java
index 1a32b95..75650ff 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/mnode/MNodeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/mnode/MNodeTest.java
@@ -18,7 +18,7 @@
  */
 package org.apache.iotdb.db.metadata.mnode;
 
-import org.apache.iotdb.db.metadata.MetaUtils;
+import org.apache.iotdb.db.metadata.utils.MetaUtils;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.junit.Before;