You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2020/11/02 14:12:26 UTC

[iotdb] branch master updated: Add restrictions for LAST query filter (#1917)

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

qiaojialin 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 0cf358c  Add restrictions for LAST query filter (#1917)
0cf358c is described below

commit 0cf358c15fef85dcd4d39d8285dcf5d5abbcf4a2
Author: wshao08 <59...@users.noreply.github.com>
AuthorDate: Mon Nov 2 22:12:06 2020 +0800

    Add restrictions for LAST query filter (#1917)
---
 .../DML Data Manipulation Language.md              |  6 +-
 docs/UserGuide/Operation Manual/SQL Reference.md   |  7 +--
 .../DML Data Manipulation Language.md              |  6 +-
 .../zh/UserGuide/Operation Manual/SQL Reference.md |  7 +--
 .../java/org/apache/iotdb/db/metadata/MTree.java   |  2 +-
 .../iotdb/db/qp/physical/crud/LastQueryPlan.java   | 21 ++++++-
 .../db/qp/physical/crud/RawDataQueryPlan.java      |  3 +-
 .../query/dataset/groupby/GroupByFillDataSet.java  |  2 +-
 .../iotdb/db/query/executor/LastQueryExecutor.java | 10 +++-
 .../apache/iotdb/db/integration/IoTDBLastIT.java   | 67 +---------------------
 10 files changed, 44 insertions(+), 87 deletions(-)

diff --git a/docs/UserGuide/Operation Manual/DML Data Manipulation Language.md b/docs/UserGuide/Operation Manual/DML Data Manipulation Language.md
index ffc2751..b6aa4a8 100644
--- a/docs/UserGuide/Operation Manual/DML Data Manipulation Language.md	
+++ b/docs/UserGuide/Operation Manual/DML Data Manipulation Language.md	
@@ -398,7 +398,7 @@ select last <Path> [COMMA <Path>]* from < PrefixPath > [COMMA < PrefixPath >]* <
 
 which means: Query and return the last data points of timeseries prefixPath.path.
 
-Only time filter is supported in \<WhereClause\>. Any value filters given in the \<WhereClause\> take no effects.
+Only time filter with '>' or '>=' is supported in \<WhereClause\>. Any other filters given in the \<WhereClause\> will give an exception.
 
 The result will be returned in a three column table format.
 
@@ -417,10 +417,10 @@ Example 1: get the last point of root.ln.wf01.wt01.speed:
 ```
 
 Example 2: get the last speed, status and temperature points of root.ln.wf01.wt01,
-whose timestamp in range [5, 10).
+whose timestamp larger or equal to 5.
 
 ```
-> select last speed, status, temperature from root.ln.wf01.wt01 where time >= 5 and time < 10
+> select last speed, status, temperature from root.ln.wf01.wt01 where time >= 5
 
 | Time | Path                         | Value |
 | ---  | ---------------------------- | ----- |
diff --git a/docs/UserGuide/Operation Manual/SQL Reference.md b/docs/UserGuide/Operation Manual/SQL Reference.md
index 943910a..cb143ba 100644
--- a/docs/UserGuide/Operation Manual/SQL Reference.md	
+++ b/docs/UserGuide/Operation Manual/SQL Reference.md	
@@ -609,14 +609,13 @@ TimeExpr : TIME PrecedenceEqualOperator (<TimeValue> | <RelativeTime>)
 Eg. SELECT LAST s1 FROM root.sg.d1
 Eg. SELECT LAST s1, s2 FROM root.sg.d1
 Eg. SELECT LAST s1 FROM root.sg.d1, root.sg.d2
-Eg. SELECT LAST s1 FROM root.sg.d1 where time < 100
-Eg. SELECT LAST s1, s2 FROM root.sg.d1 where time > 100 and time <= 500
-Eg. SELECT LAST s1, s2 FROM root.sg.d1 where time < 300 or time > 1000
+Eg. SELECT LAST s1 FROM root.sg.d1 where time > 100
+Eg. SELECT LAST s1, s2 FROM root.sg.d1 where time >= 500
 
 Rules:
 1. the statement needs to satisfy this constraint: <PrefixPath> + <Path> = <Timeseries>
 
-2. SELECT LAST does not support value filter.
+2. SELECT LAST only supports time filter that contains '>' or '>=' currently.
 
 3. The result set of last query will always be displayed in a fixed three column table format.
 For example, "select last s1, s2 from root.sg.d1, root.sg.d2", the query result would be:
diff --git a/docs/zh/UserGuide/Operation Manual/DML Data Manipulation Language.md b/docs/zh/UserGuide/Operation Manual/DML Data Manipulation Language.md
index fa2c6df..bdd32a9 100644
--- a/docs/zh/UserGuide/Operation Manual/DML Data Manipulation Language.md	
+++ b/docs/zh/UserGuide/Operation Manual/DML Data Manipulation Language.md	
@@ -423,7 +423,7 @@ select last <Path> [COMMA <Path>]* from < PrefixPath > [COMMA < PrefixPath >]* <
 
 其含义是:查询时间序列prefixPath.path中最近时间戳的数据
 
-\<WhereClause\>中当前只支持时间过滤条件,任何非时间过滤都将不会起任何作用。
+\<WhereClause\>中当前只支持含有'>'或'>='的时间过滤条件,任何其他过滤条件都将会返回异常。
 
 结果集为三列的结构
 
@@ -441,10 +441,10 @@ select last <Path> [COMMA <Path>]* from < PrefixPath > [COMMA < PrefixPath >]* <
 |  5   | root.ln.wf01.wt01.speed | 100   |
 ```
 
-示例 2:查询 root.ln.wf01.wt01 下 speed,status,temperature 的在时间区间[5, 10)之内的最新数据点
+示例 2:查询 root.ln.wf01.wt01 下 speed,status,temperature 时间戳大于等于5的最新数据点。
 
 ```
-> select last speed, status, temperature from root.ln.wf01  where time >= 5 and time < 10
+> select last speed, status, temperature from root.ln.wf01  where time >= 5
 
 | Time | Path                         | Value |
 | ---  | ---------------------------- | ----- |
diff --git a/docs/zh/UserGuide/Operation Manual/SQL Reference.md b/docs/zh/UserGuide/Operation Manual/SQL Reference.md
index 657b1a6..3b7cec2 100644
--- a/docs/zh/UserGuide/Operation Manual/SQL Reference.md	
+++ b/docs/zh/UserGuide/Operation Manual/SQL Reference.md	
@@ -596,14 +596,13 @@ TimeExpr : TIME PrecedenceEqualOperator (<TimeValue> | <RelativeTime>)
 Eg. SELECT LAST s1 FROM root.sg.d1
 Eg. SELECT LAST s1, s2 FROM root.sg.d1
 Eg. SELECT LAST s1 FROM root.sg.d1, root.sg.d2
-Eg. SELECT LAST s1 FROM root.sg.d1 where time < 100
-Eg. SELECT LAST s1, s2 FROM root.sg.d1 where time > 100 and time <= 500
-Eg. SELECT LAST s1, s2 FROM root.sg.d1 where time < 300 or time > 1000
+Eg. SELECT LAST s1 FROM root.sg.d1 where time > 100
+Eg. SELECT LAST s1, s2 FROM root.sg.d1 where time >= 500
 
 规则:
 1. 需要满足PrefixPath.Path 为一条完整的时间序列,即 <PrefixPath> + <Path> = <Timeseries>
 
-2. SELECT LAST 语句不支持值过滤条件.
+2. 当前SELECT LAST 语句只支持包含'>'或'>='的时间过滤条件
 
 3. 结果集以三列的表格的固定形式返回。
 例如 "select last s1, s2 from root.sg.d1, root.sg.d2", 结果集返回如下:
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
index 64ee0eb..8ef7219 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
@@ -109,7 +109,7 @@ public class MTree implements Serializable {
       try {
         last = calculateLastPairForOneSeriesLocally(node.getPartialPath(),
             node.getSchema().getType(), queryContext, null, Collections.emptySet());
-        return last.getTimestamp();
+        return (last != null ? last.getTimestamp() : Long.MIN_VALUE);
       } catch (Exception e) {
         logger.error("Something wrong happened while trying to get last time value pair of {}",
             node.getFullPath(), e);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/LastQueryPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/LastQueryPlan.java
index 7b3f7cb..8b7475f 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/LastQueryPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/LastQueryPlan.java
@@ -19,9 +19,13 @@
 
 package org.apache.iotdb.db.qp.physical.crud;
 
+import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.tsfile.read.expression.IExpression;
 import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
+import org.apache.iotdb.tsfile.read.filter.TimeFilter.TimeGt;
+import org.apache.iotdb.tsfile.read.filter.TimeFilter.TimeGtEq;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 
 public class LastQueryPlan extends RawDataQueryPlan {
 
@@ -30,9 +34,22 @@ public class LastQueryPlan extends RawDataQueryPlan {
     setOperatorType(Operator.OperatorType.LAST);
   }
 
-  public void setExpression(IExpression expression) {
-    if (expression instanceof GlobalTimeExpression) {
+  public void setExpression(IExpression expression) throws QueryProcessException {
+    if (isValidExpression(expression)) {
       super.setExpression(expression);
+    } else {
+      throw new QueryProcessException("Only \'>\' and \'>=\' are supported in LAST query");
+    }
+  }
+
+  // Only > and >= are supported in time filter
+  private boolean isValidExpression(IExpression expression) {
+    if (expression instanceof GlobalTimeExpression) {
+      Filter filter = ((GlobalTimeExpression) expression).getFilter();
+      if (filter instanceof TimeGtEq || filter instanceof TimeGt) {
+        return true;
+      }
     }
+    return false;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/RawDataQueryPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/RawDataQueryPlan.java
index acecbe4..c1a3dd9 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/RawDataQueryPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/RawDataQueryPlan.java
@@ -25,6 +25,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -50,7 +51,7 @@ public class RawDataQueryPlan extends QueryPlan {
     return expression;
   }
 
-  public void setExpression(IExpression expression) {
+  public void setExpression(IExpression expression) throws QueryProcessException {
     this.expression = expression;
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByFillDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByFillDataSet.java
index 4144b7b..d953033 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByFillDataSet.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByFillDataSet.java
@@ -102,7 +102,7 @@ public class GroupByFillDataSet extends QueryDataSet {
       lastTimeValuePair = LastQueryExecutor.calculateLastPairForOneSeriesLocally(
           (PartialPath) paths.get(i), dataTypes.get(i), context, null,
           groupByFillPlan.getAllMeasurementsInDevice(paths.get(i).getDevice()));
-      if (lastTimeValuePair.getValue() != null) {
+      if (lastTimeValuePair != null && lastTimeValuePair.getValue() != null) {
         lastTimeArray[i] = lastTimeValuePair.getTimestamp();
       }
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/LastQueryExecutor.java b/server/src/main/java/org/apache/iotdb/db/query/executor/LastQueryExecutor.java
index 1231d08..43db2dc 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/executor/LastQueryExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/LastQueryExecutor.java
@@ -86,7 +86,7 @@ public class LastQueryExecutor {
       lastTimeValuePair = calculateLastPairForOneSeries(
               selectedSeries.get(i), dataTypes.get(i), context,
               lastQueryPlan.getAllMeasurementsInDevice(selectedSeries.get(i).getDevice()));
-      if (lastTimeValuePair.getValue() != null) {
+      if (lastTimeValuePair != null && lastTimeValuePair.getValue() != null) {
         RowRecord resultRecord = new RowRecord(lastTimeValuePair.getTimestamp());
         Field pathField = new Field(TSDataType.TEXT);
         if (selectedSeries.get(i).getTsAlias() != null) {
@@ -125,7 +125,7 @@ public class LastQueryExecutor {
    * get last result for one series
    *
    * @param context query context
-   * @return TimeValuePair
+   * @return TimeValuePair, result can be null
    */
   public static TimeValuePair calculateLastPairForOneSeriesLocally(
       PartialPath seriesPath, TSDataType tsDataType, QueryContext context,
@@ -145,6 +145,8 @@ public class LastQueryExecutor {
         TimeValuePair timeValuePair = IoTDB.metaManager.getLastCache(seriesPath);
         if (timeValuePair != null && satisfyFilter(filter, timeValuePair)) {
           return timeValuePair;
+        } else if (timeValuePair != null) {
+          return null;
         }
       }
 
@@ -152,6 +154,8 @@ public class LastQueryExecutor {
         TimeValuePair timeValuePair =  node.getCachedLast();
         if (timeValuePair != null && satisfyFilter(filter, timeValuePair)) {
           return timeValuePair;
+        } else if (timeValuePair != null) {
+          return null;
         }
       }
     }
@@ -173,7 +177,7 @@ public class LastQueryExecutor {
     TimeValuePair resultPair = lastReader.readLastPoint();
 
     // Update cached last value with low priority unless "FROM" expression exists
-    if (lastCacheEnabled && filter == null) {
+    if (lastCacheEnabled) {
       IoTDB.metaManager.updateLastCache(
           seriesPath, resultPair, false, Long.MIN_VALUE, node);
     }
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLastIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLastIT.java
index 64f886e..bfb073d 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLastIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLastIT.java
@@ -447,73 +447,10 @@ public class IoTDBLastIT {
   }
 
   @Test
-  public void lastWithTimeFilterTest() throws SQLException, MetadataException {
-    String[] retArray =
-        new String[]{
-            "200,root.ln.wf01.wt01.temperature,25.2",
-            "500,root.ln.wf01.wt01.temperature,22.1",
-            "300,root.ln.wf01.wt01.temperature,15.7",
-        };
-
-    try (Connection connection =
-             DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
-         Statement statement = connection.createStatement()) {
-
-      boolean hasResultSet = statement.execute(
-          "select last temperature from root.ln.wf01.wt01 where time < 300");
-
-      assertTrue(hasResultSet);
-      int cnt = 0;
-      try (ResultSet resultSet = statement.getResultSet()) {
-        while (resultSet.next()) {
-          String ans =
-              resultSet.getString(TIMESTAMP_STR) + ","
-                  + resultSet.getString(TIMESEIRES_STR) + ","
-                  + resultSet.getString(VALUE_STR);
-          Assert.assertEquals(retArray[cnt], ans);
-          cnt++;
-        }
-      }
-
-      statement.execute(
-          "select last temperature from root.ln.wf01.wt01 where time >= 300 and time < 1000");
-      try (ResultSet resultSet = statement.getResultSet()) {
-        while (resultSet.next()) {
-          String ans =
-              resultSet.getString(TIMESTAMP_STR) + ","
-                  + resultSet.getString(TIMESEIRES_STR) + ","
-                  + resultSet.getString(VALUE_STR);
-          Assert.assertEquals(retArray[cnt], ans);
-          cnt++;
-        }
-      }
-
-      statement.execute(
-          "select last temperature from root.ln.wf01.wt01 where time <= 300 or time > 1000");
-      try (ResultSet resultSet = statement.getResultSet()) {
-        while (resultSet.next()) {
-          String ans =
-              resultSet.getString(TIMESTAMP_STR) + ","
-                  + resultSet.getString(TIMESEIRES_STR) + ","
-                  + resultSet.getString(VALUE_STR);
-          Assert.assertEquals(retArray[cnt], ans);
-          cnt++;
-        }
-      }
-      statement.execute(
-          "select last temperature from root.ln.wf01.wt01 where time > 1000");
-      try (ResultSet resultSet = statement.getResultSet()) {
-        Assert.assertFalse(resultSet.next());
-      }
-    }
-  }
-
-  @Test
   public void lastCacheWithFilterTest() throws SQLException, MetadataException {
     String[] retArray =
         new String[]{
             "500,root.ln.wf01.wt01.temperature,22.1",
-            "300,root.ln.wf01.wt01.temperature,15.7",
         };
 
     try (Connection connection =
@@ -522,7 +459,7 @@ public class IoTDBLastIT {
 
       statement.execute("select last temperature from root.ln.wf01.wt01");
       statement.execute(
-          "select last temperature from root.ln.wf01.wt01 where time >= 300 and time < 1000");
+          "select last temperature from root.ln.wf01.wt01 where time >= 300");
       int cnt = 0;
       try (ResultSet resultSet = statement.getResultSet()) {
         while (resultSet.next()) {
@@ -536,7 +473,7 @@ public class IoTDBLastIT {
       }
 
       statement.execute(
-          "select last temperature from root.ln.wf01.wt01 where time <= 300");
+          "select last temperature from root.ln.wf01.wt01 where time > 600");
       try (ResultSet resultSet = statement.getResultSet()) {
         while (resultSet.next()) {
           String ans =