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

[iotdb] branch master updated: [IOTDB-3268] Refactor measurement check logic in session api for better performance (#5995)

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

rong 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 44885d5a3d [IOTDB-3268] Refactor measurement check logic in session api for better performance (#5995)
44885d5a3d is described below

commit 44885d5a3d8cfae37d18b344626b46fe752e002a
Author: Liao Lanyu <48...@users.noreply.github.com>
AuthorDate: Wed May 25 11:23:21 2022 +0800

    [IOTDB-3268] Refactor measurement check logic in session api for better performance (#5995)
---
 .../thrift/impl/DataNodeTSIServiceImpl.java        | 136 ++++++++++++++++----
 .../db/service/thrift/impl/TSServiceImpl.java      | 143 +++++++++++++++------
 2 files changed, 218 insertions(+), 61 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeTSIServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeTSIServiceImpl.java
index 37c51deef6..86bc50adef 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeTSIServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/DataNodeTSIServiceImpl.java
@@ -21,6 +21,8 @@ package org.apache.iotdb.db.service.thrift.impl;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.IoTDBException;
+import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.utils.PathUtils;
 import org.apache.iotdb.db.auth.AuthorityChecker;
 import org.apache.iotdb.db.conf.IoTDBConfig;
@@ -98,6 +100,7 @@ import org.apache.iotdb.service.rpc.thrift.TSRawDataQueryReq;
 import org.apache.iotdb.service.rpc.thrift.TSSetSchemaTemplateReq;
 import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
 import org.apache.iotdb.service.rpc.thrift.TSUnsetSchemaTemplateReq;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
@@ -115,6 +118,7 @@ import static org.apache.iotdb.db.service.basic.ServiceProvider.CURRENT_RPC_VERS
 import static org.apache.iotdb.db.service.basic.ServiceProvider.QUERY_FREQUENCY_RECORDER;
 import static org.apache.iotdb.db.service.basic.ServiceProvider.QUERY_TIME_MANAGER;
 import static org.apache.iotdb.db.service.basic.ServiceProvider.SLOW_SQL_LOGGER;
+import static org.apache.iotdb.db.utils.ErrorHandlingUtils.onIoTDBException;
 import static org.apache.iotdb.db.utils.ErrorHandlingUtils.onNPEOrUnexpectedException;
 import static org.apache.iotdb.db.utils.ErrorHandlingUtils.onQueryException;
 
@@ -327,7 +331,7 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
       }
 
       // measurementAlias is also a nodeName
-      isLegalMeasurements(Collections.singletonList(req.getMeasurementAlias()));
+      isLegalSingleMeasurements(Collections.singletonList(req.getMeasurementAlias()));
       // Step 1: transfer from TSCreateTimeseriesReq to Statement
       CreateTimeSeriesStatement statement =
           (CreateTimeSeriesStatement) StatementGenerator.createStatement(req);
@@ -350,6 +354,8 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
               SCHEMA_FETCHER);
 
       return result.status;
+    } catch (IoTDBException e) {
+      return onIoTDBException(e, OperationType.INSERT_RECORDS, e.getErrorCode());
     } catch (Exception e) {
       return onNPEOrUnexpectedException(
           e, OperationType.CREATE_TIMESERIES, TSStatusCode.EXECUTE_STATEMENT_ERROR);
@@ -383,9 +389,9 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
       }
 
       // check whether measurement is legal according to syntax convention
-      isLegalMeasurements(req.getMeasurementAlias());
+      isLegalSingleMeasurements(req.getMeasurementAlias());
 
-      isLegalMeasurements(req.getMeasurements());
+      isLegalSingleMeasurements(req.getMeasurements());
 
       // Step 1: transfer from CreateAlignedTimeSeriesReq to Statement
       CreateAlignedTimeSeriesStatement statement =
@@ -409,6 +415,8 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
               SCHEMA_FETCHER);
 
       return result.status;
+    } catch (IoTDBException e) {
+      return onIoTDBException(e, OperationType.INSERT_RECORDS, e.getErrorCode());
     } catch (Exception e) {
       return onNPEOrUnexpectedException(
           e, OperationType.CREATE_ALIGNED_TIMESERIES, TSStatusCode.EXECUTE_STATEMENT_ERROR);
@@ -431,7 +439,7 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
       }
 
       // check whether measurement is legal according to syntax convention
-      isLegalMeasurements(req.getMeasurementAliasList());
+      isLegalSingleMeasurements(req.getMeasurementAliasList());
 
       // Step 1: transfer from CreateMultiTimeSeriesReq to Statement
       CreateMultiTimeSeriesStatement statement =
@@ -455,6 +463,8 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
               SCHEMA_FETCHER);
 
       return result.status;
+    } catch (IoTDBException e) {
+      return onIoTDBException(e, OperationType.INSERT_RECORDS, e.getErrorCode());
     } catch (Exception e) {
       return onNPEOrUnexpectedException(
           e, OperationType.CREATE_MULTI_TIMESERIES, TSStatusCode.EXECUTE_STATEMENT_ERROR);
@@ -637,7 +647,7 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
       }
 
       // check whether measurement is legal according to syntax convention
-      isLegalMeasurementLists(req.getMeasurementsList());
+      isLegalSingleMeasurementLists(req.getMeasurementsList());
 
       // Step 1: TODO(INSERT) transfer from TSInsertTabletsReq to Statement
       InsertRowsStatement statement = (InsertRowsStatement) StatementGenerator.createStatement(req);
@@ -660,6 +670,8 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
               SCHEMA_FETCHER);
 
       return result.status;
+    } catch (IoTDBException e) {
+      return onIoTDBException(e, OperationType.INSERT_RECORDS, e.getErrorCode());
     } catch (Exception e) {
       return onNPEOrUnexpectedException(
           e, OperationType.INSERT_RECORDS, TSStatusCode.EXECUTE_STATEMENT_ERROR);
@@ -685,7 +697,7 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
       }
 
       // check whether measurement is legal according to syntax convention
-      isLegalMeasurementLists(req.getMeasurementsList());
+      isLegalSingleMeasurementLists(req.getMeasurementsList());
 
       // Step 1: TODO(INSERT) transfer from TSInsertTabletsReq to Statement
       InsertRowsOfOneDeviceStatement statement =
@@ -709,6 +721,8 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
               SCHEMA_FETCHER);
 
       return result.status;
+    } catch (IoTDBException e) {
+      return onIoTDBException(e, OperationType.INSERT_RECORDS, e.getErrorCode());
     } catch (Exception e) {
       return onNPEOrUnexpectedException(
           e, OperationType.INSERT_RECORDS_OF_ONE_DEVICE, TSStatusCode.EXECUTE_STATEMENT_ERROR);
@@ -734,7 +748,7 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
       }
 
       // check whether measurement is legal according to syntax convention
-      isLegalMeasurementLists(req.getMeasurementsList());
+      isLegalSingleMeasurementLists(req.getMeasurementsList());
 
       // Step 1: TODO(INSERT) transfer from TSInsertTabletsReq to Statement
       InsertRowsOfOneDeviceStatement statement =
@@ -758,6 +772,8 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
               SCHEMA_FETCHER);
 
       return result.status;
+    } catch (IoTDBException e) {
+      return onIoTDBException(e, OperationType.INSERT_RECORDS, e.getErrorCode());
     } catch (Exception e) {
       return onNPEOrUnexpectedException(
           e,
@@ -783,7 +799,7 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
           req.getTimestamp());
 
       // check whether measurement is legal according to syntax convention
-      isLegalMeasurements(req.getMeasurements());
+      isLegalSingleMeasurements(req.getMeasurements());
 
       InsertRowStatement statement = (InsertRowStatement) StatementGenerator.createStatement(req);
 
@@ -805,6 +821,8 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
               SCHEMA_FETCHER);
 
       return result.status;
+    } catch (IoTDBException e) {
+      return onIoTDBException(e, OperationType.INSERT_RECORDS, e.getErrorCode());
     } catch (Exception e) {
       return onNPEOrUnexpectedException(
           e, OperationType.INSERT_RECORD, TSStatusCode.EXECUTE_STATEMENT_ERROR);
@@ -821,7 +839,7 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
         return getNotLoggedInStatus();
       }
 
-      isLegalMeasurementLists(req.getMeasurementsList());
+      isLegalSingleMeasurementLists(req.getMeasurementsList());
 
       // Step 1: TODO(INSERT) transfer from TSInsertTabletsReq to Statement
       InsertMultiTabletsStatement statement =
@@ -845,6 +863,8 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
               SCHEMA_FETCHER);
 
       return result.status;
+    } catch (IoTDBException e) {
+      return onIoTDBException(e, OperationType.INSERT_RECORDS, e.getErrorCode());
     } catch (Exception e) {
       return onNPEOrUnexpectedException(
           e, OperationType.INSERT_TABLETS, TSStatusCode.EXECUTE_STATEMENT_ERROR);
@@ -862,7 +882,7 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
       }
 
       // check whether measurement is legal according to syntax convention
-      isLegalMeasurements(req.getMeasurements());
+      isLegalSingleMeasurements(req.getMeasurements());
       // Step 1: TODO(INSERT) transfer from TSInsertTabletReq to Statement
       InsertTabletStatement statement =
           (InsertTabletStatement) StatementGenerator.createStatement(req);
@@ -885,6 +905,8 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
               SCHEMA_FETCHER);
 
       return result.status;
+    } catch (IoTDBException e) {
+      return onIoTDBException(e, OperationType.INSERT_RECORDS, e.getErrorCode());
     } catch (Exception e) {
       return onNPEOrUnexpectedException(
           e, OperationType.INSERT_TABLET, TSStatusCode.EXECUTE_STATEMENT_ERROR);
@@ -910,7 +932,7 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
       }
 
       // check whether measurement is legal according to syntax convention
-      isLegalMeasurementLists(req.getMeasurementsList());
+      isLegalSingleMeasurementLists(req.getMeasurementsList());
 
       InsertRowsStatement statement = (InsertRowsStatement) StatementGenerator.createStatement(req);
 
@@ -931,6 +953,8 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
               SCHEMA_FETCHER);
 
       return result.status;
+    } catch (IoTDBException e) {
+      return onIoTDBException(e, OperationType.INSERT_RECORDS, e.getErrorCode());
     } catch (Exception e) {
       return onNPEOrUnexpectedException(
           e, OperationType.INSERT_STRING_RECORDS, TSStatusCode.EXECUTE_STATEMENT_ERROR);
@@ -1058,13 +1082,13 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
 
   @Override
   public TSStatus createSchemaTemplate(TSCreateSchemaTemplateReq req) {
-    // todo: check measurement using isLegalMeasurements()
+    // todo: check measurement using isLegalSingleMeasurements()
     throw new UnsupportedOperationException();
   }
 
   @Override
   public TSStatus appendSchemaTemplate(TSAppendSchemaTemplateReq req) {
-    // todo: check measurement using isLegalMeasurements()
+    // todo: check measurement using isLegalSingleMeasurements()
     throw new UnsupportedOperationException();
   }
 
@@ -1108,7 +1132,7 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
           req.getTimestamp());
 
       // check whether measurement is legal according to syntax convention
-      isLegalMeasurements(req.getMeasurements());
+      isLegalSingleMeasurements(req.getMeasurements());
 
       InsertRowStatement statement = (InsertRowStatement) StatementGenerator.createStatement(req);
 
@@ -1130,6 +1154,8 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
               SCHEMA_FETCHER);
 
       return result.status;
+    } catch (IoTDBException e) {
+      return onIoTDBException(e, OperationType.INSERT_RECORDS, e.getErrorCode());
     } catch (Exception e) {
       return onNPEOrUnexpectedException(
           e, OperationType.INSERT_STRING_RECORD, TSStatusCode.EXECUTE_STATEMENT_ERROR);
@@ -1195,28 +1221,90 @@ public class DataNodeTSIServiceImpl implements TSIEventHandler {
     return new QueryId(String.valueOf(id));
   }
 
-  // check whether measurement is legal according to syntax convention
-  protected void isLegalMeasurementLists(List<List<String>> measurementLists) throws TException {
+  /**
+   * check whether measurement is legal according to syntax convention measurement can only be a
+   * single node name
+   */
+  protected void isLegalSingleMeasurementLists(List<List<String>> measurementLists)
+      throws MetadataException {
     if (measurementLists == null) {
       return;
     }
+    StringBuilder path = new StringBuilder(IoTDBConstant.PATH_ROOT);
     for (List<String> measurementList : measurementLists) {
-      isLegalMeasurements(measurementList);
+      for (String measurement : measurementList) {
+        if (measurement != null) {
+          if (measurement.contains(TsFileConstant.PATH_SEPARATOR)
+              && !(measurement.startsWith(TsFileConstant.BACK_QUOTE_STRING)
+                  && measurement.endsWith(TsFileConstant.BACK_QUOTE_STRING))) {
+            throw new IllegalPathException(measurement);
+          } else {
+            path.append(TsFileConstant.PATH_SEPARATOR);
+            path.append(measurement);
+          }
+        }
+      }
+    }
+    try {
+      PathUtils.isLegalPath(path.toString());
+    } catch (IllegalPathException e) {
+      throw new MetadataException("find wrong node name according to syntax convention");
     }
   }
 
-  // check whether measurement is legal according to syntax convention
-  protected void isLegalMeasurements(List<String> measurements) throws TException {
+  /**
+   * check whether measurement is legal according to syntax convention measurement can only be a
+   * single node name
+   */
+  protected void isLegalSingleMeasurements(List<String> measurements) throws MetadataException {
     if (measurements == null) {
       return;
     }
+    StringBuilder path = new StringBuilder(IoTDBConstant.PATH_ROOT);
     for (String measurement : measurements) {
-      try {
-        if (measurement != null) {
-          PathUtils.isLegalPath(measurement);
+      if (measurement != null) {
+        if (measurement.contains(TsFileConstant.PATH_SEPARATOR)
+            && !(measurement.startsWith(TsFileConstant.BACK_QUOTE_STRING)
+                && measurement.endsWith(TsFileConstant.BACK_QUOTE_STRING))) {
+          throw new IllegalPathException(measurement);
+        } else {
+          path.append(TsFileConstant.PATH_SEPARATOR);
+          path.append(measurement);
         }
-      } catch (IllegalPathException e) {
-        throw new TException(e.getMessage());
+      }
+    }
+    try {
+      PathUtils.isLegalPath(path.toString());
+    } catch (IllegalPathException e) {
+      throw new MetadataException("find wrong node name according to syntax convention");
+    }
+  }
+
+  /**
+   * check whether measurement is legal according to syntax convention measurement could be like a.b
+   * (more than one node name), in template?
+   */
+  protected void isLegalMeasurementLists(List<List<String>> measurementLists)
+      throws IllegalPathException {
+    if (measurementLists == null) {
+      return;
+    }
+    for (List<String> measurementList : measurementLists) {
+      isLegalMeasurements(measurementList);
+    }
+  }
+
+  /**
+   * check whether measurement is legal according to syntax convention measurement could be like a.b
+   * (more than one node name), in template?
+   */
+  protected void isLegalMeasurements(List<String> measurements) throws IllegalPathException {
+    if (measurements == null) {
+      return;
+    }
+    for (String measurement : measurements) {
+      if (measurement != null) {
+        PathUtils.isLegalPath(measurement);
       }
     }
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java
index 958e31421a..9c144c1f39 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/TSServiceImpl.java
@@ -122,6 +122,7 @@ import org.apache.iotdb.service.rpc.thrift.TSSetSchemaTemplateReq;
 import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneReq;
 import org.apache.iotdb.service.rpc.thrift.TSTracingInfo;
 import org.apache.iotdb.service.rpc.thrift.TSUnsetSchemaTemplateReq;
+import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
@@ -1174,7 +1175,7 @@ public class TSServiceImpl implements TSIEventHandler {
   }
 
   @Override
-  public TSStatus insertRecords(TSInsertRecordsReq req) throws TException {
+  public TSStatus insertRecords(TSInsertRecordsReq req) {
     if (!SESSION_MANAGER.checkLogin(req.getSessionId())) {
       return getNotLoggedInStatus();
     }
@@ -1187,11 +1188,12 @@ public class TSServiceImpl implements TSIEventHandler {
           req.getTimestamps().get(0));
     }
     boolean allCheckSuccess = true;
-    // check whether measurement is legal according to syntax convention
-    isLegalMeasurementLists(req.getMeasurementsList());
+
     InsertRowsPlan insertRowsPlan = new InsertRowsPlan();
     for (int i = 0; i < req.prefixPaths.size(); i++) {
       try {
+        // check whether measurement is legal according to syntax convention
+        isLegalSingleMeasurements(req.getMeasurementsList().get(i));
         InsertRowPlan plan =
             new InsertRowPlan(
                 new PartialPath(req.getPrefixPaths().get(i)),
@@ -1248,7 +1250,7 @@ public class TSServiceImpl implements TSIEventHandler {
   }
 
   @Override
-  public TSStatus insertRecordsOfOneDevice(TSInsertRecordsOfOneDeviceReq req) throws TException {
+  public TSStatus insertRecordsOfOneDevice(TSInsertRecordsOfOneDeviceReq req) {
     if (!SESSION_MANAGER.checkLogin(req.getSessionId())) {
       return getNotLoggedInStatus();
     }
@@ -1260,11 +1262,11 @@ public class TSServiceImpl implements TSIEventHandler {
           req.prefixPath,
           req.getTimestamps().get(0));
     }
-    // check whether measurement is legal according to syntax convention
-    isLegalMeasurementLists(req.getMeasurementsList());
 
     List<TSStatus> statusList = new ArrayList<>();
     try {
+      // check whether measurement is legal according to syntax convention
+      isLegalSingleMeasurementLists(req.getMeasurementsList());
       InsertRowsOfOneDevicePlan plan =
           new InsertRowsOfOneDevicePlan(
               new PartialPath(req.getPrefixPath()),
@@ -1296,8 +1298,7 @@ public class TSServiceImpl implements TSIEventHandler {
   }
 
   @Override
-  public TSStatus insertStringRecordsOfOneDevice(TSInsertStringRecordsOfOneDeviceReq req)
-      throws TException {
+  public TSStatus insertStringRecordsOfOneDevice(TSInsertStringRecordsOfOneDeviceReq req) {
     if (!SESSION_MANAGER.checkLogin(req.getSessionId())) {
       return getNotLoggedInStatus();
     }
@@ -1309,13 +1310,14 @@ public class TSServiceImpl implements TSIEventHandler {
           req.prefixPath,
           req.getTimestamps().get(0));
     }
-    // check whether measurement is legal according to syntax convention
-    isLegalMeasurementLists(req.getMeasurementsList());
+
     boolean allCheckSuccess = true;
     InsertRowsPlan insertRowsPlan = new InsertRowsPlan();
     for (int i = 0; i < req.timestamps.size(); i++) {
       InsertRowPlan plan = new InsertRowPlan();
       try {
+        // check whether measurement is legal according to syntax convention
+        isLegalSingleMeasurements(req.getMeasurementsList().get(i));
         plan.setDevicePath(new PartialPath(req.getPrefixPath()));
         plan.setTime(req.getTimestamps().get(i));
         addMeasurementAndValue(plan, req.getMeasurementsList().get(i), req.getValuesList().get(i));
@@ -1356,7 +1358,7 @@ public class TSServiceImpl implements TSIEventHandler {
   }
 
   @Override
-  public TSStatus insertStringRecords(TSInsertStringRecordsReq req) throws TException {
+  public TSStatus insertStringRecords(TSInsertStringRecordsReq req) {
     if (!SESSION_MANAGER.checkLogin(req.getSessionId())) {
       return getNotLoggedInStatus();
     }
@@ -1369,13 +1371,13 @@ public class TSServiceImpl implements TSIEventHandler {
           req.getTimestamps().get(0));
     }
 
-    // check whether measurement is legal according to syntax convention
-    isLegalMeasurementLists(req.getMeasurementsList());
     boolean allCheckSuccess = true;
     InsertRowsPlan insertRowsPlan = new InsertRowsPlan();
     for (int i = 0; i < req.prefixPaths.size(); i++) {
       InsertRowPlan plan = new InsertRowPlan();
       try {
+        // check whether measurement is legal according to syntax convention
+        isLegalSingleMeasurements(req.getMeasurementsList().get(i));
         plan.setDevicePath(new PartialPath(req.getPrefixPaths().get(i)));
         plan.setTime(req.getTimestamps().get(i));
         addMeasurementAndValue(plan, req.getMeasurementsList().get(i), req.getValuesList().get(i));
@@ -1485,7 +1487,7 @@ public class TSServiceImpl implements TSIEventHandler {
 
       // check whether measurement is legal according to syntax convention
 
-      isLegalMeasurements(req.getMeasurements());
+      isLegalSingleMeasurements(req.getMeasurements());
 
       InsertRowPlan plan =
           new InsertRowPlan(
@@ -1518,7 +1520,7 @@ public class TSServiceImpl implements TSIEventHandler {
           req.getTimestamp());
 
       // check whether measurement is legal according to syntax convention
-      isLegalMeasurements(req.getMeasurements());
+      isLegalSingleMeasurements(req.getMeasurements());
 
       InsertRowPlan plan = new InsertRowPlan();
       plan.setDevicePath(new PartialPath(req.getPrefixPath()));
@@ -1574,7 +1576,7 @@ public class TSServiceImpl implements TSIEventHandler {
 
       // check whether measurement is legal according to syntax convention
 
-      isLegalMeasurements(req.getMeasurements());
+      isLegalSingleMeasurements(req.getMeasurements());
 
       InsertTabletPlan insertTabletPlan =
           new InsertTabletPlan(new PartialPath(req.getPrefixPath()), req.measurements);
@@ -1623,9 +1625,9 @@ public class TSServiceImpl implements TSIEventHandler {
   }
 
   private InsertTabletPlan constructInsertTabletPlan(TSInsertTabletsReq req, int i)
-      throws IllegalPathException, TException {
+      throws MetadataException {
     // check whether measurement is legal according to syntax convention
-    isLegalMeasurementLists(req.getMeasurementsList());
+    isLegalSingleMeasurementLists(req.getMeasurementsList());
     InsertTabletPlan insertTabletPlan =
         new InsertTabletPlan(new PartialPath(req.prefixPaths.get(i)), req.measurementsList.get(i));
     insertTabletPlan.setTimes(
@@ -1646,8 +1648,7 @@ public class TSServiceImpl implements TSIEventHandler {
   }
 
   /** construct one InsertMultiTabletsPlan and process it */
-  public TSStatus insertTabletsInternally(TSInsertTabletsReq req)
-      throws IllegalPathException, TException {
+  public TSStatus insertTabletsInternally(TSInsertTabletsReq req) throws MetadataException {
     List<InsertTabletPlan> insertTabletPlanList = new ArrayList<>();
     InsertMultiTabletsPlan insertMultiTabletsPlan = new InsertMultiTabletsPlan();
     for (int i = 0; i < req.prefixPaths.size(); i++) {
@@ -1718,7 +1719,7 @@ public class TSServiceImpl implements TSIEventHandler {
       }
 
       // measurementAlias is also a nodeName
-      isLegalMeasurements(Collections.singletonList(req.getMeasurementAlias()));
+      isLegalSingleMeasurements(Collections.singletonList(req.getMeasurementAlias()));
       CreateTimeSeriesPlan plan =
           new CreateTimeSeriesPlan(
               new PartialPath(req.path),
@@ -1759,9 +1760,9 @@ public class TSServiceImpl implements TSIEventHandler {
 
       // check whether measurement is legal according to syntax convention
 
-      isLegalMeasurements(req.getMeasurements());
+      isLegalSingleMeasurements(req.getMeasurements());
 
-      isLegalMeasurements(req.getMeasurementAlias());
+      isLegalSingleMeasurements(req.getMeasurementAlias());
 
       if (AUDIT_LOGGER.isDebugEnabled()) {
         AUDIT_LOGGER.debug(
@@ -1822,7 +1823,7 @@ public class TSServiceImpl implements TSIEventHandler {
 
       // measurementAlias is also a nodeName
 
-      isLegalMeasurements(req.measurementAliasList);
+      isLegalSingleMeasurements(req.measurementAliasList);
 
       CreateMultiTimeSeriesPlan multiPlan = new CreateMultiTimeSeriesPlan();
       List<PartialPath> paths = new ArrayList<>(req.paths.size());
@@ -1946,6 +1947,8 @@ public class TSServiceImpl implements TSIEventHandler {
       TSStatus status = SESSION_MANAGER.checkAuthority(plan, req.getSessionId());
 
       return status != null ? status : executeNonQueryPlan(plan);
+    } catch (IoTDBException e) {
+      return onIoTDBException(e, OperationType.CREATE_SCHEMA_TEMPLATE, e.getErrorCode());
     } catch (Exception e) {
       return onNPEOrUnexpectedException(
           e, OperationType.CREATE_SCHEMA_TEMPLATE, TSStatusCode.EXECUTE_STATEMENT_ERROR);
@@ -1953,9 +1956,13 @@ public class TSServiceImpl implements TSIEventHandler {
   }
 
   @Override
-  public TSStatus appendSchemaTemplate(TSAppendSchemaTemplateReq req) throws TException {
-    // check whether measurement is legal according to syntax convention
-    isLegalMeasurements(req.getMeasurements());
+  public TSStatus appendSchemaTemplate(TSAppendSchemaTemplateReq req) {
+    try {
+      // check whether measurement is legal according to syntax convention
+      isLegalMeasurements(req.getMeasurements());
+    } catch (IoTDBException e) {
+      onIoTDBException(e, OperationType.EXECUTE_NON_QUERY_PLAN, e.getErrorCode());
+    }
 
     int size = req.getMeasurementsSize();
     String[] measurements = new String[size];
@@ -2130,28 +2137,90 @@ public class TSServiceImpl implements TSIEventHandler {
     }
   }
 
-  // check whether measurement is legal according to syntax convention
-  protected void isLegalMeasurementLists(List<List<String>> measurementLists) throws TException {
+  /**
+   * check whether measurement is legal according to syntax convention measurement can only be a
+   * single node name
+   */
+  protected void isLegalSingleMeasurementLists(List<List<String>> measurementLists)
+      throws MetadataException {
     if (measurementLists == null) {
       return;
     }
+    StringBuilder path = new StringBuilder(IoTDBConstant.PATH_ROOT);
     for (List<String> measurementList : measurementLists) {
-      isLegalMeasurements(measurementList);
+      for (String measurement : measurementList) {
+        if (measurement != null) {
+          if (measurement.contains(TsFileConstant.PATH_SEPARATOR)
+              && !(measurement.startsWith(TsFileConstant.BACK_QUOTE_STRING)
+                  && measurement.endsWith(TsFileConstant.BACK_QUOTE_STRING))) {
+            throw new IllegalPathException(measurement);
+          } else {
+            path.append(TsFileConstant.PATH_SEPARATOR);
+            path.append(measurement);
+          }
+        }
+      }
+    }
+    try {
+      PathUtils.isLegalPath(path.toString());
+    } catch (IllegalPathException e) {
+      throw new MetadataException("find wrong node name according to syntax convention");
     }
   }
 
-  // check whether measurement is legal according to syntax convention
-  protected void isLegalMeasurements(List<String> measurements) throws TException {
+  /**
+   * check whether measurement is legal according to syntax convention measurement can only be a
+   * single node name
+   */
+  protected void isLegalSingleMeasurements(List<String> measurements) throws MetadataException {
     if (measurements == null) {
       return;
     }
+    StringBuilder path = new StringBuilder(IoTDBConstant.PATH_ROOT);
     for (String measurement : measurements) {
-      try {
-        if (measurement != null) {
-          PathUtils.isLegalPath(measurement);
+      if (measurement != null) {
+        if (measurement.contains(TsFileConstant.PATH_SEPARATOR)
+            && !(measurement.startsWith(TsFileConstant.BACK_QUOTE_STRING)
+                && measurement.endsWith(TsFileConstant.BACK_QUOTE_STRING))) {
+          throw new IllegalPathException(measurement);
+        } else {
+          path.append(TsFileConstant.PATH_SEPARATOR);
+          path.append(measurement);
         }
-      } catch (IllegalPathException e) {
-        throw new TException(e.getMessage());
+      }
+    }
+    try {
+      PathUtils.isLegalPath(path.toString());
+    } catch (IllegalPathException e) {
+      throw new MetadataException("find wrong node name according to syntax convention");
+    }
+  }
+
+  /**
+   * check whether measurement is legal according to syntax convention measurement could be like a.b
+   * (more than one node name), in template?
+   */
+  protected void isLegalMeasurementLists(List<List<String>> measurementLists)
+      throws IllegalPathException {
+    if (measurementLists == null) {
+      return;
+    }
+    for (List<String> measurementList : measurementLists) {
+      isLegalMeasurements(measurementList);
+    }
+  }
+
+  /**
+   * check whether measurement is legal according to syntax convention measurement could be like a.b
+   * (more than one node name), in template?
+   */
+  protected void isLegalMeasurements(List<String> measurements) throws IllegalPathException {
+    if (measurements == null) {
+      return;
+    }
+    for (String measurement : measurements) {
+      if (measurement != null) {
+        PathUtils.isLegalPath(measurement);
       }
     }
   }