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 12:17:36 UTC

[incubator-iotdb] branch master updated: [IOTDB-612] add limit&offset to show timeseries (#1085)

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/incubator-iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new ebc3f23  [IOTDB-612] add limit&offset to show timeseries (#1085)
ebc3f23 is described below

commit ebc3f2333c89bd34872b62ca12cdc8b263c61484
Author: Jialin Qiao <qj...@mails.tsinghua.edu.cn>
AuthorDate: Wed Apr 22 20:17:28 2020 +0800

    [IOTDB-612] add limit&offset to show timeseries (#1085)
    
    * add limit&offset to show timeseries
---
 .../org/apache/iotdb/db/qp/strategy/SqlBase.g4     |  4 +-
 .../org/apache/iotdb/db/metadata/MManager.java     | 59 ++++++++++++++++------
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  8 +--
 .../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 +++++++++
 .../apache/iotdb/db/integration/IoTDBTagIT.java    | 41 +++++++++++++++
 .../iotdb/db/qp/plan/LogicalPlanSmallTest.java     | 15 +++---
 .../apache/iotdb/db/qp/plan/PhysicalPlanTest.java  | 11 ++++
 11 files changed, 191 insertions(+), 39 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 18f74bf..4de9b2b 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..a18f3d5 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
@@ -680,7 +680,7 @@ public class MManager {
         throw new MetadataException("The key " + plan.getKey() + " is not a tag.");
       }
       Map<String, Set<LeafMNode>> value2Node = tagIndex.get(plan.getKey());
-      Set<LeafMNode> allMatchedNodes = new HashSet<>();
+      Set<LeafMNode> allMatchedNodes = new TreeSet<>(Comparator.comparing(MNode::getFullPath));
       if (plan.isContains()) {
         for (Entry<String, Set<LeafMNode>> entry : value2Node.entrySet()) {
           String tagValue = entry.getKey();
@@ -698,8 +698,16 @@ public class MManager {
       }
       List<ShowTimeSeriesResult> res = new LinkedList<>();
       String[] prefixNodes = MetaUtils.getNodeNames(plan.getPath().getFullPath());
+      int curOffset = -1;
+      int count = 0;
+      int limit = plan.getLimit();
+      int offset = plan.getOffset();
       for (LeafMNode leaf : allMatchedNodes) {
         if (match(leaf.getFullPath(), prefixNodes)) {
+          curOffset ++;
+          if (curOffset < offset) {
+            continue;
+          }
           try {
             Pair<Map<String, String>, Map<String, String>> pair =
                     tagLogFile.read(config.getTagAttributeTotalSize(), leaf.getOffset());
@@ -709,6 +717,10 @@ public class MManager {
                     getStorageGroupName(leaf.getFullPath()), measurementSchema.getType().toString(),
                     measurementSchema.getEncodingType().toString(),
                     measurementSchema.getCompressor().toString(), pair.left));
+            count ++;
+            if (count == limit) {
+              return res;
+            }
           } catch (IOException e) {
             throw new MetadataException(
                 "Something went wrong while deserialize tag info of " + leaf.getFullPath(), e);
@@ -738,29 +750,44 @@ 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);
+
+        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()));
-            continue;
+                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], 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 ++;
+          if (count == plan.getLimit()) {
+            return res;
+          }
+
         } 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..b275838 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
@@ -357,17 +357,17 @@ public class PlanExecutor implements IPlanExecutor {
     return listDataSet;
   }
 
-  private QueryDataSet processShowTimeseriesWithIndex(ShowTimeSeriesPlan timeSeriesPlan)
+  private QueryDataSet processShowTimeseriesWithIndex(ShowTimeSeriesPlan showTimeSeriesPlan)
       throws MetadataException {
     List<ShowTimeSeriesResult> timeseriesList = MManager.getInstance()
-        .getAllTimeseriesSchema(timeSeriesPlan);
+        .getAllTimeseriesSchema(showTimeSeriesPlan);
     return getQueryDataSet(timeseriesList);
   }
 
-  private QueryDataSet processShowTimeseries(ShowTimeSeriesPlan timeSeriesPlan)
+  private QueryDataSet processShowTimeseries(ShowTimeSeriesPlan showTimeSeriesPlan)
       throws MetadataException {
     List<ShowTimeSeriesResult> timeseriesList = MManager.getInstance()
-        .getAllTimeseriesSchema(timeSeriesPlan.getPath().toString());
+        .showTimeseries(showTimeSeriesPlan);
     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 f8bb52a..5df1f3a 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/integration/IoTDBTagIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java
index d63833c..ccd8e14 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java
@@ -216,6 +216,47 @@ public class IoTDBTagIT {
     }
   }
 
+
+  @Test
+  public void queryWithLimitTest() throws ClassNotFoundException {
+    String[] ret = {"root.turbine.d1.s2,temperature2,root.turbine,FLOAT,RLE,SNAPPY,v1,v2,v1,v2",
+        "root.turbine.d1.s3,temperature3,root.turbine,FLOAT,RLE,SNAPPY,v1,v2,v1,v2"};
+    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()) {
+      statement.execute("create timeseries root.turbine.d1.s1(temperature1) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+          "tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)");
+      statement.execute("create timeseries root.turbine.d1.s2(temperature2) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+          "tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)");
+      statement.execute("create timeseries root.turbine.d1.s3(temperature3) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+          "tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)");
+
+      boolean hasResult = statement.execute("show timeseries root.turbine.d1 where tag1=v1 limit 2 offset 1");
+      assertTrue(hasResult);
+      ResultSet resultSet = statement.getResultSet();
+      int count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+            + "," + resultSet.getString("alias")
+            + "," + resultSet.getString("storage group")
+            + "," + resultSet.getString("dataType")
+            + "," + resultSet.getString("encoding")
+            + "," + resultSet.getString("compression")
+            + "," + resultSet.getString("attr1")
+            + "," + resultSet.getString("attr2")
+            + "," + resultSet.getString("tag1")
+            + "," + resultSet.getString("tag2");
+        assertEquals(ret[count], ans);
+        count++;
+      }
+      assertEquals(ret.length, count);
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+  }
+
   @Test
   public void deleteTest() throws ClassNotFoundException {
     String[] ret1 = {
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..40b487d 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
@@ -29,6 +29,8 @@ import org.apache.iotdb.db.qp.logical.RootOperator;
 import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
 import org.apache.iotdb.db.qp.logical.sys.DeleteStorageGroupOperator;
 import org.apache.iotdb.db.qp.logical.sys.SetStorageGroupOperator;
+import org.apache.iotdb.db.qp.logical.sys.ShowTimeSeriesOperator;
+import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
 import org.apache.iotdb.db.qp.strategy.ParseDriver;
 import org.apache.iotdb.db.qp.strategy.optimizer.ConcatPathOptimizer;
 import org.apache.iotdb.tsfile.read.common.Path;
@@ -130,12 +132,12 @@ public class LogicalPlanSmallTest {
     // expected to throw SQLParserException: Out of range. OFFSET <OFFSETValue>: OFFSETValue should be Int32.
   }
 
-  @Test(expected = SQLParserException.class)
+  @Test(expected = ParseCancellationException.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)
@@ -163,12 +165,13 @@ public class LogicalPlanSmallTest {
     // expected to throw SQLParserException: Out of range. SOFFSET <SOFFSETValue>: SOFFSETValue should be Int32.
   }
 
-  @Test(expected = SQLParserException.class)
+  @Test
   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.
+    Assert.assertEquals(1, ((QueryOperator) operator).getSeriesOffset());
+    Assert.assertEquals(1, ((QueryOperator) operator).getSeriesLimit());
   }
 
   @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);