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/06/23 03:50:55 UTC

[GitHub] [incubator-iotdb] mychaow opened a new pull request #1408: move getSeriesSchemas to MManager

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


   


----------------------------------------------------------------
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 #1408: move getSeriesSchemas to MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1800,4 +1807,207 @@ private void checkMTreeModified() {
       }
     }
   }
+
+  /**
+   * get schema for device.
+   * Attention!!!  Only support insertPlan and insertTabletsPlan
+   * @param deviceId
+   * @param measurementList
+   * @param plan
+   * @return
+   * @throws MetadataException
+   */
+  public MeasurementSchema[] getSeriesSchemasAndLock(String deviceId, String[] measurementList, PhysicalPlan plan) throws MetadataException {
+    MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
+
+    MNode deviceNode = null;
+    // 1. get device node
+    deviceNode = getDeviceNodeWithAutoCreateAndReadLock(deviceId);
+    // To reduce the String number in memory, set the deviceId from MManager to insertPlan
+    if (plan instanceof InsertPlan) {
+      ((InsertPlan) plan).setDeviceId(deviceNode.getFullPath());
+    } else if (plan instanceof InsertTabletPlan) {
+      ((InsertTabletPlan) plan).setDeviceId(deviceNode.getFullPath());
+    } else {
+      throw new MetadataException(String.format(
+        "Only support insert and insertTablets, deviceId[%s], plans [%s]", deviceId, plan.getOperatorType()));
+    }

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 #1408: move getSeriesSchemas to MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -915,227 +926,19 @@ public void insert(InsertPlan insertPlan) throws QueryProcessException {
       StorageEngine.getInstance().insert(insertPlan);
       if (insertPlan.getFailedMeasurements() != null) {
         throw new StorageEngineException(
-            "failed to insert points " + insertPlan.getFailedMeasurements());
+            "failed to insert measurements " + insertPlan.getFailedMeasurements());
       }
     } catch (StorageEngineException | MetadataException e) {
       throw new QueryProcessException(e);
-    }
-  }
-
-  protected MeasurementSchema[] getSeriesSchemas(InsertPlan insertPlan) throws MetadataException {
-    String[] measurementList = insertPlan.getMeasurements();
-    String deviceId = insertPlan.getDeviceId();
-    MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
-
-    MNode node = null;
-    try {
-      node = mManager.getDeviceNodeWithAutoCreateAndReadLock(deviceId);
-      // To reduce the String number in memory, set the deviceId from MManager to insertPlan
-      insertPlan.setDeviceId(node.getFullPath());
-    } catch (PathNotExistException e) {
-      // ignore
-    }
-    try {
-      for (int i = 0; i < measurementList.length; i++) {
-        try {
-          schemas[i] = getSeriesSchema(node, insertPlan, i);
-          if (schemas[i] != null) {
-            measurementList[i] = schemas[i].getMeasurementId();
-          }
-        } catch (MetadataException e) {
-          logger.warn("meet error when check {}.{}, message: {}", deviceId, measurementList[i],
-              e.getMessage());
-          if (enablePartialInsert) {
-            insertPlan.markMeasurementInsertionFailed(i);
-          } else {
-            throw e;
-          }
-        }
-      }
     } finally {
-      if (node != null) {
-        node.readUnlock();
-      }
-    }
-    return schemas;
-  }
-
-  /**
-   * @param loc index of measurement in insertPlan
-   */
-  private MeasurementSchema getSeriesSchema(MNode deviceNode, InsertPlan insertPlan, int loc)
-      throws MetadataException {
-    String measurement = insertPlan.getMeasurements()[loc];
-    String deviceId = insertPlan.getDeviceId();
-    Object value = insertPlan.getValues()[loc];
-    boolean isInferType = insertPlan.isInferType();
-
-    MeasurementSchema measurementSchema;
-    if (deviceNode != null && !deviceNode.hasChild(measurement)) {
-      // devices exists in MTree
-      if (!IoTDBDescriptor.getInstance().getConfig().isAutoCreateSchemaEnabled()) {
-        // but measurement not in MTree and cannot auto-create, try the cache
-        measurementSchema = MManager.getInstance().getSeriesSchema(deviceId, measurement);
-        if (measurementSchema == null) {
-          throw new PathNotExistException(deviceId + PATH_SEPARATOR + measurement);
-        }
-      } else {
-        // auto-create
-        TSDataType dataType = TypeInferenceUtils.getPredictedDataType(value, isInferType);
-        Path path = new Path(deviceId, measurement);
-        internalCreateTimeseries(path.toString(), dataType);
-
-        MeasurementMNode measurementNode = (MeasurementMNode) mManager
-            .getChild(deviceNode, measurement);
-        measurementSchema = measurementNode.getSchema();
-        if (!isInferType) {
-          checkType(insertPlan, loc, measurementNode.getSchema().getType());
-        }
-      }
-    } else if (deviceNode != null) {
-      // device and measurement exists in MTree
-      MeasurementMNode measurementNode = (MeasurementMNode) MManager.getInstance()
-          .getChild(deviceNode, measurement);
-      measurementSchema = measurementNode.getSchema();
-    } else {
-      // device in not in MTree, try the cache
-      measurementSchema = mManager.getSeriesSchema(deviceId, measurement);
-    }
-    return measurementSchema;
-  }
-
-  private void checkType(InsertPlan plan, int loc, TSDataType type) {
-    plan.getTypes()[loc] = type;
-    try {
-      switch (type) {
-        case INT32:
-          if (!(plan.getValues()[loc] instanceof Integer)) {
-            plan.getValues()[loc] =
-                Integer.parseInt(((Binary) plan.getValues()[loc]).getStringValue());
-          }
-          break;
-        case INT64:
-          if (!(plan.getValues()[loc] instanceof Long)) {
-            plan.getValues()[loc] =
-                Long.parseLong(((Binary) plan.getValues()[loc]).getStringValue());
-          }
-          break;
-        case DOUBLE:
-          if (!(plan.getValues()[loc] instanceof Double)) {
-            plan.getValues()[loc] =
-                Double.parseDouble(((Binary) plan.getValues()[loc]).getStringValue());
-          }
-          break;
-        case FLOAT:
-          if (!(plan.getValues()[loc] instanceof Float)) {
-            plan.getValues()[loc] =
-                Float.parseFloat(((Binary) plan.getValues()[loc]).getStringValue());
-          }
-          break;
-        case BOOLEAN:
-          if (!(plan.getValues()[loc] instanceof Boolean)) {
-            plan.getValues()[loc] =
-                Boolean.parseBoolean(((Binary) plan.getValues()[loc]).getStringValue());
-          }
-          break;
-        case TEXT:
-          // need to do nothing
-          break;
-      }
-    } catch (ClassCastException e) {
-      logger.error("inconsistent type between client and server");
-    }
-  }
-
-  /**
-   * create timeseries with ignore PathAlreadyExistException
-   */
-  private void internalCreateTimeseries(String path, TSDataType dataType) throws MetadataException {
-    try {
-      mManager.createTimeseries(
-          path,
-          dataType,
-          getDefaultEncoding(dataType),
-          TSFileDescriptor.getInstance().getConfig().getCompressor(),
-          Collections.emptyMap());
-    } catch (PathAlreadyExistException e) {
-      if (logger.isDebugEnabled()) {
-        logger.debug("Ignore PathAlreadyExistException when Concurrent inserting"
-            + " a non-exist time series {}", path);
-      }
-    }
-  }
-
-  /**
-   * Get default encoding by dataType
-   */
-  private TSEncoding getDefaultEncoding(TSDataType dataType) {
-    IoTDBConfig conf = IoTDBDescriptor.getInstance().getConfig();
-    switch (dataType) {
-      case BOOLEAN:
-        return conf.getDefaultBooleanEncoding();
-      case INT32:
-        return conf.getDefaultInt32Encoding();
-      case INT64:
-        return conf.getDefaultInt64Encoding();
-      case FLOAT:
-        return conf.getDefaultFloatEncoding();
-      case DOUBLE:
-        return conf.getDefaultDoubleEncoding();
-      case TEXT:
-        return conf.getDefaultTextEncoding();
-      default:
-        throw new UnSupportedDataTypeException(
-            String.format("Data type %s is not supported.", dataType.toString()));
+      mManager.unlockInsert(insertPlan.getDeviceId());

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] qiaojialin commented on a change in pull request #1408: move getSeriesSchemas to MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1846,4 +1836,219 @@ public void createMTreeSnapshot() {
       lock.readLock().unlock();
     }
   }
+
+  /**
+   * get schema for device.
+   * Attention!!!  Only support insertPlan and insertTabletsPlan
+   * @param deviceId
+   * @param measurementList
+   * @param plan
+   * @return
+   * @throws MetadataException
+   */
+  public MeasurementSchema[] getSeriesSchemas(String deviceId, String[] measurementList, PhysicalPlan plan) throws MetadataException {
+    MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
+
+    MNode deviceNode = null;
+    // 1. get device node
+    deviceNode = getDeviceNode(deviceId);
+
+    // 2. get schema of each measurement
+    for (int i = 0; i < measurementList.length; i++) {
+      try {
+        // if do not has measurement
+        if (!deviceNode.hasChild(measurementList[i])) {
+          // could not create it
+          if (!config.isAutoCreateSchemaEnabled()) {
+            throw new MetadataException(String.format(
+              "Current deviceId[%s] does not contain measurement:%s", deviceId, measurementList[i]));
+          }
+
+          // create it
+          Path path = new Path(deviceId, measurementList[i]);
+          TSDataType dataType = getTypeInLoc(plan, i);
+
+          createTimeseries(
+            path.getFullPath(),
+            dataType,
+            getDefaultEncoding(dataType),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+        }
+
+        MeasurementMNode measurementNode = (MeasurementMNode) getChild(deviceNode, measurementList[i]);
+
+        // check type is match
+        TSDataType insertDataType = null;
+        if (plan instanceof InsertPlan) {
+          if (!((InsertPlan)plan).isNeedInferType()) {
+            // only when InsertPlan's values is object[], we should check type
+            insertDataType = getTypeInLoc(plan, i);
+          } else {
+            insertDataType = measurementNode.getSchema().getType();
+          }
+        } else if (plan instanceof InsertTabletPlan) {
+          insertDataType = getTypeInLoc(plan, i);
+        }
+
+        if (measurementNode.getSchema().getType() != insertDataType) {
+          logger.warn("DataType mismatch, Insert measurement {} type {}, metadata tree type {}",
+            measurementList[i], insertDataType, measurementNode.getSchema().getType());
+          if (!config.isEnablePartialInsert()) {
+            throw new MetadataException(String.format(
+              "DataType mismatch, Insert measurement %s type %s, metadata tree type %s",
+              measurementList[i], insertDataType, measurementNode.getSchema().getType()));
+          } else {
+            // mark failed measurement
+            if (plan instanceof InsertTabletPlan) {
+              ((InsertTabletPlan) plan).markMeasurementInsertionFailed(i);
+            } else if (plan instanceof InsertPlan) {
+              ((InsertPlan) plan).markMeasurementInsertionFailed(i);
+            }
+            continue;
+          }
+        }
+
+        // maybe need to convert value type to the true type
+        if ((plan instanceof InsertPlan)) {
+          changeValueToRealType((InsertPlan) plan, i, measurementNode.getSchema().getType());
+        }
+
+        schemas[i] = measurementNode.getSchema();
+        if (schemas[i] != null) {
+          measurementList[i] = schemas[i].getMeasurementId();
+        }
+      } catch (MetadataException e) {
+        logger.warn("meet error when check {}.{}, message: {}", deviceId, measurementList[i],
+          e.getMessage());
+        if (config.isEnablePartialInsert()) {
+          // mark failed measurement
+          if (plan instanceof InsertPlan) {
+            ((InsertPlan) plan).markMeasurementInsertionFailed(i);
+          } else if (plan instanceof InsertTabletPlan) {
+            ((InsertTabletPlan) plan).markMeasurementInsertionFailed(i);
+          }
+        } else {
+          throw e;
+        }
+      }
+    }
+    return schemas;
+  }
+
+  private void changeValueToRealType(InsertPlan plan, int loc, TSDataType type) throws MetadataException {
+    plan.getTypes()[loc] = type;
+    try {
+      switch (type) {
+        case INT32:
+          if (!(plan.getValues()[loc] instanceof Integer)) {
+            plan.getValues()[loc] =
+              Integer.parseInt(String.valueOf(plan.getValues()[loc]));
+          }
+          break;
+        case INT64:
+          if (!(plan.getValues()[loc] instanceof Long)) {
+            plan.getValues()[loc] =
+              Long.parseLong(String.valueOf(plan.getValues()[loc]));
+          }
+          break;
+        case DOUBLE:
+          if (!(plan.getValues()[loc] instanceof Double)) {
+            plan.getValues()[loc] =
+              Double.parseDouble(String.valueOf(plan.getValues()[loc]));
+          }
+          break;
+        case FLOAT:
+          if (!(plan.getValues()[loc] instanceof Float)) {
+            plan.getValues()[loc] =
+              Float.parseFloat(String.valueOf(plan.getValues()[loc]));
+          }
+          break;
+        case BOOLEAN:
+          if (!(plan.getValues()[loc] instanceof Boolean)) {
+            plan.getValues()[loc] =
+              Boolean.parseBoolean(String.valueOf(plan.getValues()[loc]));
+          }
+          break;
+        case TEXT:
+          // need to do nothing

Review comment:
       The String needs to be cast to Binary

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1846,4 +1836,219 @@ public void createMTreeSnapshot() {
       lock.readLock().unlock();
     }
   }
+
+  /**
+   * get schema for device.
+   * Attention!!!  Only support insertPlan and insertTabletsPlan
+   * @param deviceId
+   * @param measurementList
+   * @param plan
+   * @return
+   * @throws MetadataException
+   */
+  public MeasurementSchema[] getSeriesSchemas(String deviceId, String[] measurementList, PhysicalPlan plan) throws MetadataException {
+    MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
+
+    MNode deviceNode = null;
+    // 1. get device node
+    deviceNode = getDeviceNode(deviceId);
+
+    // 2. get schema of each measurement
+    for (int i = 0; i < measurementList.length; i++) {
+      try {
+        // if do not has measurement
+        if (!deviceNode.hasChild(measurementList[i])) {
+          // could not create it
+          if (!config.isAutoCreateSchemaEnabled()) {
+            throw new MetadataException(String.format(
+              "Current deviceId[%s] does not contain measurement:%s", deviceId, measurementList[i]));
+          }
+
+          // create it
+          Path path = new Path(deviceId, measurementList[i]);
+          TSDataType dataType = getTypeInLoc(plan, i);
+
+          createTimeseries(
+            path.getFullPath(),
+            dataType,
+            getDefaultEncoding(dataType),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+        }
+
+        MeasurementMNode measurementNode = (MeasurementMNode) getChild(deviceNode, measurementList[i]);
+
+        // check type is match
+        TSDataType insertDataType = null;
+        if (plan instanceof InsertPlan) {
+          if (!((InsertPlan)plan).isNeedInferType()) {
+            // only when InsertPlan's values is object[], we should check type
+            insertDataType = getTypeInLoc(plan, i);
+          } else {
+            insertDataType = measurementNode.getSchema().getType();
+          }
+        } else if (plan instanceof InsertTabletPlan) {
+          insertDataType = getTypeInLoc(plan, i);
+        }
+
+        if (measurementNode.getSchema().getType() != insertDataType) {
+          logger.warn("DataType mismatch, Insert measurement {} type {}, metadata tree type {}",
+            measurementList[i], insertDataType, measurementNode.getSchema().getType());
+          if (!config.isEnablePartialInsert()) {
+            throw new MetadataException(String.format(
+              "DataType mismatch, Insert measurement %s type %s, metadata tree type %s",
+              measurementList[i], insertDataType, measurementNode.getSchema().getType()));
+          } else {
+            // mark failed measurement
+            if (plan instanceof InsertTabletPlan) {
+              ((InsertTabletPlan) plan).markMeasurementInsertionFailed(i);
+            } else if (plan instanceof InsertPlan) {
+              ((InsertPlan) plan).markMeasurementInsertionFailed(i);
+            }
+            continue;
+          }
+        }
+
+        // maybe need to convert value type to the true type
+        if ((plan instanceof InsertPlan)) {
+          changeValueToRealType((InsertPlan) plan, i, measurementNode.getSchema().getType());
+        }
+
+        schemas[i] = measurementNode.getSchema();
+        if (schemas[i] != null) {
+          measurementList[i] = schemas[i].getMeasurementId();
+        }
+      } catch (MetadataException e) {
+        logger.warn("meet error when check {}.{}, message: {}", deviceId, measurementList[i],
+          e.getMessage());
+        if (config.isEnablePartialInsert()) {
+          // mark failed measurement
+          if (plan instanceof InsertPlan) {
+            ((InsertPlan) plan).markMeasurementInsertionFailed(i);
+          } else if (plan instanceof InsertTabletPlan) {
+            ((InsertTabletPlan) plan).markMeasurementInsertionFailed(i);
+          }
+        } else {
+          throw e;
+        }
+      }
+    }
+    return schemas;
+  }
+
+  private void changeValueToRealType(InsertPlan plan, int loc, TSDataType type) throws MetadataException {

Review comment:
       ```suggestion
     private void changeStringValueToRealType(InsertPlan plan, int loc, TSDataType type) throws MetadataException {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1846,4 +1836,219 @@ public void createMTreeSnapshot() {
       lock.readLock().unlock();
     }
   }
+
+  /**
+   * get schema for device.
+   * Attention!!!  Only support insertPlan and insertTabletsPlan
+   * @param deviceId
+   * @param measurementList
+   * @param plan
+   * @return
+   * @throws MetadataException
+   */
+  public MeasurementSchema[] getSeriesSchemas(String deviceId, String[] measurementList, PhysicalPlan plan) throws MetadataException {
+    MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
+
+    MNode deviceNode = null;
+    // 1. get device node
+    deviceNode = getDeviceNode(deviceId);
+
+    // 2. get schema of each measurement
+    for (int i = 0; i < measurementList.length; i++) {
+      try {
+        // if do not has measurement
+        if (!deviceNode.hasChild(measurementList[i])) {
+          // could not create it
+          if (!config.isAutoCreateSchemaEnabled()) {
+            throw new MetadataException(String.format(
+              "Current deviceId[%s] does not contain measurement:%s", deviceId, measurementList[i]));
+          }
+
+          // create it
+          Path path = new Path(deviceId, measurementList[i]);
+          TSDataType dataType = getTypeInLoc(plan, i);
+
+          createTimeseries(
+            path.getFullPath(),
+            dataType,
+            getDefaultEncoding(dataType),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+        }
+
+        MeasurementMNode measurementNode = (MeasurementMNode) getChild(deviceNode, measurementList[i]);
+
+        // check type is match
+        TSDataType insertDataType = null;
+        if (plan instanceof InsertPlan) {
+          if (!((InsertPlan)plan).isNeedInferType()) {
+            // only when InsertPlan's values is object[], we should check type
+            insertDataType = getTypeInLoc(plan, i);
+          } else {
+            insertDataType = measurementNode.getSchema().getType();
+          }
+        } else if (plan instanceof InsertTabletPlan) {
+          insertDataType = getTypeInLoc(plan, i);
+        }
+
+        if (measurementNode.getSchema().getType() != insertDataType) {
+          logger.warn("DataType mismatch, Insert measurement {} type {}, metadata tree type {}",
+            measurementList[i], insertDataType, measurementNode.getSchema().getType());
+          if (!config.isEnablePartialInsert()) {
+            throw new MetadataException(String.format(
+              "DataType mismatch, Insert measurement %s type %s, metadata tree type %s",
+              measurementList[i], insertDataType, measurementNode.getSchema().getType()));
+          } else {
+            // mark failed measurement
+            if (plan instanceof InsertTabletPlan) {
+              ((InsertTabletPlan) plan).markMeasurementInsertionFailed(i);
+            } else if (plan instanceof InsertPlan) {
+              ((InsertPlan) plan).markMeasurementInsertionFailed(i);
+            }
+            continue;
+          }
+        }
+
+        // maybe need to convert value type to the true type
+        if ((plan instanceof InsertPlan)) {
+          changeValueToRealType((InsertPlan) plan, i, measurementNode.getSchema().getType());
+        }
+
+        schemas[i] = measurementNode.getSchema();
+        if (schemas[i] != null) {
+          measurementList[i] = schemas[i].getMeasurementId();
+        }
+      } catch (MetadataException e) {
+        logger.warn("meet error when check {}.{}, message: {}", deviceId, measurementList[i],
+          e.getMessage());
+        if (config.isEnablePartialInsert()) {
+          // mark failed measurement
+          if (plan instanceof InsertPlan) {
+            ((InsertPlan) plan).markMeasurementInsertionFailed(i);
+          } else if (plan instanceof InsertTabletPlan) {
+            ((InsertTabletPlan) plan).markMeasurementInsertionFailed(i);
+          }
+        } else {
+          throw e;
+        }
+      }
+    }
+    return schemas;
+  }
+
+  private void changeValueToRealType(InsertPlan plan, int loc, TSDataType type) throws MetadataException {
+    plan.getTypes()[loc] = type;
+    try {
+      switch (type) {
+        case INT32:
+          if (!(plan.getValues()[loc] instanceof Integer)) {

Review comment:
       Do we need this check? I think only String value need to be inferred.




----------------------------------------------------------------
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] qiaojialin commented on a change in pull request #1408: move getSeriesSchemas to MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1800,4 +1807,207 @@ private void checkMTreeModified() {
       }
     }
   }
+
+  /**
+   * get schema for device.
+   * Attention!!!  Only support insertPlan and insertTabletsPlan
+   * @param deviceId
+   * @param measurementList
+   * @param plan
+   * @return
+   * @throws MetadataException
+   */
+  public MeasurementSchema[] getSeriesSchemasAndLock(String deviceId, String[] measurementList, PhysicalPlan plan) throws MetadataException {
+    MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
+
+    MNode deviceNode = null;
+    // 1. get device node
+    deviceNode = getDeviceNodeWithAutoCreateAndReadLock(deviceId);
+    // To reduce the String number in memory, set the deviceId from MManager to insertPlan
+    if (plan instanceof InsertPlan) {
+      ((InsertPlan) plan).setDeviceId(deviceNode.getFullPath());
+    } else if (plan instanceof InsertTabletPlan) {
+      ((InsertTabletPlan) plan).setDeviceId(deviceNode.getFullPath());
+    } else {
+      throw new MetadataException(String.format(
+        "Only support insert and insertTablets, deviceId[%s], plans [%s]", deviceId, plan.getOperatorType()));
+    }

Review comment:
       No need to set this, the insertplan will be gc soon

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1800,4 +1807,207 @@ private void checkMTreeModified() {
       }
     }
   }
+
+  /**
+   * get schema for device.
+   * Attention!!!  Only support insertPlan and insertTabletsPlan
+   * @param deviceId
+   * @param measurementList
+   * @param plan
+   * @return
+   * @throws MetadataException
+   */
+  public MeasurementSchema[] getSeriesSchemasAndLock(String deviceId, String[] measurementList, PhysicalPlan plan) throws MetadataException {
+    MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
+
+    MNode deviceNode = null;
+    // 1. get device node
+    deviceNode = getDeviceNodeWithAutoCreateAndReadLock(deviceId);
+    // To reduce the String number in memory, set the deviceId from MManager to insertPlan
+    if (plan instanceof InsertPlan) {
+      ((InsertPlan) plan).setDeviceId(deviceNode.getFullPath());
+    } else if (plan instanceof InsertTabletPlan) {
+      ((InsertTabletPlan) plan).setDeviceId(deviceNode.getFullPath());
+    } else {
+      throw new MetadataException(String.format(
+        "Only support insert and insertTablets, deviceId[%s], plans [%s]", deviceId, plan.getOperatorType()));
+    }
+
+    // 2. get schema of each measurement
+    for (int i = 0; i < measurementList.length; i++) {
+      try {
+        // if do not has measurement
+        boolean isCreated = false;
+        if (!deviceNode.hasChild(measurementList[i])) {
+          // could not create it
+          if (!config.isAutoCreateSchemaEnabled()) {
+            throw new MetadataException(String.format(
+              "Current deviceId[%s] does not contain measurement:%s", deviceId, measurementList[i]));
+          }
+
+          // create it
+          Path path = new Path(deviceId, measurementList[i]);
+          TSDataType dataType = getTypeInLoc(plan, i);
+
+          createTimeseries(
+            path.getFullPath(),
+            dataType,
+            getDefaultEncoding(dataType),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+          isCreated = true;
+        }

Review comment:
       what if the measurement already exists? 
   
   add 
   
   else {
    isCreated = true
   }
   
   besides,combine these to one line

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1800,4 +1807,207 @@ private void checkMTreeModified() {
       }
     }
   }
+
+  /**
+   * get schema for device.
+   * Attention!!!  Only support insertPlan and insertTabletsPlan
+   * @param deviceId
+   * @param measurementList
+   * @param plan
+   * @return
+   * @throws MetadataException
+   */
+  public MeasurementSchema[] getSeriesSchemasAndLock(String deviceId, String[] measurementList, PhysicalPlan plan) throws MetadataException {
+    MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
+
+    MNode deviceNode = null;
+    // 1. get device node
+    deviceNode = getDeviceNodeWithAutoCreateAndReadLock(deviceId);
+    // To reduce the String number in memory, set the deviceId from MManager to insertPlan
+    if (plan instanceof InsertPlan) {
+      ((InsertPlan) plan).setDeviceId(deviceNode.getFullPath());
+    } else if (plan instanceof InsertTabletPlan) {
+      ((InsertTabletPlan) plan).setDeviceId(deviceNode.getFullPath());
+    } else {
+      throw new MetadataException(String.format(
+        "Only support insert and insertTablets, deviceId[%s], plans [%s]", deviceId, plan.getOperatorType()));
+    }
+
+    // 2. get schema of each measurement
+    for (int i = 0; i < measurementList.length; i++) {
+      try {
+        // if do not has measurement
+        boolean isCreated = false;
+        if (!deviceNode.hasChild(measurementList[i])) {
+          // could not create it
+          if (!config.isAutoCreateSchemaEnabled()) {
+            throw new MetadataException(String.format(
+              "Current deviceId[%s] does not contain measurement:%s", deviceId, measurementList[i]));
+          }
+
+          // create it
+          Path path = new Path(deviceId, measurementList[i]);
+          TSDataType dataType = getTypeInLoc(plan, i);
+
+          createTimeseries(
+            path.getFullPath(),
+            dataType,
+            getDefaultEncoding(dataType),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+          isCreated = true;
+        }
+
+        MeasurementMNode measurementNode = (MeasurementMNode) getChild(deviceNode, measurementList[i]);
+
+        // check type is match
+        if (plan instanceof InsertTabletPlan) {
+          TSDataType trueType = getTypeInLoc(plan, i);
+          if (measurementNode.getSchema().getType() != trueType) {
+            logger.warn("Datatype mismatch, Insert measurement {} type {}, metadata tree type {}",
+              measurementList[i], trueType, measurementNode.getSchema().getType());
+            if (!config.isEnablePartialInsert()) {
+              throw new MetadataException(String.format(
+                "Datatype mismatch, Insert measurement %s type %s, metadata tree type %s",
+                measurementList[i], measurementNode.getSchema().getType(),
+                trueType));
+            } else {
+              // mark failed measurement
+              ((InsertTabletPlan) plan).markMeasurementInsertionFailed(i);
+              continue;
+            }
+          }
+        }
+
+        if ((plan instanceof InsertPlan) && isCreated) {

Review comment:
       Is the isCreated needed? If a measurement does not exist, we already throw an exception

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1800,4 +1807,207 @@ private void checkMTreeModified() {
       }
     }
   }
+
+  /**
+   * get schema for device.
+   * Attention!!!  Only support insertPlan and insertTabletsPlan
+   * @param deviceId
+   * @param measurementList
+   * @param plan
+   * @return
+   * @throws MetadataException
+   */
+  public MeasurementSchema[] getSeriesSchemasAndLock(String deviceId, String[] measurementList, PhysicalPlan plan) throws MetadataException {
+    MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
+
+    MNode deviceNode = null;
+    // 1. get device node
+    deviceNode = getDeviceNodeWithAutoCreateAndReadLock(deviceId);
+    // To reduce the String number in memory, set the deviceId from MManager to insertPlan
+    if (plan instanceof InsertPlan) {
+      ((InsertPlan) plan).setDeviceId(deviceNode.getFullPath());
+    } else if (plan instanceof InsertTabletPlan) {
+      ((InsertTabletPlan) plan).setDeviceId(deviceNode.getFullPath());
+    } else {
+      throw new MetadataException(String.format(
+        "Only support insert and insertTablets, deviceId[%s], plans [%s]", deviceId, plan.getOperatorType()));
+    }
+
+    // 2. get schema of each measurement
+    for (int i = 0; i < measurementList.length; i++) {
+      try {
+        // if do not has measurement
+        boolean isCreated = false;
+        if (!deviceNode.hasChild(measurementList[i])) {
+          // could not create it
+          if (!config.isAutoCreateSchemaEnabled()) {
+            throw new MetadataException(String.format(
+              "Current deviceId[%s] does not contain measurement:%s", deviceId, measurementList[i]));
+          }
+
+          // create it
+          Path path = new Path(deviceId, measurementList[i]);
+          TSDataType dataType = getTypeInLoc(plan, i);
+
+          createTimeseries(
+            path.getFullPath(),
+            dataType,
+            getDefaultEncoding(dataType),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+          isCreated = true;
+        }
+
+        MeasurementMNode measurementNode = (MeasurementMNode) getChild(deviceNode, measurementList[i]);
+
+        // check type is match
+        if (plan instanceof InsertTabletPlan) {
+          TSDataType trueType = getTypeInLoc(plan, i);

Review comment:
       insertDataType

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -907,6 +907,17 @@ public void delete(Path path, long timestamp) throws QueryProcessException {
     }
   }
 
+  protected MeasurementSchema[] getSeriesSchemas(InsertPlan insertPlan)
+    throws MetadataException {
+    return mManager.getSeriesSchemasAndLock(insertPlan.getDeviceId(), insertPlan.getMeasurements(), insertPlan);
+  }
+
+  protected MeasurementSchema[] getSeriesSchemas(InsertTabletPlan insertTabletPlan)
+    throws MetadataException {
+    return mManager.getSeriesSchemasAndLock(insertTabletPlan.getDeviceId(),
+      insertTabletPlan.getMeasurements(), insertTabletPlan);
+  }
+

Review comment:
       remove these two methods and use mManager.getSeriesSchemasAndLock direclty




----------------------------------------------------------------
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 #1408: move getSeriesSchemas to MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -907,6 +907,17 @@ public void delete(Path path, long timestamp) throws QueryProcessException {
     }
   }
 
+  protected MeasurementSchema[] getSeriesSchemas(InsertPlan insertPlan)
+    throws MetadataException {
+    return mManager.getSeriesSchemasAndLock(insertPlan.getDeviceId(), insertPlan.getMeasurements(), insertPlan);
+  }
+
+  protected MeasurementSchema[] getSeriesSchemas(InsertTabletPlan insertTabletPlan)
+    throws MetadataException {
+    return mManager.getSeriesSchemasAndLock(insertTabletPlan.getDeviceId(),
+      insertTabletPlan.getMeasurements(), insertTabletPlan);
+  }
+

Review comment:
       These functions is just for cluster version, cluster version will override this two function.




----------------------------------------------------------------
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] qiaojialin commented on a change in pull request #1408: move getSeriesSchemas to MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java
##########
@@ -57,9 +58,9 @@
   private TSDataType[] types;
   private MeasurementSchema[] schemas;
 
-  // if inferType is false, use the type of values directly
-  // if inferType is true, values is String[], and infer types from them
-  private boolean inferType = false;
+  // if values is String[], isNeedInferType must be true, so we could infer types from them

Review comment:
       ```suggestion
     // if isNeedInferType is true, the values must be String[], so we could infer types from them
   ```




----------------------------------------------------------------
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 #1408: move getSeriesSchemas to MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1800,4 +1807,207 @@ private void checkMTreeModified() {
       }
     }
   }
+
+  /**
+   * get schema for device.
+   * Attention!!!  Only support insertPlan and insertTabletsPlan
+   * @param deviceId
+   * @param measurementList
+   * @param plan
+   * @return
+   * @throws MetadataException
+   */
+  public MeasurementSchema[] getSeriesSchemasAndLock(String deviceId, String[] measurementList, PhysicalPlan plan) throws MetadataException {
+    MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
+
+    MNode deviceNode = null;
+    // 1. get device node
+    deviceNode = getDeviceNodeWithAutoCreateAndReadLock(deviceId);
+    // To reduce the String number in memory, set the deviceId from MManager to insertPlan
+    if (plan instanceof InsertPlan) {
+      ((InsertPlan) plan).setDeviceId(deviceNode.getFullPath());
+    } else if (plan instanceof InsertTabletPlan) {
+      ((InsertTabletPlan) plan).setDeviceId(deviceNode.getFullPath());
+    } else {
+      throw new MetadataException(String.format(
+        "Only support insert and insertTablets, deviceId[%s], plans [%s]", deviceId, plan.getOperatorType()));
+    }
+
+    // 2. get schema of each measurement
+    for (int i = 0; i < measurementList.length; i++) {
+      try {
+        // if do not has measurement
+        boolean isCreated = false;
+        if (!deviceNode.hasChild(measurementList[i])) {
+          // could not create it
+          if (!config.isAutoCreateSchemaEnabled()) {
+            throw new MetadataException(String.format(
+              "Current deviceId[%s] does not contain measurement:%s", deviceId, measurementList[i]));
+          }
+
+          // create it
+          Path path = new Path(deviceId, measurementList[i]);
+          TSDataType dataType = getTypeInLoc(plan, i);
+
+          createTimeseries(
+            path.getFullPath(),
+            dataType,
+            getDefaultEncoding(dataType),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+          isCreated = true;
+        }
+
+        MeasurementMNode measurementNode = (MeasurementMNode) getChild(deviceNode, measurementList[i]);
+
+        // check type is match
+        if (plan instanceof InsertTabletPlan) {
+          TSDataType trueType = getTypeInLoc(plan, i);
+          if (measurementNode.getSchema().getType() != trueType) {
+            logger.warn("Datatype mismatch, Insert measurement {} type {}, metadata tree type {}",
+              measurementList[i], trueType, measurementNode.getSchema().getType());
+            if (!config.isEnablePartialInsert()) {
+              throw new MetadataException(String.format(
+                "Datatype mismatch, Insert measurement %s type %s, metadata tree type %s",
+                measurementList[i], measurementNode.getSchema().getType(),
+                trueType));
+            } else {
+              // mark failed measurement
+              ((InsertTabletPlan) plan).markMeasurementInsertionFailed(i);
+              continue;
+            }
+          }
+        }
+
+        if ((plan instanceof InsertPlan) && isCreated) {

Review comment:
       if a measurement exist, but the value is string (like jdbc insert), we need to convert the value to the true type




----------------------------------------------------------------
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] qiaojialin commented on a change in pull request #1408: move getSeriesSchemas to MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1846,4 +1836,219 @@ public void createMTreeSnapshot() {
       lock.readLock().unlock();
     }
   }
+
+  /**
+   * get schema for device.
+   * Attention!!!  Only support insertPlan and insertTabletsPlan
+   * @param deviceId
+   * @param measurementList
+   * @param plan
+   * @return
+   * @throws MetadataException
+   */
+  public MeasurementSchema[] getSeriesSchemas(String deviceId, String[] measurementList, PhysicalPlan plan) throws MetadataException {
+    MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
+
+    MNode deviceNode = null;
+    // 1. get device node
+    deviceNode = getDeviceNode(deviceId);
+
+    // 2. get schema of each measurement
+    for (int i = 0; i < measurementList.length; i++) {
+      try {
+        // if do not has measurement
+        if (!deviceNode.hasChild(measurementList[i])) {
+          // could not create it
+          if (!config.isAutoCreateSchemaEnabled()) {
+            throw new MetadataException(String.format(
+              "Current deviceId[%s] does not contain measurement:%s", deviceId, measurementList[i]));
+          }
+
+          // create it
+          Path path = new Path(deviceId, measurementList[i]);
+          TSDataType dataType = getTypeInLoc(plan, i);
+
+          createTimeseries(
+            path.getFullPath(),
+            dataType,
+            getDefaultEncoding(dataType),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+        }
+
+        MeasurementMNode measurementNode = (MeasurementMNode) getChild(deviceNode, measurementList[i]);
+
+        // check type is match
+        TSDataType insertDataType = null;
+        if (plan instanceof InsertPlan) {
+          if (!((InsertPlan)plan).isNeedInferType()) {
+            // only when InsertPlan's values is object[], we should check type
+            insertDataType = getTypeInLoc(plan, i);
+          } else {
+            insertDataType = measurementNode.getSchema().getType();
+          }
+        } else if (plan instanceof InsertTabletPlan) {
+          insertDataType = getTypeInLoc(plan, i);
+        }
+
+        if (measurementNode.getSchema().getType() != insertDataType) {
+          logger.warn("DataType mismatch, Insert measurement {} type {}, metadata tree type {}",
+            measurementList[i], insertDataType, measurementNode.getSchema().getType());
+          if (!config.isEnablePartialInsert()) {
+            throw new MetadataException(String.format(
+              "DataType mismatch, Insert measurement %s type %s, metadata tree type %s",
+              measurementList[i], insertDataType, measurementNode.getSchema().getType()));
+          } else {
+            // mark failed measurement
+            if (plan instanceof InsertTabletPlan) {
+              ((InsertTabletPlan) plan).markMeasurementInsertionFailed(i);
+            } else if (plan instanceof InsertPlan) {
+              ((InsertPlan) plan).markMeasurementInsertionFailed(i);
+            }
+            continue;
+          }
+        }
+
+        // maybe need to convert value type to the true type
+        if ((plan instanceof InsertPlan)) {

Review comment:
       ```suggestion
           if ((plan instanceof InsertPlan && ((InsertPlan) plan).isNeedInferType()) {
   ```




----------------------------------------------------------------
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 #1408: move getSeriesSchemas to MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
##########
@@ -915,227 +926,19 @@ public void insert(InsertPlan insertPlan) throws QueryProcessException {
       StorageEngine.getInstance().insert(insertPlan);
       if (insertPlan.getFailedMeasurements() != null) {
         throw new StorageEngineException(
-            "failed to insert points " + insertPlan.getFailedMeasurements());
+            "failed to insert measurements " + insertPlan.getFailedMeasurements());
       }
     } catch (StorageEngineException | MetadataException e) {
       throw new QueryProcessException(e);
-    }
-  }
-
-  protected MeasurementSchema[] getSeriesSchemas(InsertPlan insertPlan) throws MetadataException {
-    String[] measurementList = insertPlan.getMeasurements();
-    String deviceId = insertPlan.getDeviceId();
-    MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
-
-    MNode node = null;
-    try {
-      node = mManager.getDeviceNodeWithAutoCreateAndReadLock(deviceId);
-      // To reduce the String number in memory, set the deviceId from MManager to insertPlan
-      insertPlan.setDeviceId(node.getFullPath());
-    } catch (PathNotExistException e) {
-      // ignore
-    }
-    try {
-      for (int i = 0; i < measurementList.length; i++) {
-        try {
-          schemas[i] = getSeriesSchema(node, insertPlan, i);
-          if (schemas[i] != null) {
-            measurementList[i] = schemas[i].getMeasurementId();
-          }
-        } catch (MetadataException e) {
-          logger.warn("meet error when check {}.{}, message: {}", deviceId, measurementList[i],
-              e.getMessage());
-          if (enablePartialInsert) {
-            insertPlan.markMeasurementInsertionFailed(i);
-          } else {
-            throw e;
-          }
-        }
-      }
     } finally {
-      if (node != null) {
-        node.readUnlock();
-      }
-    }
-    return schemas;
-  }
-
-  /**
-   * @param loc index of measurement in insertPlan
-   */
-  private MeasurementSchema getSeriesSchema(MNode deviceNode, InsertPlan insertPlan, int loc)
-      throws MetadataException {
-    String measurement = insertPlan.getMeasurements()[loc];
-    String deviceId = insertPlan.getDeviceId();
-    Object value = insertPlan.getValues()[loc];
-    boolean isInferType = insertPlan.isInferType();
-
-    MeasurementSchema measurementSchema;
-    if (deviceNode != null && !deviceNode.hasChild(measurement)) {
-      // devices exists in MTree
-      if (!IoTDBDescriptor.getInstance().getConfig().isAutoCreateSchemaEnabled()) {
-        // but measurement not in MTree and cannot auto-create, try the cache
-        measurementSchema = MManager.getInstance().getSeriesSchema(deviceId, measurement);
-        if (measurementSchema == null) {
-          throw new PathNotExistException(deviceId + PATH_SEPARATOR + measurement);
-        }
-      } else {
-        // auto-create
-        TSDataType dataType = TypeInferenceUtils.getPredictedDataType(value, isInferType);
-        Path path = new Path(deviceId, measurement);
-        internalCreateTimeseries(path.toString(), dataType);
-
-        MeasurementMNode measurementNode = (MeasurementMNode) mManager
-            .getChild(deviceNode, measurement);
-        measurementSchema = measurementNode.getSchema();
-        if (!isInferType) {
-          checkType(insertPlan, loc, measurementNode.getSchema().getType());
-        }
-      }
-    } else if (deviceNode != null) {
-      // device and measurement exists in MTree
-      MeasurementMNode measurementNode = (MeasurementMNode) MManager.getInstance()
-          .getChild(deviceNode, measurement);
-      measurementSchema = measurementNode.getSchema();
-    } else {
-      // device in not in MTree, try the cache
-      measurementSchema = mManager.getSeriesSchema(deviceId, measurement);
-    }
-    return measurementSchema;
-  }
-
-  private void checkType(InsertPlan plan, int loc, TSDataType type) {
-    plan.getTypes()[loc] = type;
-    try {
-      switch (type) {
-        case INT32:
-          if (!(plan.getValues()[loc] instanceof Integer)) {
-            plan.getValues()[loc] =
-                Integer.parseInt(((Binary) plan.getValues()[loc]).getStringValue());
-          }
-          break;
-        case INT64:
-          if (!(plan.getValues()[loc] instanceof Long)) {
-            plan.getValues()[loc] =
-                Long.parseLong(((Binary) plan.getValues()[loc]).getStringValue());
-          }
-          break;
-        case DOUBLE:
-          if (!(plan.getValues()[loc] instanceof Double)) {
-            plan.getValues()[loc] =
-                Double.parseDouble(((Binary) plan.getValues()[loc]).getStringValue());
-          }
-          break;
-        case FLOAT:
-          if (!(plan.getValues()[loc] instanceof Float)) {
-            plan.getValues()[loc] =
-                Float.parseFloat(((Binary) plan.getValues()[loc]).getStringValue());
-          }
-          break;
-        case BOOLEAN:
-          if (!(plan.getValues()[loc] instanceof Boolean)) {
-            plan.getValues()[loc] =
-                Boolean.parseBoolean(((Binary) plan.getValues()[loc]).getStringValue());
-          }
-          break;
-        case TEXT:
-          // need to do nothing
-          break;
-      }
-    } catch (ClassCastException e) {
-      logger.error("inconsistent type between client and server");
-    }
-  }
-
-  /**
-   * create timeseries with ignore PathAlreadyExistException
-   */
-  private void internalCreateTimeseries(String path, TSDataType dataType) throws MetadataException {
-    try {
-      mManager.createTimeseries(
-          path,
-          dataType,
-          getDefaultEncoding(dataType),
-          TSFileDescriptor.getInstance().getConfig().getCompressor(),
-          Collections.emptyMap());
-    } catch (PathAlreadyExistException e) {
-      if (logger.isDebugEnabled()) {
-        logger.debug("Ignore PathAlreadyExistException when Concurrent inserting"
-            + " a non-exist time series {}", path);
-      }
-    }
-  }
-
-  /**
-   * Get default encoding by dataType
-   */
-  private TSEncoding getDefaultEncoding(TSDataType dataType) {
-    IoTDBConfig conf = IoTDBDescriptor.getInstance().getConfig();
-    switch (dataType) {
-      case BOOLEAN:
-        return conf.getDefaultBooleanEncoding();
-      case INT32:
-        return conf.getDefaultInt32Encoding();
-      case INT64:
-        return conf.getDefaultInt64Encoding();
-      case FLOAT:
-        return conf.getDefaultFloatEncoding();
-      case DOUBLE:
-        return conf.getDefaultDoubleEncoding();
-      case TEXT:
-        return conf.getDefaultTextEncoding();
-      default:
-        throw new UnSupportedDataTypeException(
-            String.format("Data type %s is not supported.", dataType.toString()));
+      mManager.unlockInsert(insertPlan.getDeviceId());

Review comment:
       I would suggest you separate the locking part from `getSeriesSchemasAndLock` and explicitly call it in this method, as it is always better to explicitly put `lock` and `unlock` in the same code block.




----------------------------------------------------------------
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 merged pull request #1408: move getSeriesSchemas to MManager

Posted by GitBox <gi...@apache.org>.
jt2594838 merged pull request #1408:
URL: https://github.com/apache/incubator-iotdb/pull/1408


   


----------------------------------------------------------------
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 #1408: move getSeriesSchemas to MManager

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1846,4 +1836,219 @@ public void createMTreeSnapshot() {
       lock.readLock().unlock();
     }
   }
+
+  /**
+   * get schema for device.
+   * Attention!!!  Only support insertPlan and insertTabletsPlan
+   * @param deviceId
+   * @param measurementList
+   * @param plan
+   * @return
+   * @throws MetadataException
+   */
+  public MeasurementSchema[] getSeriesSchemas(String deviceId, String[] measurementList, PhysicalPlan plan) throws MetadataException {
+    MeasurementSchema[] schemas = new MeasurementSchema[measurementList.length];
+
+    MNode deviceNode = null;
+    // 1. get device node
+    deviceNode = getDeviceNode(deviceId);
+
+    // 2. get schema of each measurement
+    for (int i = 0; i < measurementList.length; i++) {
+      try {
+        // if do not has measurement
+        if (!deviceNode.hasChild(measurementList[i])) {
+          // could not create it
+          if (!config.isAutoCreateSchemaEnabled()) {
+            throw new MetadataException(String.format(
+              "Current deviceId[%s] does not contain measurement:%s", deviceId, measurementList[i]));
+          }
+
+          // create it
+          Path path = new Path(deviceId, measurementList[i]);
+          TSDataType dataType = getTypeInLoc(plan, i);
+
+          createTimeseries(
+            path.getFullPath(),
+            dataType,
+            getDefaultEncoding(dataType),
+            TSFileDescriptor.getInstance().getConfig().getCompressor(),
+            Collections.emptyMap());
+        }
+
+        MeasurementMNode measurementNode = (MeasurementMNode) getChild(deviceNode, measurementList[i]);
+
+        // check type is match
+        TSDataType insertDataType = null;
+        if (plan instanceof InsertPlan) {
+          if (!((InsertPlan)plan).isNeedInferType()) {
+            // only when InsertPlan's values is object[], we should check type
+            insertDataType = getTypeInLoc(plan, i);
+          } else {
+            insertDataType = measurementNode.getSchema().getType();
+          }
+        } else if (plan instanceof InsertTabletPlan) {
+          insertDataType = getTypeInLoc(plan, i);
+        }
+
+        if (measurementNode.getSchema().getType() != insertDataType) {
+          logger.warn("DataType mismatch, Insert measurement {} type {}, metadata tree type {}",
+            measurementList[i], insertDataType, measurementNode.getSchema().getType());
+          if (!config.isEnablePartialInsert()) {
+            throw new MetadataException(String.format(
+              "DataType mismatch, Insert measurement %s type %s, metadata tree type %s",
+              measurementList[i], insertDataType, measurementNode.getSchema().getType()));
+          } else {
+            // mark failed measurement
+            if (plan instanceof InsertTabletPlan) {
+              ((InsertTabletPlan) plan).markMeasurementInsertionFailed(i);
+            } else if (plan instanceof InsertPlan) {
+              ((InsertPlan) plan).markMeasurementInsertionFailed(i);
+            }
+            continue;
+          }
+        }
+
+        // maybe need to convert value type to the true type
+        if ((plan instanceof InsertPlan)) {
+          changeValueToRealType((InsertPlan) plan, i, measurementNode.getSchema().getType());
+        }
+
+        schemas[i] = measurementNode.getSchema();
+        if (schemas[i] != null) {
+          measurementList[i] = schemas[i].getMeasurementId();
+        }
+      } catch (MetadataException e) {
+        logger.warn("meet error when check {}.{}, message: {}", deviceId, measurementList[i],
+          e.getMessage());
+        if (config.isEnablePartialInsert()) {
+          // mark failed measurement
+          if (plan instanceof InsertPlan) {
+            ((InsertPlan) plan).markMeasurementInsertionFailed(i);
+          } else if (plan instanceof InsertTabletPlan) {
+            ((InsertTabletPlan) plan).markMeasurementInsertionFailed(i);
+          }
+        } else {
+          throw e;
+        }
+      }
+    }
+    return schemas;
+  }
+
+  private void changeValueToRealType(InsertPlan plan, int loc, TSDataType type) throws MetadataException {
+    plan.getTypes()[loc] = type;
+    try {
+      switch (type) {
+        case INT32:
+          if (!(plan.getValues()[loc] instanceof Integer)) {

Review comment:
       yes, could remove this check.




----------------------------------------------------------------
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