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

[iotdb] branch master updated: Fix show timeseries failure caused by broken views. (#10000)

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

zyk 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 e510ec02a0d Fix show timeseries failure caused by broken views. (#10000)
e510ec02a0d is described below

commit e510ec02a0d0198fbc46dabd16bc23c02c0fcf3b
Author: 橘子 <70...@users.noreply.github.com>
AuthorDate: Wed May 31 20:02:20 2023 +0800

    Fix show timeseries failure caused by broken views. (#10000)
---
 .../impl/schema/DeleteLogicalViewProcedure.java    |  4 +-
 .../metadata/view/BrokenViewException.java         | 41 +++++++++++++
 .../metadata/view/ViewNotExistException.java       | 70 ++++++++++++++++++++++
 .../schema/source/LogicalViewSchemaSource.java     | 35 +++++++----
 .../schema/source/TimeSeriesSchemaSource.java      | 38 ++++++++----
 .../visitor/CompleteMeasurementSchemaVisitor.java  |  9 +--
 6 files changed, 165 insertions(+), 32 deletions(-)

diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java
index e333a3af225..8547eeaed6a 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java
@@ -36,7 +36,7 @@ import org.apache.iotdb.confignode.procedure.exception.ProcedureYieldException;
 import org.apache.iotdb.confignode.procedure.impl.statemachine.StateMachineProcedure;
 import org.apache.iotdb.confignode.procedure.state.schema.DeleteLogicalViewState;
 import org.apache.iotdb.confignode.procedure.store.ProcedureType;
-import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.view.ViewNotExistException;
 import org.apache.iotdb.mpp.rpc.thrift.TConstructViewSchemaBlackListReq;
 import org.apache.iotdb.mpp.rpc.thrift.TDeleteViewSchemaReq;
 import org.apache.iotdb.mpp.rpc.thrift.TInvalidateMatchedSchemaCacheReq;
@@ -95,7 +95,7 @@ public class DeleteLogicalViewProcedure
           } else {
             setFailure(
                 new ProcedureException(
-                    new PathNotExistException(
+                    new ViewNotExistException(
                         patternTree.getAllPathPatterns().stream()
                             .map(PartialPath::getFullPath)
                             .collect(Collectors.toList()),
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/view/BrokenViewException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/view/BrokenViewException.java
new file mode 100644
index 00000000000..a87f5fcf3d7
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/view/BrokenViewException.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.exception.metadata.view;
+
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.commons.path.MeasurementPath;
+import org.apache.iotdb.commons.path.PartialPath;
+
+import java.util.List;
+
+public class BrokenViewException extends MetadataException {
+  public BrokenViewException(String sourcePath, List<MeasurementPath> matchedPaths) {
+    super(
+        String.format(
+            "View is broken! The source path [%s] maps to unmatched %s path(s): %s.",
+            sourcePath, matchedPaths.size(), matchedPaths.toString()));
+  }
+
+  public BrokenViewException(String viewPath, String sourcePath, List<PartialPath> matchedPaths) {
+    super(
+        String.format(
+            "View [%s] is broken! The source path [%s] maps to unmatched %s path(s): %s.",
+            viewPath, sourcePath, matchedPaths.size(), matchedPaths.toString()));
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/view/ViewNotExistException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/view/ViewNotExistException.java
new file mode 100644
index 00000000000..4bcd29c4071
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/view/ViewNotExistException.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.exception.metadata.view;
+
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.rpc.TSStatusCode;
+
+import java.util.List;
+
+public class ViewNotExistException extends MetadataException {
+
+  private static final String VIEW_NOT_EXIST_WRONG_MESSAGE = "View [%s] does not exist";
+
+  private static final String NORMAL_VIEW_NOT_EXIST_WRONG_MESSAGE =
+      "View [%s] does not exist or is represented by schema template";
+
+  private static final String TEMPLATE_VIEW_NOT_EXIST_WRONG_MESSAGE =
+      "View [%s] does not exist or is not represented by schema template";
+
+  public ViewNotExistException(String path) {
+    super(
+        String.format(VIEW_NOT_EXIST_WRONG_MESSAGE, path),
+        TSStatusCode.PATH_NOT_EXIST.getStatusCode());
+  }
+
+  public ViewNotExistException(String path, boolean isUserException) {
+    super(
+        String.format(VIEW_NOT_EXIST_WRONG_MESSAGE, path),
+        TSStatusCode.PATH_NOT_EXIST.getStatusCode(),
+        isUserException);
+  }
+
+  public ViewNotExistException(List<String> paths) {
+    super(
+        String.format(
+            VIEW_NOT_EXIST_WRONG_MESSAGE,
+            paths.size() == 1
+                ? paths.get(0)
+                : paths.get(0) + " ... " + paths.get(paths.size() - 1)),
+        TSStatusCode.PATH_NOT_EXIST.getStatusCode());
+  }
+
+  public ViewNotExistException(List<String> paths, boolean isTemplateSeries) {
+    super(
+        String.format(
+            isTemplateSeries
+                ? TEMPLATE_VIEW_NOT_EXIST_WRONG_MESSAGE
+                : NORMAL_VIEW_NOT_EXIST_WRONG_MESSAGE,
+            paths.size() == 1
+                ? paths.get(0)
+                : paths.get(0) + " ... " + paths.get(paths.size() - 1)),
+        TSStatusCode.PATH_NOT_EXIST.getStatusCode());
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/source/LogicalViewSchemaSource.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/source/LogicalViewSchemaSource.java
index 41911b2dbdd..d324452997a 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/source/LogicalViewSchemaSource.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/source/LogicalViewSchemaSource.java
@@ -70,6 +70,9 @@ public class LogicalViewSchemaSource implements ISchemaSource<ITimeSeriesSchemaI
    */
   private List<ITimeSeriesSchemaInfo> delayedLogicalViewList;
 
+  private static final String unknownDataTypeString = "UNKNOWN";
+  private static final String viewTypeOfLogicalView = "logical";
+
   LogicalViewSchemaSource(
       PartialPath pathPattern, long limit, long offset, SchemaFilter schemaFilter) {
     this.pathPattern = pathPattern;
@@ -126,7 +129,7 @@ public class LogicalViewSchemaSource implements ISchemaSource<ITimeSeriesSchemaI
     return schemaRegion.getSchemaRegionStatistics().getSeriesNumber();
   }
 
-  private List<TSDataType> analyzeDataTypeOfDelayedViews() {
+  private List<String> analyzeDataTypeOfDelayedViews() {
     if (this.delayedLogicalViewList == null || this.delayedLogicalViewList.size() <= 0) {
       return new ArrayList<>();
     }
@@ -150,14 +153,24 @@ public class LogicalViewSchemaSource implements ISchemaSource<ITimeSeriesSchemaI
     CompleteMeasurementSchemaVisitor completeMeasurementSchemaVisitor =
         new CompleteMeasurementSchemaVisitor();
     Map<NodeRef<Expression>, TSDataType> expressionTypes = new HashMap<>();
-    List<TSDataType> dataTypeList = new ArrayList<>();
+    List<String> dataTypeStringList = new ArrayList<>();
     for (ViewExpression viewExpression : viewExpressionList) {
-      Expression expression = transformToExpressionVisitor.process(viewExpression, null);
-      expression = completeMeasurementSchemaVisitor.process(expression, schemaTree);
-      ExpressionTypeAnalyzer.analyzeExpression(expressionTypes, expression);
-      dataTypeList.add(expressionTypes.get(NodeRef.of(expression)));
+      Expression expression = null;
+      boolean viewIsBroken = false;
+      try {
+        expression = transformToExpressionVisitor.process(viewExpression, null);
+        expression = completeMeasurementSchemaVisitor.process(expression, schemaTree);
+        ExpressionTypeAnalyzer.analyzeExpression(expressionTypes, expression);
+      } catch (Exception e) {
+        viewIsBroken = true;
+      }
+      if (viewIsBroken) {
+        dataTypeStringList.add(unknownDataTypeString);
+      } else {
+        dataTypeStringList.add(expressionTypes.get(NodeRef.of(expression)).toString());
+      }
     }
-    return dataTypeList;
+    return dataTypeStringList;
   }
 
   @Override
@@ -166,22 +179,22 @@ public class LogicalViewSchemaSource implements ISchemaSource<ITimeSeriesSchemaI
     if (this.delayedLogicalViewList == null || this.delayedLogicalViewList.size() <= 0) {
       return;
     }
-    List<TSDataType> dataTypeList = this.analyzeDataTypeOfDelayedViews();
+    List<String> dataTypeStringList = this.analyzeDataTypeOfDelayedViews();
     // process delayed tasks
     for (int index = 0; index < this.delayedLogicalViewList.size(); index++) {
       ITimeSeriesSchemaInfo series = this.delayedLogicalViewList.get(index);
-      TSDataType expressionTypeOfThisView = dataTypeList.get(index);
+      String expressionTypeOfThisView = dataTypeStringList.get(index);
 
       builder.getTimeColumnBuilder().writeLong(0);
       builder.writeNullableText(0, series.getFullPath());
       builder.writeNullableText(1, database);
 
-      builder.writeNullableText(2, expressionTypeOfThisView.toString());
+      builder.writeNullableText(2, expressionTypeOfThisView);
 
       builder.writeNullableText(3, mapToString(series.getTags()));
       builder.writeNullableText(4, mapToString(series.getAttributes()));
 
-      builder.writeNullableText(5, "logical");
+      builder.writeNullableText(5, viewTypeOfLogicalView);
       builder.writeNullableText(
           6, ((LogicalViewSchema) series.getSchema()).getExpression().toString());
       builder.declarePosition();
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/source/TimeSeriesSchemaSource.java b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/source/TimeSeriesSchemaSource.java
index 7f669fa9f70..e4ef3e38900 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/source/TimeSeriesSchemaSource.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/schema/source/TimeSeriesSchemaSource.java
@@ -73,6 +73,10 @@ public class TimeSeriesSchemaSource implements ISchemaSource<ITimeSeriesSchemaIn
    */
   private List<ITimeSeriesSchemaInfo> delayedLogicalViewList;
 
+  private static final String unknownDataTypeString = "UNKNOWN";
+  private static final String viewTypeOfLogicalView = "logical";
+  private static final String viewTypeOfNonView = "";
+
   TimeSeriesSchemaSource(
       PartialPath pathPattern,
       boolean isPrefixMatch,
@@ -136,7 +140,7 @@ public class TimeSeriesSchemaSource implements ISchemaSource<ITimeSeriesSchemaIn
     builder.writeNullableText(7, mapToString(series.getAttributes()));
     builder.writeNullableText(8, deadbandInfo.left);
     builder.writeNullableText(9, deadbandInfo.right);
-    builder.writeNullableText(10, "");
+    builder.writeNullableText(10, viewTypeOfNonView);
     builder.declarePosition();
   }
 
@@ -150,7 +154,7 @@ public class TimeSeriesSchemaSource implements ISchemaSource<ITimeSeriesSchemaIn
     return schemaRegion.getSchemaRegionStatistics().getSeriesNumber();
   }
 
-  private List<TSDataType> analyzeDataTypeOfDelayedViews() {
+  private List<String> analyzeDataTypeOfDelayedViews() {
     if (this.delayedLogicalViewList == null || this.delayedLogicalViewList.size() <= 0) {
       return new ArrayList<>();
     }
@@ -174,14 +178,24 @@ public class TimeSeriesSchemaSource implements ISchemaSource<ITimeSeriesSchemaIn
     CompleteMeasurementSchemaVisitor completeMeasurementSchemaVisitor =
         new CompleteMeasurementSchemaVisitor();
     Map<NodeRef<Expression>, TSDataType> expressionTypes = new HashMap<>();
-    List<TSDataType> dataTypeList = new ArrayList<>();
+    List<String> dataTypeStringList = new ArrayList<>();
     for (ViewExpression viewExpression : viewExpressionList) {
-      Expression expression = transformToExpressionVisitor.process(viewExpression, null);
-      expression = completeMeasurementSchemaVisitor.process(expression, schemaTree);
-      ExpressionTypeAnalyzer.analyzeExpression(expressionTypes, expression);
-      dataTypeList.add(expressionTypes.get(NodeRef.of(expression)));
+      Expression expression = null;
+      boolean viewIsBroken = false;
+      try {
+        expression = transformToExpressionVisitor.process(viewExpression, null);
+        expression = completeMeasurementSchemaVisitor.process(expression, schemaTree);
+        ExpressionTypeAnalyzer.analyzeExpression(expressionTypes, expression);
+      } catch (Exception e) {
+        viewIsBroken = true;
+      }
+      if (viewIsBroken) {
+        dataTypeStringList.add(unknownDataTypeString);
+      } else {
+        dataTypeStringList.add(expressionTypes.get(NodeRef.of(expression)).toString());
+      }
     }
-    return dataTypeList;
+    return dataTypeStringList;
   }
 
   @Override
@@ -189,11 +203,11 @@ public class TimeSeriesSchemaSource implements ISchemaSource<ITimeSeriesSchemaIn
     if (this.delayedLogicalViewList == null || this.delayedLogicalViewList.size() <= 0) {
       return;
     }
-    List<TSDataType> dataTypeList = this.analyzeDataTypeOfDelayedViews();
+    List<String> dataTypeStringList = this.analyzeDataTypeOfDelayedViews();
     // process delayed tasks
     for (int index = 0; index < this.delayedLogicalViewList.size(); index++) {
       ITimeSeriesSchemaInfo series = this.delayedLogicalViewList.get(index);
-      TSDataType expressionTypeOfThisView = dataTypeList.get(index);
+      String expressionTypeOfThisView = dataTypeStringList.get(index);
 
       Pair<String, String> deadbandInfo =
           MetaUtils.parseDeadbandInfo(series.getSchema().getProps());
@@ -201,14 +215,14 @@ public class TimeSeriesSchemaSource implements ISchemaSource<ITimeSeriesSchemaIn
       builder.writeNullableText(0, series.getFullPath());
       builder.writeNullableText(1, series.getAlias());
       builder.writeNullableText(2, database);
-      builder.writeNullableText(3, expressionTypeOfThisView.toString());
+      builder.writeNullableText(3, expressionTypeOfThisView);
       builder.writeNullableText(4, null);
       builder.writeNullableText(5, null);
       builder.writeNullableText(6, mapToString(series.getTags()));
       builder.writeNullableText(7, mapToString(series.getAttributes()));
       builder.writeNullableText(8, deadbandInfo.left);
       builder.writeNullableText(9, deadbandInfo.right);
-      builder.writeNullableText(10, "logical");
+      builder.writeNullableText(10, viewTypeOfLogicalView);
       builder.declarePosition();
     }
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/CompleteMeasurementSchemaVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/CompleteMeasurementSchemaVisitor.java
index fe9716f3b18..d7bf53d9827 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/CompleteMeasurementSchemaVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/visitor/CompleteMeasurementSchemaVisitor.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.mpp.plan.expression.visitor;
 
 import org.apache.iotdb.commons.path.MeasurementPath;
 import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.db.exception.metadata.view.BrokenViewException;
 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.binary.BinaryExpression;
@@ -29,7 +30,6 @@ import org.apache.iotdb.db.mpp.plan.expression.multi.FunctionExpression;
 import org.apache.iotdb.db.mpp.plan.expression.ternary.TernaryExpression;
 import org.apache.iotdb.db.mpp.plan.expression.unary.UnaryExpression;
 
-import java.rmi.UnexpectedException;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -95,12 +95,7 @@ public class CompleteMeasurementSchemaVisitor extends ExpressionVisitor<Expressi
       } catch (Exception notAMeasurementPath) {
         List<MeasurementPath> actualPaths = schemaTree.searchMeasurementPaths(path).left;
         if (actualPaths.size() != 1) {
-          throw new UnexpectedException(
-              "given one path ["
-                  + path.toString()
-                  + "], "
-                  + "but got unmatched path(s) :"
-                  + actualPaths.toString());
+          throw new BrokenViewException(path.getFullPath(), actualPaths);
         }
         return new TimeSeriesOperand(actualPaths.get(0));
       }