You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ro...@apache.org on 2022/01/15 10:53:16 UTC

[iotdb] 01/01: [IOTDB-2310][IOTDB-2311][IOTDB-2372] Add query type check for REST query API

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

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

commit 24a9c9ff9fca50077ab19564db7cb87e5cf73d8b
Author: Steve Yurong Su <ro...@apache.org>
AuthorDate: Sat Jan 15 18:52:21 2022 +0800

    [IOTDB-2310][IOTDB-2311][IOTDB-2372] Add query type check for REST query API
---
 .../handler/PhysicalPlanValidationHandler.java     | 78 ++++++++++++++++++++++
 .../rest/handler/RequestValidationHandler.java     |  5 ++
 .../db/protocol/rest/impl/RestApiServiceImpl.java  |  5 +-
 .../main/java/org/apache/iotdb/db/qp/Planner.java  | 29 ++++++++
 4 files changed, 116 insertions(+), 1 deletion(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/protocol/rest/handler/PhysicalPlanValidationHandler.java b/server/src/main/java/org/apache/iotdb/db/protocol/rest/handler/PhysicalPlanValidationHandler.java
new file mode 100644
index 0000000..75bb686
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/protocol/rest/handler/PhysicalPlanValidationHandler.java
@@ -0,0 +1,78 @@
+/*
+ * 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.protocol.rest.handler;
+
+import org.apache.iotdb.db.exception.query.LogicalOperatorException;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.logical.crud.GroupByQueryOperator;
+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.SelectIntoOperator;
+
+public class PhysicalPlanValidationHandler {
+
+  public static void checkRestQuery(Operator operator) throws LogicalOperatorException {
+    if (operator instanceof SelectIntoOperator) {
+      throw new LogicalOperatorException("select into clauses are not supported.");
+    }
+
+    if (!(operator instanceof QueryOperator)) {
+      return;
+    }
+    QueryOperator queryOperator = (QueryOperator) operator;
+
+    if (queryOperator.isAlignByDevice()) {
+      throw new LogicalOperatorException("align by device clauses are not supported.");
+    }
+
+    // disable align
+    if (!queryOperator.isAlignByTime()) {
+      throw new LogicalOperatorException("disable align clauses are not supported.");
+    }
+  }
+
+  public static void checkGrafanaQuery(Operator operator) throws LogicalOperatorException {
+    if (!(operator instanceof QueryOperator)) {
+      return;
+    }
+    QueryOperator queryOperator = (QueryOperator) operator;
+
+    if (queryOperator.isAlignByDevice()) {
+      throw new LogicalOperatorException("align by device clauses are not supported.");
+    }
+
+    // disable align
+    if (!queryOperator.isAlignByTime()) {
+      throw new LogicalOperatorException("disable align clauses are not supported.");
+    }
+
+    if (queryOperator instanceof LastQueryOperator) {
+      throw new LogicalOperatorException("last clauses are not supported.");
+    }
+
+    if (!(queryOperator instanceof GroupByQueryOperator) && queryOperator.isGroupByLevel()) {
+      throw new LogicalOperatorException(
+          "group by level without time interval clauses are not supported.");
+    }
+
+    if (queryOperator.getSpecialClauseComponent() != null
+        && !queryOperator.getSpecialClauseComponent().isAscending()) {
+      throw new LogicalOperatorException("order by time desc clauses are not supported.");
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/protocol/rest/handler/RequestValidationHandler.java b/server/src/main/java/org/apache/iotdb/db/protocol/rest/handler/RequestValidationHandler.java
index 2a98d31..4d04699 100644
--- a/server/src/main/java/org/apache/iotdb/db/protocol/rest/handler/RequestValidationHandler.java
+++ b/server/src/main/java/org/apache/iotdb/db/protocol/rest/handler/RequestValidationHandler.java
@@ -21,6 +21,8 @@ import org.apache.iotdb.db.protocol.rest.model.ExpressionRequest;
 import org.apache.iotdb.db.protocol.rest.model.InsertTabletRequest;
 import org.apache.iotdb.db.protocol.rest.model.SQL;
 
+import org.apache.commons.lang3.Validate;
+
 import java.util.Objects;
 
 public class RequestValidationHandler {
@@ -29,6 +31,9 @@ public class RequestValidationHandler {
 
   public static void validateSQL(SQL sql) {
     Objects.requireNonNull(sql.getSql(), "sql should not be null");
+    Validate.isTrue(
+        sql.getRowLimit() == null || sql.getRowLimit() <= 0,
+        "rowLimit can not be null and should be positive");
   }
 
   public static void validateInsertTabletRequest(InsertTabletRequest insertTabletRequest) {
diff --git a/server/src/main/java/org/apache/iotdb/db/protocol/rest/impl/RestApiServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/protocol/rest/impl/RestApiServiceImpl.java
index 0549668..1308d2a 100644
--- a/server/src/main/java/org/apache/iotdb/db/protocol/rest/impl/RestApiServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/protocol/rest/impl/RestApiServiceImpl.java
@@ -43,6 +43,8 @@ import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.SecurityContext;
 
+import java.time.ZoneId;
+
 public class RestApiServiceImpl extends RestApiService {
 
   public static ServiceProvider serviceProvider = IoTDB.serviceProvider;
@@ -91,7 +93,8 @@ public class RestApiServiceImpl extends RestApiService {
     try {
       RequestValidationHandler.validateSQL(sql);
 
-      PhysicalPlan physicalPlan = planner.parseSQLToPhysicalPlan(sql.getSql());
+      PhysicalPlan physicalPlan =
+          planner.parseSQLToRestQueryPlan(sql.getSql(), ZoneId.systemDefault());
       if (!(physicalPlan instanceof QueryPlan)) {
         return Response.ok()
             .entity(
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
index 02258df..1cfa66a 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
@@ -23,6 +23,7 @@ import org.apache.iotdb.db.exception.query.LogicalOperatorException;
 import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
 import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.protocol.rest.handler.PhysicalPlanValidationHandler;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.db.qp.logical.crud.FilterOperator;
 import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
@@ -150,4 +151,32 @@ public class Planner {
     // from logical operator to physical plan
     return new PhysicalGenerator().transformToPhysicalPlan(operator);
   }
+
+  public PhysicalPlan parseSQLToRestQueryPlan(String sqlStr, ZoneId zoneId)
+      throws QueryProcessException {
+    // from SQL to logical operator
+    Operator operator = LogicalGenerator.generate(sqlStr, zoneId);
+    // check if there are logical errors
+    LogicalChecker.check(operator);
+    // extra check for rest query
+    PhysicalPlanValidationHandler.checkRestQuery(operator);
+    // optimize the logical operator
+    operator = logicalOptimize(operator);
+    // from logical operator to physical plan
+    return new PhysicalGenerator().transformToPhysicalPlan(operator);
+  }
+
+  public PhysicalPlan parseSQLToGrafanaQueryPlan(String sqlStr, ZoneId zoneId)
+      throws QueryProcessException {
+    // from SQL to logical operator
+    Operator operator = LogicalGenerator.generate(sqlStr, zoneId);
+    // check if there are logical errors
+    LogicalChecker.check(operator);
+    // extra check for grafana query
+    PhysicalPlanValidationHandler.checkGrafanaQuery(operator);
+    // optimize the logical operator
+    operator = logicalOptimize(operator);
+    // from logical operator to physical plan
+    return new PhysicalGenerator().transformToPhysicalPlan(operator);
+  }
 }