You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by hu...@apache.org on 2023/01/12 15:11:31 UTC

[iotdb] 01/03: implement type check and bind for target path

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

hui pushed a commit to branch lmh/intoType
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 561f4a0bf064b72ba7ed8cd15551b7a25f99b5f0
Author: Minghui Liu <li...@foxmail.com>
AuthorDate: Thu Jan 12 21:39:35 2023 +0800

    implement type check and bind for target path
---
 .../iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java  | 26 +++++++++++++++
 .../iotdb/db/mpp/plan/analyze/SelectIntoUtils.java | 38 ++++++++++++++++++++++
 .../parameter/DeviceViewIntoPathDescriptor.java    | 25 +++++++++++++-
 .../planner/plan/parameter/IntoPathDescriptor.java | 19 ++++++++++-
 .../apache/iotdb/db/utils/TypeInferenceUtils.java  | 37 ++++++++++++++++++---
 5 files changed, 138 insertions(+), 7 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
index 36324cac78..83bd6fe098 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
@@ -1273,6 +1273,7 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
     intoComponent.validate(sourceDevices, sourceColumns);
 
     DeviceViewIntoPathDescriptor deviceViewIntoPathDescriptor = new DeviceViewIntoPathDescriptor();
+    PathPatternTree targetPathTree = new PathPatternTree();
     IntoComponent.IntoDeviceMeasurementIterator intoDeviceMeasurementIterator =
         intoComponent.getIntoDeviceMeasurementIterator();
     for (PartialPath sourceDevice : sourceDevices) {
@@ -1293,12 +1294,24 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
         }
         deviceViewIntoPathDescriptor.specifyTargetDeviceMeasurement(
             sourceDevice, targetDevice, sourceColumn.toString(), targetMeasurement);
+
+        targetPathTree.appendFullPath(targetDevice, targetMeasurement);
+        deviceViewIntoPathDescriptor.recordSourceColumnDataType(
+            sourceColumn.toString(), analysis.getType(sourceColumn));
+
         intoDeviceMeasurementIterator.nextMeasurement();
       }
 
       intoDeviceMeasurementIterator.nextDevice();
     }
     deviceViewIntoPathDescriptor.validate();
+
+    // fetch schema of target paths
+    long startTime = System.nanoTime();
+    ISchemaTree targetSchemaTree = schemaFetcher.fetchSchema(targetPathTree);
+    QueryMetricsManager.getInstance().recordPlanCost(SCHEMA_FETCHER, System.nanoTime() - startTime);
+    deviceViewIntoPathDescriptor.bindType(targetSchemaTree);
+
     analysis.setDeviceViewIntoPathDescriptor(deviceViewIntoPathDescriptor);
   }
 
@@ -1320,6 +1333,7 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
     intoComponent.validate(sourceColumns);
 
     IntoPathDescriptor intoPathDescriptor = new IntoPathDescriptor();
+    PathPatternTree targetPathTree = new PathPatternTree();
     IntoComponent.IntoPathIterator intoPathIterator = intoComponent.getIntoPathIterator();
     for (Expression sourceColumn : sourceColumns) {
       PartialPath deviceTemplate = intoPathIterator.getDeviceTemplate();
@@ -1336,9 +1350,21 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
       intoPathDescriptor.specifyTargetPath(sourceColumn.toString(), targetPath);
       intoPathDescriptor.specifyDeviceAlignment(
           targetPath.getDevicePath().toString(), isAlignedDevice);
+
+      targetPathTree.appendFullPath(targetPath);
+      intoPathDescriptor.recordSourceColumnDataType(
+          sourceColumn.toString(), analysis.getType(sourceColumn));
+
       intoPathIterator.next();
     }
     intoPathDescriptor.validate();
+
+    // fetch schema of target paths
+    long startTime = System.nanoTime();
+    ISchemaTree targetSchemaTree = schemaFetcher.fetchSchema(targetPathTree);
+    QueryMetricsManager.getInstance().recordPlanCost(SCHEMA_FETCHER, System.nanoTime() - startTime);
+    intoPathDescriptor.bindType(targetSchemaTree);
+
     analysis.setIntoPathDescriptor(intoPathDescriptor);
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/SelectIntoUtils.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/SelectIntoUtils.java
index 18107823e5..9cfdbc194d 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/SelectIntoUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/SelectIntoUtils.java
@@ -19,15 +19,22 @@
 
 package org.apache.iotdb.db.mpp.plan.analyze;
 
+import org.apache.iotdb.commons.path.MeasurementPath;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.exception.sql.SemanticException;
+import org.apache.iotdb.db.mpp.common.schematree.ISchemaTree;
 import org.apache.iotdb.db.mpp.plan.expression.Expression;
 import org.apache.iotdb.db.mpp.plan.expression.leaf.TimeSeriesOperand;
+import org.apache.iotdb.db.utils.TypeInferenceUtils;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Pair;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.regex.Matcher;
 
+import static com.google.common.base.Preconditions.checkState;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.DOUBLE_COLONS;
 import static org.apache.iotdb.commons.conf.IoTDBConstant.LEVELED_PATH_TEMPLATE_PATTERN;
 
@@ -102,4 +109,35 @@ public class SelectIntoUtils {
     }
     return true;
   }
+
+  public static List<Pair<String, PartialPath>> bindTypeForSourceTargetPathPairList(
+      List<Pair<String, PartialPath>> sourceTargetPathPairList,
+      Map<String, TSDataType> sourceToDataTypeMap,
+      ISchemaTree targetSchemaTree) {
+    List<Pair<String, PartialPath>> sourceTypeBoundTargetPathPairList = new ArrayList<>();
+    for (Pair<String, PartialPath> sourceTargetPathPair : sourceTargetPathPairList) {
+      String sourceColumn = sourceTargetPathPair.left;
+      TSDataType sourceColumnType = sourceToDataTypeMap.get(sourceColumn);
+
+      MeasurementPath targetPathWithSchema;
+      PartialPath targetPath = sourceTargetPathPair.right;
+      List<MeasurementPath> actualTargetPaths =
+          targetSchemaTree.searchMeasurementPaths(targetPath).left;
+      if (actualTargetPaths.isEmpty()) {
+        targetPathWithSchema = new MeasurementPath(targetPath, sourceColumnType);
+      } else {
+        checkState(actualTargetPaths.size() == 1);
+        MeasurementPath actualTargetPath = actualTargetPaths.get(0);
+        if (!TypeInferenceUtils.canAutoCast(sourceColumnType, actualTargetPath.getSeriesType())) {
+          throw new SemanticException(
+              String.format(
+                  "The data type of target path (%s[%s]) is not compatible with the data type of source column (%s[%s]).",
+                  targetPath, actualTargetPath.getSeriesType(), sourceColumn, sourceColumnType));
+        }
+        targetPathWithSchema = actualTargetPath;
+      }
+      sourceTypeBoundTargetPathPairList.add(new Pair<>(sourceColumn, targetPathWithSchema));
+    }
+    return sourceTypeBoundTargetPathPairList;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/DeviceViewIntoPathDescriptor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/DeviceViewIntoPathDescriptor.java
index fc0486af0b..2d92bbee49 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/DeviceViewIntoPathDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/DeviceViewIntoPathDescriptor.java
@@ -22,6 +22,9 @@ package org.apache.iotdb.db.mpp.plan.planner.plan.parameter;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.path.PathDeserializeUtil;
 import org.apache.iotdb.db.exception.sql.SemanticException;
+import org.apache.iotdb.db.mpp.common.schematree.ISchemaTree;
+import org.apache.iotdb.db.mpp.plan.analyze.SelectIntoUtils;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
@@ -42,11 +45,14 @@ import static org.apache.iotdb.db.mpp.plan.statement.component.IntoComponent.DUP
 public class DeviceViewIntoPathDescriptor {
 
   // device -> List<(sourceColumn, targetPath)>
-  private final Map<String, List<Pair<String, PartialPath>>> deviceToSourceTargetPathPairListMap;
+  private Map<String, List<Pair<String, PartialPath>>> deviceToSourceTargetPathPairListMap;
 
   // targetDevice -> isAlignedDevice
   private final Map<String, Boolean> targetDeviceToAlignedMap;
 
+  // sourceColumn -> dataType (not serialize & deserialize)
+  private Map<String, TSDataType> sourceToDataTypeMap;
+
   public DeviceViewIntoPathDescriptor() {
     this.deviceToSourceTargetPathPairListMap = new HashMap<>();
     this.targetDeviceToAlignedMap = new HashMap<>();
@@ -77,6 +83,10 @@ public class DeviceViewIntoPathDescriptor {
     targetDeviceToAlignedMap.put(targetDevice, isAligned);
   }
 
+  public void recordSourceColumnDataType(String sourceColumn, TSDataType dataType) {
+    sourceToDataTypeMap.put(sourceColumn, dataType);
+  }
+
   public void validate() {
     List<PartialPath> targetPaths =
         deviceToSourceTargetPathPairListMap.values().stream()
@@ -88,6 +98,19 @@ public class DeviceViewIntoPathDescriptor {
     }
   }
 
+  public void bindType(ISchemaTree targetSchemaTree) {
+    Map<String, List<Pair<String, PartialPath>>> deviceToSourceTypeBoundTargetPathPairListMap =
+        new HashMap<>();
+    for (Map.Entry<String, List<Pair<String, PartialPath>>> sourceTargetEntry :
+        this.deviceToSourceTargetPathPairListMap.entrySet()) {
+      deviceToSourceTypeBoundTargetPathPairListMap.put(
+          sourceTargetEntry.getKey(),
+          SelectIntoUtils.bindTypeForSourceTargetPathPairList(
+              sourceTargetEntry.getValue(), sourceToDataTypeMap, targetSchemaTree));
+    }
+    this.deviceToSourceTargetPathPairListMap = deviceToSourceTypeBoundTargetPathPairListMap;
+  }
+
   public Map<String, List<Pair<String, PartialPath>>> getDeviceToSourceTargetPathPairListMap() {
     return deviceToSourceTargetPathPairListMap;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoPathDescriptor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoPathDescriptor.java
index b71e470f01..78237b27f7 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoPathDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/IntoPathDescriptor.java
@@ -22,6 +22,9 @@ package org.apache.iotdb.db.mpp.plan.planner.plan.parameter;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.path.PathDeserializeUtil;
 import org.apache.iotdb.db.exception.sql.SemanticException;
+import org.apache.iotdb.db.mpp.common.schematree.ISchemaTree;
+import org.apache.iotdb.db.mpp.plan.analyze.SelectIntoUtils;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
@@ -42,14 +45,18 @@ import static org.apache.iotdb.db.mpp.plan.statement.component.IntoComponent.DUP
 public class IntoPathDescriptor {
 
   // List<(sourceColumn, targetPath)>
-  private final List<Pair<String, PartialPath>> sourceTargetPathPairList;
+  private List<Pair<String, PartialPath>> sourceTargetPathPairList;
 
   // targetDevice -> isAlignedDevice
   private final Map<String, Boolean> targetDeviceToAlignedMap;
 
+  // sourceColumn -> dataType (not serialize & deserialize)
+  private Map<String, TSDataType> sourceToDataTypeMap;
+
   public IntoPathDescriptor() {
     this.sourceTargetPathPairList = new ArrayList<>();
     this.targetDeviceToAlignedMap = new HashMap<>();
+    this.sourceToDataTypeMap = new HashMap<>();
   }
 
   public IntoPathDescriptor(
@@ -71,6 +78,10 @@ public class IntoPathDescriptor {
     targetDeviceToAlignedMap.put(targetDevice, isAligned);
   }
 
+  public void recordSourceColumnDataType(String sourceColumn, TSDataType dataType) {
+    sourceToDataTypeMap.put(sourceColumn, dataType);
+  }
+
   public void validate() {
     List<PartialPath> targetPaths =
         sourceTargetPathPairList.stream().map(Pair::getRight).collect(Collectors.toList());
@@ -79,6 +90,12 @@ public class IntoPathDescriptor {
     }
   }
 
+  public void bindType(ISchemaTree targetSchemaTree) {
+    this.sourceTargetPathPairList =
+        SelectIntoUtils.bindTypeForSourceTargetPathPairList(
+            sourceTargetPathPairList, sourceToDataTypeMap, targetSchemaTree);
+  }
+
   public List<Pair<String, PartialPath>> getSourceTargetPathPairList() {
     return sourceTargetPathPairList;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/TypeInferenceUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/TypeInferenceUtils.java
index 96c2b8292b..e4690d1260 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/TypeInferenceUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/TypeInferenceUtils.java
@@ -29,19 +29,19 @@ import org.apache.commons.lang3.StringUtils;
 
 public class TypeInferenceUtils {
 
-  private static TSDataType booleanStringInferType =
+  private static final TSDataType booleanStringInferType =
       IoTDBDescriptor.getInstance().getConfig().getBooleanStringInferType();
 
-  private static TSDataType integerStringInferType =
+  private static final TSDataType integerStringInferType =
       IoTDBDescriptor.getInstance().getConfig().getIntegerStringInferType();
 
-  private static TSDataType longStringInferType =
+  private static final TSDataType longStringInferType =
       IoTDBDescriptor.getInstance().getConfig().getLongStringInferType();
 
-  private static TSDataType floatingStringInferType =
+  private static final TSDataType floatingStringInferType =
       IoTDBDescriptor.getInstance().getConfig().getFloatingStringInferType();
 
-  private static TSDataType nanStringInferType =
+  private static final TSDataType nanStringInferType =
       IoTDBDescriptor.getInstance().getConfig().getNanStringInferType();
 
   private TypeInferenceUtils() {}
@@ -157,4 +157,31 @@ public class TypeInferenceUtils {
         throw new IllegalArgumentException("Invalid Aggregation function: " + aggrFuncName);
     }
   }
+
+  public static boolean canAutoCast(TSDataType fromType, TSDataType toType) {
+    if (fromType.equals(toType)) {
+      return true;
+    }
+
+    switch (fromType) {
+      case INT32:
+        switch (toType) {
+          case INT64:
+          case FLOAT:
+          case DOUBLE:
+            return true;
+          default:
+            return false;
+        }
+      case INT64:
+      case FLOAT:
+        return toType.equals(TSDataType.DOUBLE);
+      case DOUBLE:
+      case BOOLEAN:
+      case TEXT:
+        return false;
+      default:
+        throw new IllegalArgumentException("Unknown data type: " + fromType);
+    }
+  }
 }