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

[iotdb] 03/08: from sql visitor to logical operator

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

rong pushed a commit to branch select-into
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit d5413d3570151d812e5a86b39d94a52bae1ecc1c
Author: Steve Yurong Su <ro...@apache.org>
AuthorDate: Tue Jul 13 16:17:56 2021 +0800

    from sql visitor to logical operator
---
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4   |  13 ++-
 .../apache/iotdb/db/qp/constant/SQLConstant.java   |   4 +
 .../org/apache/iotdb/db/qp/logical/Operator.java   |   2 +
 .../db/qp/logical/crud/SelectIntoOperator.java     |  54 ++++++++++
 .../apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java    | 118 +++++++++++++--------
 .../iotdb/db/qp/physical/PhysicalPlanTest.java     |  16 +--
 6 files changed, 149 insertions(+), 58 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
index cdffba1..54acf2b 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
@@ -104,8 +104,7 @@ statement
     | START TRIGGER triggerName=ID #startTrigger
     | STOP TRIGGER triggerName=ID #stopTrigger
     | SHOW TRIGGERS #showTriggers
-    | selectClause fromClause whereClause? specialClause? #selectStatement
-    | selectClause intoClause fromClause whereClause? specialClause? #selectIntoStatement
+    | selectClause intoClause? fromClause whereClause? specialClause? #selectStatement
     | CREATE (CONTINUOUS QUERY | CQ) continuousQueryName=ID
       resampleClause?
       cqSelectIntoClause #createContinuousQueryStatement
@@ -139,6 +138,11 @@ intoClause
     : INTO intoPath (COMMA intoPath)*
     ;
 
+intoPath
+    : fullPath
+    | nodeNameWithoutStar
+    ;
+
 alias
     : LR_BRACKET ID RR_BRACKET
     ;
@@ -346,11 +350,6 @@ cqSelectIntoClause
     : BEGIN selectClause INTO intoPath fromClause cqGroupByTimeClause END
     ;
 
-intoPath
-    : fullPath
-    | nodeNameWithoutStar
-    ;
-
 cqGroupByTimeClause
     : GROUP BY TIME LR_BRACKET
       DURATION
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
index 7509bf0..bf2e9c2 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
@@ -179,6 +179,8 @@ public class SQLConstant {
   public static final int TOK_CONTINUOUS_QUERY_DROP = 107;
   public static final int TOK_SHOW_CONTINUOUS_QUERIES = 108;
 
+  public static final int TOK_SELECT_INTO = 109;
+
   public static final Map<Integer, String> tokenNames = new HashMap<>();
 
   public static String[] getSingleRootArray() {
@@ -247,6 +249,8 @@ public class SQLConstant {
     tokenNames.put(TOK_CONTINUOUS_QUERY_CREATE, "TOK_CONTINUOUS_QUERY_CREATE");
     tokenNames.put(TOK_CONTINUOUS_QUERY_DROP, "TOK_CONTINUOUS_QUERY_DROP");
     tokenNames.put(TOK_SHOW_CONTINUOUS_QUERIES, "TOK_SHOW_CONTINUOUS_QUERIES");
+
+    tokenNames.put(TOK_SELECT_INTO, "TOK_SELECT_INTO");
   }
 
   public static boolean isReservedPath(PartialPath pathStr) {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
index 56eba91..53234cb 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
@@ -126,6 +126,8 @@ public abstract class Operator {
     UDAF,
     UDTF,
 
+    SELECT_INTO,
+
     CREATE_FUNCTION,
     DROP_FUNCTION,
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectIntoOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectIntoOperator.java
new file mode 100644
index 0000000..322b9f8
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectIntoOperator.java
@@ -0,0 +1,54 @@
+/*
+ * 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.qp.logical.crud;
+
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.logical.Operator;
+
+import java.util.List;
+
+public class SelectIntoOperator extends Operator {
+
+  private QueryOperator queryOperator;
+
+  private List<PartialPath> intoPaths;
+
+  public SelectIntoOperator() {
+    super(SQLConstant.TOK_SELECT_INTO);
+    operatorType = OperatorType.SELECT_INTO;
+  }
+
+  public void setQueryOperator(QueryOperator queryOperator) {
+    this.queryOperator = queryOperator;
+  }
+
+  public QueryOperator getQueryOperator() {
+    return queryOperator;
+  }
+
+  public void setIntoPaths(List<PartialPath> intoPaths) {
+    this.intoPaths = intoPaths;
+  }
+
+  public List<PartialPath> getIntoPaths() {
+    return intoPaths;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
index 909bf07..8a40fa4 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
@@ -45,6 +45,7 @@ import org.apache.iotdb.db.qp.logical.crud.InsertOperator;
 import org.apache.iotdb.db.qp.logical.crud.LastQueryOperator;
 import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
 import org.apache.iotdb.db.qp.logical.crud.SelectComponent;
+import org.apache.iotdb.db.qp.logical.crud.SelectIntoOperator;
 import org.apache.iotdb.db.qp.logical.crud.SpecialClauseComponent;
 import org.apache.iotdb.db.qp.logical.crud.UDFQueryOperator;
 import org.apache.iotdb.db.qp.logical.crud.WhereComponent;
@@ -156,6 +157,7 @@ import org.apache.iotdb.db.qp.sql.SqlBaseParser.InsertColumnsSpecContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.InsertMultiValueContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.InsertStatementContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.InsertValuesSpecContext;
+import org.apache.iotdb.db.qp.sql.SqlBaseParser.IntoPathContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.KillQueryContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.LimitClauseContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.LimitStatementContext;
@@ -284,7 +286,9 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
       "For delete statement, where clause can only contain atomic expressions like : "
           + "time > XXX, time <= XXX, or two atomic expressions connected by 'AND'";
 
-  private static final Pattern cqLevelNodePattern = Pattern.compile("\\$\\{\\w+}");
+  // used to match "{x}", where x is a integer.
+  // for create-cq clause and select-into clause.
+  private static final Pattern leveledPathNodePattern = Pattern.compile("\\$\\{\\w+}");
 
   private ZoneId zoneId;
   private QueryOperator queryOp;
@@ -1045,8 +1049,8 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
         queryOp.setWhereComponent(whereComponent);
       }
     }
-
-    return queryOp;
+    // 4. Check whether it's a select-into clause
+    return ctx.intoClause() == null ? queryOp : parseAndConstructSelectIntoOperator(ctx);
   }
 
   public void parseSelectClause(SelectClauseContext ctx) {
@@ -1100,6 +1104,68 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
         resultColumnContext.AS() == null ? null : resultColumnContext.ID().getText());
   }
 
+  private SelectIntoOperator parseAndConstructSelectIntoOperator(SelectStatementContext ctx) {
+    if (queryOp.getFromComponent().getPrefixPaths().size() != 1) {
+      throw new SQLParserException(
+          "select into: the number of prefix paths in the from clause should be 1.");
+    }
+
+    int sourcePathsCount = queryOp.getSelectComponent().getResultColumns().size();
+    if (sourcePathsCount != ctx.intoClause().intoPath().size()) {
+      throw new SQLParserException(
+          "select into: the number of source paths and the number of target paths should be the same.");
+    }
+
+    SelectIntoOperator selectIntoOperator = new SelectIntoOperator();
+    selectIntoOperator.setQueryOperator(queryOp);
+    List<PartialPath> intoPaths = new ArrayList<>();
+    for (int i = 0; i < sourcePathsCount; ++i) {
+      intoPaths.add(parseIntoPath(ctx.intoClause().intoPath(i)));
+    }
+    selectIntoOperator.setIntoPaths(intoPaths);
+    return selectIntoOperator;
+  }
+
+  private PartialPath parseIntoPath(IntoPathContext intoPathContext) {
+    int levelLimitOfSourcePrefixPath = queryOp.getSpecialClauseComponent().getLevel();
+    if (levelLimitOfSourcePrefixPath == -1) {
+      levelLimitOfSourcePrefixPath =
+          queryOp.getFromComponent().getPrefixPaths().get(0).getNodeLength() - 1;
+    }
+
+    PartialPath intoPath = null;
+    if (intoPathContext.fullPath() != null) {
+      intoPath = parseFullPath(intoPathContext.fullPath());
+
+      Matcher m = leveledPathNodePattern.matcher(intoPath.getFullPath());
+      while (m.find()) {
+        String param = m.group();
+        int nodeIndex = 0;
+        try {
+          nodeIndex = Integer.parseInt(param.substring(2, param.length() - 1).trim());
+        } catch (NumberFormatException e) {
+          throw new SQLParserException("the x of ${x} should be an integer.");
+        }
+        if (nodeIndex < 1 || levelLimitOfSourcePrefixPath < nodeIndex) {
+          throw new SQLParserException(
+              "the x of ${x} should be greater than 0 and equal to or less than <level> or the length of queried path prefix.");
+        }
+      }
+    } else if (intoPathContext.nodeNameWithoutStar() != null) {
+      String[] intoPathNodes = new String[levelLimitOfSourcePrefixPath + 2];
+      intoPathNodes[0] = "root";
+      for (int i = 1; i <= levelLimitOfSourcePrefixPath; ++i) {
+        intoPathNodes[i] = "${" + i + "}";
+      }
+      intoPathNodes[levelLimitOfSourcePrefixPath + 1] =
+          intoPathContext.nodeNameWithoutStar().getText();
+
+      intoPath = new PartialPath(intoPathNodes);
+    }
+
+    return intoPath;
+  }
+
   @Override
   public Operator visitDropContinuousQueryStatement(DropContinuousQueryStatementContext ctx) {
     DropContinuousQueryOperator dropContinuousQueryOperator =
@@ -1199,54 +1265,18 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
     }
 
     if (queryOp.getFromComponent().getPrefixPaths().size() > 1) {
-      throw new SQLParserException("CQ: CQ currently does not support multiple series .");
+      throw new SQLParserException("CQ: CQ currently does not support multiple series.");
     }
 
     parseCqGroupByTimeClause(ctx.cqGroupByTimeClause());
 
-    int fromLen = queryOp.getFromComponent().getPrefixPaths().get(0).getNodeLength();
-    int queryLevel = queryOp.getSpecialClauseComponent().getLevel();
-    if (queryLevel >= fromLen) {
+    int groupByQueryLevel = queryOp.getSpecialClauseComponent().getLevel();
+    int fromPrefixLevelLimit = queryOp.getFromComponent().getPrefixPaths().get(0).getNodeLength();
+    if (groupByQueryLevel >= fromPrefixLevelLimit) {
       throw new SQLParserException("CQ: Level should not exceed the <from_prefix> length.");
     }
 
-    PartialPath targetPath = null;
-
-    int trueLevel = queryLevel;
-    if (trueLevel == -1) {
-      trueLevel = fromLen - 1;
-    }
-
-    if (ctx.intoPath().fullPath() != null) {
-      targetPath = parseFullPath(ctx.intoPath().fullPath());
-      Matcher m = cqLevelNodePattern.matcher(targetPath.getFullPath());
-      while (m.find()) {
-        String param = m.group();
-        int nodeIndex = 0;
-        try {
-          nodeIndex = Integer.parseInt(param.substring(2, param.length() - 1).trim());
-        } catch (NumberFormatException e) {
-          throw new SQLParserException("CQ: x of ${x} should be an integer.");
-        }
-        if (nodeIndex < 1 || nodeIndex > trueLevel) {
-          throw new SQLParserException(
-              "CQ: x of ${x} should be greater than 0 and equal to or less than <level> or the length of queried path prefix.");
-        }
-      }
-    } else if (ctx.intoPath().nodeNameWithoutStar() != null) {
-
-      List<String> targetNodes = new ArrayList<>();
-
-      targetNodes.add("root");
-
-      for (int i = 1; i <= trueLevel; i++) {
-        targetNodes.add("${" + i + "}");
-      }
-      targetNodes.add(ctx.intoPath().nodeNameWithoutStar().getText());
-      targetPath = new PartialPath(targetNodes.toArray(new String[0]));
-    }
-
-    createContinuousQueryOperator.setTargetPath(targetPath);
+    createContinuousQueryOperator.setTargetPath(parseIntoPath(ctx.intoPath()));
     createContinuousQueryOperator.setQueryOperator(queryOp);
   }
 
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanTest.java
index 56d5708..9fa6591 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/physical/PhysicalPlanTest.java
@@ -1331,7 +1331,7 @@ public class PhysicalPlanTest {
           (CreateContinuousQueryPlan) processor.parseSQLToPhysicalPlan(sql);
       fail();
     } catch (SQLParserException e) {
-      assertEquals("CQ: x of ${x} should be an integer.", e.getMessage());
+      assertTrue(e.getMessage().contains("the x of ${x} should be an integer."));
     }
   }
 
@@ -1345,9 +1345,10 @@ public class PhysicalPlanTest {
           (CreateContinuousQueryPlan) processor.parseSQLToPhysicalPlan(sql);
       fail();
     } catch (SQLParserException e) {
-      assertEquals(
-          "CQ: x of ${x} should be greater than 0 and equal to or less than <level> or the length of queried path prefix.",
-          e.getMessage());
+      assertTrue(
+          e.getMessage()
+              .contains(
+                  "the x of ${x} should be greater than 0 and equal to or less than <level> or the length of queried path prefix."));
     }
   }
 
@@ -1392,9 +1393,10 @@ public class PhysicalPlanTest {
           (CreateContinuousQueryPlan) processor.parseSQLToPhysicalPlan(sql);
       fail();
     } catch (SQLParserException e) {
-      assertEquals(
-          "CQ: x of ${x} should be greater than 0 and equal to or less than <level> or the length of queried path prefix.",
-          e.getMessage());
+      assertTrue(
+          e.getMessage()
+              .contains(
+                  "the x of ${x} should be greater than 0 and equal to or less than <level> or the length of queried path prefix."));
     }
   }