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/04/22 06:02:56 UTC

[incubator-iotdb] 01/01: add limit&slimit to show timeseries

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

qiaojialin pushed a commit to branch show_limit
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git

commit 9006c520314891294927a539485fa6c433393782
Author: qiaojialin <64...@qq.com>
AuthorDate: Wed Apr 22 14:02:35 2020 +0800

    add limit&slimit to show timeseries
---
 .../org/apache/iotdb/db/qp/strategy/SqlBase.g4     |  4 +-
 .../org/apache/iotdb/db/metadata/MManager.java     | 43 ++++++++++++++--------
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  2 +-
 .../db/qp/logical/sys/ShowTimeSeriesOperator.java  | 18 +++++++++
 .../db/qp/physical/sys/ShowTimeSeriesPlan.java     | 22 +++++++++--
 .../iotdb/db/qp/strategy/LogicalGenerator.java     | 22 +++++++----
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |  5 ++-
 .../iotdb/db/integration/IoTDBSimpleQueryIT.java   | 25 +++++++++++++
 .../iotdb/db/qp/plan/LogicalPlanSmallTest.java     |  8 ++--
 .../apache/iotdb/db/qp/plan/PhysicalPlanTest.java  | 11 ++++++
 10 files changed, 127 insertions(+), 33 deletions(-)

diff --git a/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4 b/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
index 34fbee9..a505f87 100644
--- a/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
+++ b/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
@@ -65,7 +65,7 @@ statement
     | SHOW FLUSH TASK INFO #showFlushTaskInfo
     | SHOW DYNAMIC PARAMETER #showDynamicParameter
     | SHOW VERSION #showVersion
-    | SHOW TIMESERIES prefixPath? showWhereClause? #showTimeseries
+    | SHOW TIMESERIES prefixPath? showWhereClause? limitClause? #showTimeseries
     | SHOW STORAGE GROUP #showStorageGroup
     | SHOW CHILD PATHS prefixPath? #showChildPaths
     | SHOW DEVICES prefixPath? #showDevices
@@ -189,6 +189,7 @@ specialLimit
 
 limitClause
     : LIMIT INT offsetClause?
+    | offsetClause? LIMIT INT
     ;
 
 offsetClause
@@ -197,6 +198,7 @@ offsetClause
 
 slimitClause
     : SLIMIT INT soffsetClause?
+    | soffsetClause? SLIMIT INT
     ;
 
 soffsetClause
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index 110e3d2..8cdc136 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@ -738,29 +738,42 @@ public class MManager {
   }
 
   /**
-   * Get all timeseries paths under the given path.
+   * Get the result of ShowTimeseriesPlan
    *
-   * @param path can be root, root.*  root.*.*.a etc.. if the wildcard is not at the tail, then each
-   * wildcard can only match one level, otherwise it can match to the tail.
+   * @param plan show time series query plan
    */
-  public List<ShowTimeSeriesResult> getAllTimeseriesSchema(String path) throws MetadataException {
+  public List<ShowTimeSeriesResult> showTimeseries(ShowTimeSeriesPlan plan) throws MetadataException {
     lock.readLock().lock();
     try {
-      List<String[]> ans = mtree.getAllMeasurementSchema(path);
+      List<String[]> ans = mtree.getAllMeasurementSchema(plan.getPath().getFullPath());
+      int count = 0;
+      int offset = plan.getOffset();
       List<ShowTimeSeriesResult> res = new LinkedList<>();
-      for (String[] ansString : ans) {
-        long offset = Long.parseLong(ansString[6]);
+      for (int i = 0; i < ans.size(); i++) {
+        if (i < offset) {
+          continue;
+        }
+
+        String[] ansString = ans.get(i);
+        if (count >= plan.getLimit()) {
+          return res;
+        }
+
+        long tagFileOffset = Long.parseLong(ansString[6]);
         try {
-          if (offset < 0) {
+          if (tagFileOffset < 0) {
+            // no tags/attributes
+            res.add(new ShowTimeSeriesResult(ansString[0], ansString[1], ansString[2],
+                ansString[3], ansString[4], ansString[5], Collections.emptyMap()));
+          } else {
+            // has tags/attributes
+            Pair<Map<String, String>, Map<String, String>> pair =
+                tagLogFile.read(config.getTagAttributeTotalSize(), tagFileOffset);
+            pair.left.putAll(pair.right);
             res.add(new ShowTimeSeriesResult(ansString[0], ansString[1], ansString[2],
-                    ansString[3], ansString[4], ansString[5], Collections.emptyMap()));
-            continue;
+                ansString[3], ansString[4], ansString[5], pair.left));
           }
-          Pair<Map<String, String>, Map<String, String>> pair =
-                  tagLogFile.read(config.getTagAttributeTotalSize(), offset);
-          pair.left.putAll(pair.right);
-          res.add(new ShowTimeSeriesResult(ansString[0], ansString[1], ansString[2],
-                  ansString[3], ansString[4], ansString[5], pair.left));
+          count ++;
         } catch (IOException e) {
           throw new MetadataException(
               "Something went wrong while deserialize tag info of " + ansString[0], e);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
index fb3c57d..551cf5a 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
@@ -367,7 +367,7 @@ public class PlanExecutor implements IPlanExecutor {
   private QueryDataSet processShowTimeseries(ShowTimeSeriesPlan timeSeriesPlan)
       throws MetadataException {
     List<ShowTimeSeriesResult> timeseriesList = MManager.getInstance()
-        .getAllTimeseriesSchema(timeSeriesPlan.getPath().toString());
+        .showTimeseries(timeSeriesPlan);
     return getQueryDataSet(timeseriesList);
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowTimeSeriesOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowTimeSeriesOperator.java
index be6083b..3701941 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowTimeSeriesOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowTimeSeriesOperator.java
@@ -27,6 +27,8 @@ public class ShowTimeSeriesOperator extends ShowOperator {
   private boolean isContains;
   private String key;
   private String value;
+  private int limit = Integer.MAX_VALUE;
+  private int offset = 0;
 
   public ShowTimeSeriesOperator(int tokeIntType, Path path) {
     super(tokeIntType);
@@ -60,4 +62,20 @@ public class ShowTimeSeriesOperator extends ShowOperator {
   public void setValue(String value) {
     this.value = value;
   }
+
+  public int getLimit() {
+    return limit;
+  }
+
+  public void setLimit(int limit) {
+    this.limit = limit;
+  }
+
+  public int getOffset() {
+    return offset;
+  }
+
+  public void setOffset(int offset) {
+    this.offset = offset;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowTimeSeriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowTimeSeriesPlan.java
index 3610a61..48a6491 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowTimeSeriesPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowTimeSeriesPlan.java
@@ -20,18 +20,26 @@ package org.apache.iotdb.db.qp.physical.sys;
 
 import org.apache.iotdb.tsfile.read.common.Path;
 
-public class ShowTimeSeriesPlan extends ShowPlan{
+public class ShowTimeSeriesPlan extends ShowPlan {
+
+  // path can be root, root.*  root.*.*.a etc.. if the wildcard is not at the tail, then each
+  // * wildcard can only match one level, otherwise it can match to the tail.
   private Path path;
   private boolean isContains;
   private String key;
   private String value;
+  private int limit;
+  private int offset;
 
-  public ShowTimeSeriesPlan(ShowContentType showContentType, Path path, boolean isContains, String key, String value) {
+  public ShowTimeSeriesPlan(ShowContentType showContentType, Path path, boolean isContains,
+      String key, String value, int limit, int offset) {
     super(showContentType);
     this.path = path;
     this.isContains = isContains;
     this.key = key;
     this.value = value;
+    this.limit = limit;
+    this.offset = offset;
   }
 
   public Path getPath() {
@@ -49,4 +57,12 @@ public class ShowTimeSeriesPlan extends ShowPlan{
   public String getValue() {
     return value;
   }
-}
+
+  public int getLimit() {
+    return limit;
+  }
+
+  public int getOffset() {
+    return offset;
+  }
+}
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
index 330345a..3769fcb 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
@@ -738,7 +738,11 @@ public class LogicalGenerator extends SqlBaseBaseListener {
     if (limit <= 0) {
       throw new SQLParserException("LIMIT <N>: N should be greater than 0.");
     }
-    queryOp.setRowLimit(limit);
+    if (initializedOperator instanceof ShowTimeSeriesOperator) {
+      ((ShowTimeSeriesOperator) initializedOperator).setLimit(limit);
+    } else {
+      queryOp.setRowLimit(limit);
+    }
   }
 
   @Override
@@ -751,10 +755,14 @@ public class LogicalGenerator extends SqlBaseBaseListener {
       throw new SQLParserException(
           "Out of range. OFFSET <OFFSETValue>: OFFSETValue should be Int32.");
     }
-    if (offset <= 0) {
-      throw new SQLParserException("OFFSET <OFFSETValue>: OFFSETValue should be greater than 0.");
+    if (offset < 0) {
+      throw new SQLParserException("OFFSET <OFFSETValue>: OFFSETValue should >= 0.");
+    }
+    if (initializedOperator instanceof ShowTimeSeriesOperator) {
+      ((ShowTimeSeriesOperator) initializedOperator).setOffset(offset);
+    } else {
+      queryOp.setRowOffset(offset);
     }
-    queryOp.setRowOffset(offset);
   }
 
   @Override
@@ -783,9 +791,9 @@ public class LogicalGenerator extends SqlBaseBaseListener {
       throw new SQLParserException(
           "Out of range. SOFFSET <SOFFSETValue>: SOFFSETValue should be Int32.");
     }
-    if (soffset <= 0) {
+    if (soffset < 0) {
       throw new SQLParserException(
-          "SOFFSET <SOFFSETValue>: SOFFSETValue should be greater than 0.");
+          "SOFFSET <SOFFSETValue>: SOFFSETValue should >= 0.");
     }
     queryOp.setSeriesOffset(soffset);
   }
@@ -1067,7 +1075,7 @@ public class LogicalGenerator extends SqlBaseBaseListener {
   }
 
   @Override
-  public void enterShowWhereClause(SqlBaseParser.ShowWhereClauseContext ctx) {
+  public void enterShowWhereClause(ShowWhereClauseContext ctx) {
     super.enterShowWhereClause(ctx);
 
     ShowTimeSeriesOperator operator = (ShowTimeSeriesOperator) initializedOperator;
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
index fbb5b8f..5cfc547 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
@@ -127,8 +127,9 @@ public class PhysicalGenerator {
           case SQLConstant.TOK_TIMESERIES:
             ShowTimeSeriesOperator showTimeSeriesOperator = (ShowTimeSeriesOperator) operator;
             return new ShowTimeSeriesPlan(ShowContentType.TIMESERIES,
-                    showTimeSeriesOperator.getPath(), showTimeSeriesOperator.isContains(),
-                    showTimeSeriesOperator.getKey(), showTimeSeriesOperator.getValue());
+                showTimeSeriesOperator.getPath(), showTimeSeriesOperator.isContains(),
+                showTimeSeriesOperator.getKey(), showTimeSeriesOperator.getValue(),
+                showTimeSeriesOperator.getLimit(), showTimeSeriesOperator.getOffset());
           case SQLConstant.TOK_STORAGE_GROUP:
             return new ShowPlan(ShowContentType.STORAGE_GROUP);
           case SQLConstant.TOK_DEVICES:
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java
index 0425390..28e6727 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSimpleQueryIT.java
@@ -102,6 +102,31 @@ public class IoTDBSimpleQueryIT {
 
 
   @Test
+  public void testShowTimeseriesWithLimitOffset() throws SQLException, ClassNotFoundException {
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    try(Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()){
+
+      String[] exps = new String[]{"root.sg1.d0.s2", "root.sg1.d0.s3"};
+
+      statement.execute("INSERT INTO root.sg1.d0(timestamp, s1) VALUES (5, 5)");
+      statement.execute("INSERT INTO root.sg1.d0(timestamp, s2) VALUES (5, 5)");
+      statement.execute("INSERT INTO root.sg1.d0(timestamp, s3) VALUES (5, 5)");
+
+      ResultSet resultSet = statement.executeQuery("show timeseries limit 2 offset 1");
+
+      int count = 0;
+
+      while(resultSet.next()) {
+        Assert.assertEquals(exps[count++], resultSet.getString(1));
+      }
+
+      resultSet.close();
+    }
+  }
+
+  @Test
   public void testFirstOverlappedPageFiltered() throws SQLException, ClassNotFoundException {
     Class.forName(Config.JDBC_DRIVER_NAME);
     try(Connection connection = DriverManager
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/plan/LogicalPlanSmallTest.java b/server/src/test/java/org/apache/iotdb/db/qp/plan/LogicalPlanSmallTest.java
index fb6e582..916594a 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/plan/LogicalPlanSmallTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/plan/LogicalPlanSmallTest.java
@@ -132,10 +132,10 @@ public class LogicalPlanSmallTest {
 
   @Test(expected = SQLParserException.class)
   public void testOffsetNotPositive() {
-    String sqlStr = "select * from root.vehicle.d1 where s1 < 20 and time <= now() limit 1 offset 0";
+    String sqlStr = "select * from root.vehicle.d1 where s1 < 20 and time <= now() limit 1 offset -1";
     RootOperator operator = (RootOperator) parseDriver
         .parse(sqlStr, IoTDBDescriptor.getInstance().getConfig().getZoneID());
-    // expected to throw SQLParserException: OFFSET <OFFSETValue>: OFFSETValue should be greater than 0.
+    // expected to throw SQLParserException: OFFSET <OFFSETValue>: OFFSETValue should >= 0.
   }
 
   @Test(expected = SQLParserException.class)
@@ -165,10 +165,10 @@ public class LogicalPlanSmallTest {
 
   @Test(expected = SQLParserException.class)
   public void testSoffsetNotPositive() {
-    String sqlStr = "select * from root.vehicle.d1 where s1 < 20 and time <= now() slimit 1 soffset 0";
+    String sqlStr = "select * from root.vehicle.d1 where s1 < 20 and time <= now() slimit 1 soffset -1";
     RootOperator operator = (RootOperator) parseDriver
         .parse(sqlStr, IoTDBDescriptor.getInstance().getConfig().getZoneID());
-    // expected to throw SQLParserException: SOFFSET <SOFFSETValue>: SOFFSETValue should be greater than 0.
+    // expected to throw SQLParserException: SOFFSET <SOFFSETValue>: SOFFSETValue should >= 0.
   }
 
   @Test(expected = LogicalOptimizeException.class)
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
index 78acafd..ac4fb6b 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
@@ -465,6 +465,17 @@ public class PhysicalPlanTest {
   }
 
   @Test
+  public void testOffsetLimit() throws QueryProcessException {
+    String sqlStr = "SELECT s1 FROM root.vehicle.d1,root.vehicle.d2 WHERE time < 10 "
+        + "offset 10 limit 100 soffset 1 slimit 1";
+    QueryPlan plan = (QueryPlan) processor.parseSQLToPhysicalPlan(sqlStr);
+    assertEquals(100, plan.getRowLimit());
+    assertEquals(10, plan.getRowOffset());
+    // NOTE that the parameters of the SLIMIT clause is not stored in the physicalPlan,
+    // because the SLIMIT clause takes effect before the physicalPlan is finally generated.
+  }
+
+  @Test
   public void testQueryFloat1() throws QueryProcessException {
     String sqlStr = "SELECT s1 FROM root.vehicle.d1 WHERE s1 > 20.5e3";
     PhysicalPlan plan = processor.parseSQLToPhysicalPlan(sqlStr);