You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by zy...@apache.org on 2022/07/19 07:26:46 UTC

[iotdb] branch rel/0.13 updated: [To rel/0.13][IOTDB-3741][IOTDB-3747] Default Paging of Schema Query (#6707)

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

zyk pushed a commit to branch rel/0.13
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/rel/0.13 by this push:
     new 9a5e12bc71 [To rel/0.13][IOTDB-3741][IOTDB-3747] Default Paging of Schema Query (#6707)
9a5e12bc71 is described below

commit 9a5e12bc71d1c19aa2541c0c588b4754efdbafff
Author: Marcos_Zyk <38...@users.noreply.github.com>
AuthorDate: Tue Jul 19 15:26:42 2022 +0800

    [To rel/0.13][IOTDB-3741][IOTDB-3747] Default Paging of Schema Query (#6707)
    
    [To rel/0.13][IOTDB-3741][IOTDB-3747] Default Paging of Schema Query (#6707)
---
 .../org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4   |  4 +-
 .../iotdb/cluster/query/ClusterPlanExecutor.java   |  6 ++-
 docs/UserGuide/Operate-Metadata/Node.md            | 12 ++++--
 docs/UserGuide/Operate-Metadata/Timeseries.md      |  3 ++
 docs/zh/UserGuide/Operate-Metadata/Node.md         | 10 ++++-
 docs/zh/UserGuide/Operate-Metadata/Timeseries.md   |  2 +
 .../iotdb/db/integration/IoTDBMetadataFetchIT.java | 49 ++++++++++++++++++++++
 .../org/apache/iotdb/db/metadata/MManager.java     | 14 ++++++-
 .../org/apache/iotdb/db/metadata/mtree/MTree.java  | 10 ++++-
 .../mtree/traverser/collector/MNodeCollector.java  | 19 +++++++--
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  | 22 +++++++---
 .../db/qp/logical/sys/ShowChildNodesOperator.java  | 24 ++++++++++-
 .../db/qp/logical/sys/ShowChildPathsOperator.java  | 24 ++++++++++-
 .../db/qp/logical/sys/ShowDevicesOperator.java     |  2 +-
 .../db/qp/logical/sys/ShowTimeSeriesOperator.java  |  2 +-
 .../apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java    | 32 ++++++++++----
 .../iotdb/db/metadata/MManagerBasicTest.java       | 42 +++++++++++++++++++
 .../org/apache/iotdb/db/metadata/MTreeTest.java    |  8 ++--
 18 files changed, 247 insertions(+), 38 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
index 1e322368c8..893504909e 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4
@@ -257,12 +257,12 @@ showWhereClause
 
 // Show Child Paths
 showChildPaths
-    : SHOW CHILD PATHS prefixPath?
+    : SHOW CHILD PATHS prefixPath? limitClause?
     ;
 
 // Show Child Nodes
 showChildNodes
-    : SHOW CHILD NODES prefixPath?
+    : SHOW CHILD NODES prefixPath? limitClause?
     ;
 
 // Show Functions
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java
index 04ef0124e3..0f9d748323 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/ClusterPlanExecutor.java
@@ -635,7 +635,8 @@ public class ClusterPlanExecutor extends PlanExecutor {
   }
 
   @Override
-  protected Set<String> getNodeNextChildren(PartialPath path) throws MetadataException {
+  protected Set<String> getNodeNextChildren(PartialPath path, int limit, int offset)
+      throws MetadataException {
     ConcurrentSkipListSet<String> resultSet = new ConcurrentSkipListSet<>();
     List<PartitionGroup> globalGroups = metaGroupMember.getPartitionTable().getGlobalGroups();
     // TODO: create a thread pool for each query calling.
@@ -737,7 +738,8 @@ public class ClusterPlanExecutor extends PlanExecutor {
   }
 
   @Override
-  protected Set<String> getPathNextChildren(PartialPath path) throws MetadataException {
+  protected Set<String> getPathNextChildren(PartialPath path, int limit, int offset)
+      throws MetadataException {
     ConcurrentSkipListSet<String> resultSet = new ConcurrentSkipListSet<>();
     // TODO: create a thread pool for each query calling.
     ExecutorService pool = Executors.newFixedThreadPool(THREAD_POOL_SIZE);
diff --git a/docs/UserGuide/Operate-Metadata/Node.md b/docs/UserGuide/Operate-Metadata/Node.md
index 3d0963b867..b5710b501f 100644
--- a/docs/UserGuide/Operate-Metadata/Node.md
+++ b/docs/UserGuide/Operate-Metadata/Node.md
@@ -23,10 +23,12 @@
 ## Show Child Paths
 
 ```
-SHOW CHILD PATHS pathPattern
+SHOW CHILD PATHS pathPattern ? limitClause
 ```
 
-Return all child paths of all the paths matching pathPattern.
+Return all child paths of all the paths matching pathPattern. 
+
+The default size of result set is 1000. To query more child paths, please use ```limit``` and ```offset```.
 
 Example:
 
@@ -48,11 +50,13 @@ It costs 0.002s
 ## Show Child Nodes
 
 ```
-SHOW CHILD NODES pathPattern
+SHOW CHILD NODES pathPattern ? limitClause
 ```
 
 Return all child nodes of the pathPattern.
 
+The default size of result set is 1000. To query more child paths, please use ```limit``` and ```offset```.
+
 Example:
 
 * return the child nodes of root:show child nodes root
@@ -133,6 +137,8 @@ It costs 0.002s
 
 * SHOW DEVICES pathPattern? (WITH STORAGE GROUP)? limitClause? #showDevices
 
+The default size of result set is 1000. To query more child paths, please use ```limit``` and ```offset```.
+
 Similar to `Show Timeseries`, IoTDB also supports two ways of viewing devices:
 
 * `SHOW DEVICES` statement presents all devices' information, which is equal to `SHOW DEVICES root.**`.
diff --git a/docs/UserGuide/Operate-Metadata/Timeseries.md b/docs/UserGuide/Operate-Metadata/Timeseries.md
index 7761abb5b4..87c168a3ce 100644
--- a/docs/UserGuide/Operate-Metadata/Timeseries.md
+++ b/docs/UserGuide/Operate-Metadata/Timeseries.md
@@ -85,6 +85,9 @@ IoTDB> delete timeseries root.ln.wf02.*
   There are four optional clauses added in SHOW TIMESERIES, return information of time series 
   
 Timeseries information includes: timeseries path, alias of measurement, storage group it belongs to, data type, encoding type, compression type, tags and attributes.
+
+The default size of result set is 1000. To query more child paths, please use ```limit``` and ```offset```.
+
  
 Examples:
 
diff --git a/docs/zh/UserGuide/Operate-Metadata/Node.md b/docs/zh/UserGuide/Operate-Metadata/Node.md
index e75eb6c1ef..02e907bb26 100644
--- a/docs/zh/UserGuide/Operate-Metadata/Node.md
+++ b/docs/zh/UserGuide/Operate-Metadata/Node.md
@@ -24,11 +24,13 @@
 ## 查看子路径
 
 ```
-SHOW CHILD PATHS pathPattern
+SHOW CHILD PATHS pathPattern ? limitClause
 ```
 
 可以查看此路径模式所匹配的所有路径的下一层的所有路径,即pathPattern.*所匹配的路径。
 
+查询结果集的大小默认为1000,如需查询更多信息,请使用```limit```和```offset```。
+
 示例:
 
 * 查询 root.ln 的下一层:show child paths root.ln
@@ -56,11 +58,13 @@ SHOW CHILD PATHS pathPattern
 ## 查看子节点
 
 ```
-SHOW CHILD NODES pathPattern
+SHOW CHILD NODES pathPattern ? limitClause
 ```
 
 可以查看此路径模式所匹配的节点的下一层的所有节点。
 
+查询结果集的大小默认为1000,如需查询更多信息,请使用```limit```和```offset```。
+
 示例:
 
 * 查询 root 的下一层:show child nodes root
@@ -138,6 +142,8 @@ It costs 0.002s
 
 * SHOW DEVICES pathPattern? (WITH STORAGE GROUP)? limitClause? #showDevices
 
+查询结果集的大小默认为1000,如需查询更多信息,请使用```limit```和```offset```。
+
 与 `Show Timeseries` 相似,IoTDB 目前也支持两种方式查看设备。
 
 * `SHOW DEVICES` 语句显示当前所有的设备信息,等价于 `SHOW DEVICES root.**`。
diff --git a/docs/zh/UserGuide/Operate-Metadata/Timeseries.md b/docs/zh/UserGuide/Operate-Metadata/Timeseries.md
index 8d2b9d9ad8..014c98fddf 100644
--- a/docs/zh/UserGuide/Operate-Metadata/Timeseries.md
+++ b/docs/zh/UserGuide/Operate-Metadata/Timeseries.md
@@ -83,6 +83,8 @@ IoTDB> delete timeseries root.ln.wf02.*
 
 时间序列信息具体包括:时间序列路径名,存储组,Measurement 别名,数据类型,编码方式,压缩方式,属性和标签。
 
+查询结果集的大小默认为1000,如需查询更多信息,请使用```limit```和```offset```。
+
 示例:
 
 * SHOW TIMESERIES
diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java
index 4f6cd885a9..568778ca0a 100644
--- a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java
+++ b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java
@@ -661,4 +661,53 @@ public class IoTDBMetadataFetchIT {
       }
     }
   }
+
+  @Test
+  @Category({LocalStandaloneTest.class})
+  public void showChildWithLimitOffset() throws SQLException {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      String[] sqls =
+          new String[] {
+            "show child paths root limit 1 offset 0",
+            "show child paths root limit 1 offset 1",
+            "show child paths root limit 1 offset 2",
+            "show child paths root limit 2 offset 0",
+            "show child paths root limit 2 offset 1",
+            "show child paths root limit 3 offset 0"
+          };
+      String[] standards =
+          new String[] {
+            "root.ln2,\n",
+            "root.ln,\n",
+            "root.ln1,\n",
+            "root.ln,\nroot.ln2,\n", // the result of show query will be sorted
+            "root.ln,\nroot.ln1,\n",
+            "root.ln,\nroot.ln1,\nroot.ln2,\n"
+          };
+      for (int n = 0; n < sqls.length; n++) {
+        String sql = sqls[n];
+        String standard = standards[n];
+        StringBuilder builder = new StringBuilder();
+        try {
+          boolean hasResultSet = statement.execute(sql);
+          if (hasResultSet) {
+            try (ResultSet resultSet = statement.getResultSet()) {
+              ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+              while (resultSet.next()) {
+                for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
+                  builder.append(resultSet.getString(i)).append(",");
+                }
+                builder.append("\n");
+              }
+            }
+          }
+          Assert.assertEquals(standard, builder.toString());
+        } catch (SQLException e) {
+          logger.error("showChildPaths() failed", e);
+          fail(e.getMessage());
+        }
+      }
+    }
+  }
 }
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 dfea8addf6..12d89f8bba 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
@@ -1110,7 +1110,12 @@ public class MManager {
    * @return All child nodes' seriesPath(s) of given seriesPath.
    */
   public Set<String> getChildNodePathInNextLevel(PartialPath pathPattern) throws MetadataException {
-    return mtree.getChildNodePathInNextLevel(pathPattern);
+    return mtree.getChildNodePathInNextLevel(pathPattern, 0, 0);
+  }
+
+  public Set<String> getChildNodePathInNextLevel(PartialPath pathPattern, int limit, int offset)
+      throws MetadataException {
+    return mtree.getChildNodePathInNextLevel(pathPattern, limit, offset);
   }
 
   /**
@@ -1124,7 +1129,12 @@ public class MManager {
    * @return All child nodes of given seriesPath.
    */
   public Set<String> getChildNodeNameInNextLevel(PartialPath pathPattern) throws MetadataException {
-    return mtree.getChildNodeNameInNextLevel(pathPattern);
+    return mtree.getChildNodeNameInNextLevel(pathPattern, 0, 0);
+  }
+
+  public Set<String> getChildNodeNameInNextLevel(PartialPath pathPattern, int limit, int offset)
+      throws MetadataException {
+    return mtree.getChildNodeNameInNextLevel(pathPattern, limit, offset);
   }
   // endregion
 
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
index f0501e8048..8ce8c286b7 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/MTree.java
@@ -1186,7 +1186,8 @@ public class MTree implements Serializable {
    * @param pathPattern The given path
    * @return All child nodes' seriesPath(s) of given seriesPath.
    */
-  public Set<String> getChildNodePathInNextLevel(PartialPath pathPattern) throws MetadataException {
+  public Set<String> getChildNodePathInNextLevel(PartialPath pathPattern, int limit, int offset)
+      throws MetadataException {
     try {
       MNodeCollector<Set<String>> collector =
           new MNodeCollector<Set<String>>(root, pathPattern.concatNode(ONE_LEVEL_PATH_WILDCARD)) {
@@ -1200,6 +1201,8 @@ public class MTree implements Serializable {
             }
           };
       collector.setResultSet(new TreeSet<>());
+      collector.setLimit(limit);
+      collector.setOffset(offset);
       collector.traverse();
       return collector.getResult();
     } catch (IllegalPathException e) {
@@ -1219,7 +1222,8 @@ public class MTree implements Serializable {
    * @param pathPattern Path
    * @return All child nodes' seriesPath(s) of given seriesPath.
    */
-  public Set<String> getChildNodeNameInNextLevel(PartialPath pathPattern) throws MetadataException {
+  public Set<String> getChildNodeNameInNextLevel(PartialPath pathPattern, int limit, int offset)
+      throws MetadataException {
     try {
       MNodeCollector<Set<String>> collector =
           new MNodeCollector<Set<String>>(root, pathPattern.concatNode(ONE_LEVEL_PATH_WILDCARD)) {
@@ -1229,6 +1233,8 @@ public class MTree implements Serializable {
             }
           };
       collector.setResultSet(new TreeSet<>());
+      collector.setLimit(limit);
+      collector.setOffset(offset);
       collector.traverse();
       return collector.getResult();
     } catch (IllegalPathException e) {
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MNodeCollector.java b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MNodeCollector.java
index 72528529de..3a57ba3be4 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MNodeCollector.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/mtree/traverser/collector/MNodeCollector.java
@@ -75,13 +75,26 @@ public abstract class MNodeCollector<T> extends CollectorTraverser<T> {
       // record processed node so they will not be processed twice
       if (!processedNodes.contains(node)) {
         processedNodes.add(node);
-        transferToResult(node);
+        processResult(node);
       }
       return true;
     } else {
-      transferToResult(node);
+      processResult(node);
+      return false;
     }
-    return false;
+  }
+
+  private void processResult(IMNode node) {
+    if (hasLimit) {
+      curOffset += 1;
+      if (curOffset < offset) {
+        return;
+      }
+
+      count++;
+    }
+
+    transferToResult(node);
   }
 
   protected abstract void transferToResult(IMNode node);
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 2f78c28844..5b8a0bcbfe 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
@@ -878,7 +878,11 @@ public class PlanExecutor implements IPlanExecutor {
 
   private QueryDataSet processShowChildPaths(ShowChildPathsPlan showChildPathsPlan)
       throws MetadataException {
-    Set<String> childPathsList = getPathNextChildren(showChildPathsPlan.getPath());
+    Set<String> childPathsList =
+        getPathNextChildren(
+            showChildPathsPlan.getPath(),
+            showChildPathsPlan.getLimit(),
+            showChildPathsPlan.getOffset());
     ListDataSet listDataSet =
         new ListDataSet(
             Collections.singletonList(new PartialPath(COLUMN_CHILD_PATHS, false)),
@@ -893,14 +897,19 @@ public class PlanExecutor implements IPlanExecutor {
     return listDataSet;
   }
 
-  protected Set<String> getPathNextChildren(PartialPath path) throws MetadataException {
-    return IoTDB.metaManager.getChildNodePathInNextLevel(path);
+  protected Set<String> getPathNextChildren(PartialPath path, int limit, int offset)
+      throws MetadataException {
+    return IoTDB.metaManager.getChildNodePathInNextLevel(path, limit, offset);
   }
 
   private QueryDataSet processShowChildNodes(ShowChildNodesPlan showChildNodesPlan)
       throws MetadataException {
     // getNodeNextChildren
-    Set<String> childNodesList = getNodeNextChildren(showChildNodesPlan.getPath());
+    Set<String> childNodesList =
+        getNodeNextChildren(
+            showChildNodesPlan.getPath(),
+            showChildNodesPlan.getLimit(),
+            showChildNodesPlan.getOffset());
     ListDataSet listDataSet =
         new ListDataSet(
             Collections.singletonList(new PartialPath(COLUMN_CHILD_NODES, false)),
@@ -915,8 +924,9 @@ public class PlanExecutor implements IPlanExecutor {
     return listDataSet;
   }
 
-  protected Set<String> getNodeNextChildren(PartialPath path) throws MetadataException {
-    return IoTDB.metaManager.getChildNodeNameInNextLevel(path);
+  protected Set<String> getNodeNextChildren(PartialPath path, int limit, int offset)
+      throws MetadataException {
+    return IoTDB.metaManager.getChildNodeNameInNextLevel(path, limit, offset);
   }
 
   protected List<PartialPath> getStorageGroupNames(PartialPath path, boolean isPrefixMatch)
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowChildNodesOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowChildNodesOperator.java
index c3f8b7c59c..4ba24da3e7 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowChildNodesOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowChildNodesOperator.java
@@ -30,6 +30,9 @@ public class ShowChildNodesOperator extends ShowOperator {
 
   private PartialPath path;
 
+  private int limit = 1000;
+  private int offset = 0;
+
   public ShowChildNodesOperator(int tokenIntType, PartialPath path) {
     super(tokenIntType);
     this.path = path;
@@ -39,9 +42,28 @@ public class ShowChildNodesOperator extends ShowOperator {
     return path;
   }
 
+  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;
+  }
+
   @Override
   public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
       throws QueryProcessException {
-    return new ShowChildNodesPlan(ShowContentType.CHILD_NODE, path);
+    ShowChildNodesPlan plan = new ShowChildNodesPlan(ShowContentType.CHILD_NODE, path);
+    plan.setLimit(limit);
+    plan.setOffset(offset);
+    return plan;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowChildPathsOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowChildPathsOperator.java
index 7f8ad38bd5..da96dac480 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowChildPathsOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowChildPathsOperator.java
@@ -30,6 +30,9 @@ public class ShowChildPathsOperator extends ShowOperator {
 
   private PartialPath path;
 
+  private int limit = 1000;
+  private int offset = 0;
+
   public ShowChildPathsOperator(int tokenIntType, PartialPath path) {
     super(tokenIntType);
     this.path = path;
@@ -39,9 +42,28 @@ public class ShowChildPathsOperator extends ShowOperator {
     return path;
   }
 
+  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;
+  }
+
   @Override
   public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator)
       throws QueryProcessException {
-    return new ShowChildPathsPlan(ShowContentType.CHILD_PATH, path);
+    ShowChildPathsPlan plan = new ShowChildPathsPlan(ShowContentType.CHILD_PATH, path);
+    plan.setLimit(limit);
+    plan.setOffset(offset);
+    return plan;
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowDevicesOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowDevicesOperator.java
index 42e38b42cc..df1a24e87e 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowDevicesOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowDevicesOperator.java
@@ -28,7 +28,7 @@ import org.apache.iotdb.db.qp.strategy.PhysicalGenerator;
 public class ShowDevicesOperator extends ShowOperator {
 
   private PartialPath path;
-  private int limit = 0;
+  private int limit = 1000;
   private int offset = 0;
   private boolean hasSgCol;
 
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 78ba7c8c6a..102fb078aa 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
@@ -31,7 +31,7 @@ public class ShowTimeSeriesOperator extends ShowOperator {
   private boolean isContains;
   private String key;
   private String value;
-  private int limit = 0;
+  private int limit = 1000;
   private int offset = 0;
   // if is true, the result will be sorted according to the inserting frequency of the timeseries
   private final boolean orderByHeat;
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 1c78746b0e..4c6334f59a 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
@@ -885,26 +885,34 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
 
   @Override
   public Operator visitShowChildPaths(IoTDBSqlParser.ShowChildPathsContext ctx) {
+    PartialPath path;
     if (ctx.prefixPath() != null) {
-      return new ShowChildPathsOperator(
-          SQLConstant.TOK_CHILD_PATHS, parsePrefixPath(ctx.prefixPath()));
+      path = parsePrefixPath(ctx.prefixPath());
     } else {
-      return new ShowChildPathsOperator(
-          SQLConstant.TOK_CHILD_PATHS, new PartialPath(SQLConstant.getSingleRootArray()));
+      path = new PartialPath(SQLConstant.getSingleRootArray());
+    }
+    ShowChildPathsOperator operator = new ShowChildPathsOperator(SQLConstant.TOK_CHILD_PATHS, path);
+    if (ctx.limitClause() != null) {
+      parseLimitClause(ctx.limitClause(), operator);
     }
+    return operator;
   }
 
   // Show Child Nodes
 
   @Override
   public Operator visitShowChildNodes(IoTDBSqlParser.ShowChildNodesContext ctx) {
+    PartialPath path;
     if (ctx.prefixPath() != null) {
-      return new ShowChildNodesOperator(
-          SQLConstant.TOK_CHILD_NODES, parsePrefixPath(ctx.prefixPath()));
+      path = parsePrefixPath(ctx.prefixPath());
     } else {
-      return new ShowChildNodesOperator(
-          SQLConstant.TOK_CHILD_NODES, new PartialPath(SQLConstant.getSingleRootArray()));
+      path = new PartialPath(SQLConstant.getSingleRootArray());
+    }
+    ShowChildNodesOperator operator = new ShowChildNodesOperator(SQLConstant.TOK_CHILD_NODES, path);
+    if (ctx.limitClause() != null) {
+      parseLimitClause(ctx.limitClause(), operator);
     }
+    return operator;
   }
 
   // Show Functions
@@ -2735,6 +2743,10 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
       ((ShowTimeSeriesOperator) operator).setLimit(limit);
     } else if (operator instanceof ShowDevicesOperator) {
       ((ShowDevicesOperator) operator).setLimit(limit);
+    } else if (operator instanceof ShowChildPathsOperator) {
+      ((ShowChildPathsOperator) operator).setLimit(limit);
+    } else if (operator instanceof ShowChildNodesOperator) {
+      ((ShowChildNodesOperator) operator).setLimit(limit);
     } else {
       SpecialClauseComponent specialClauseComponent = queryOp.getSpecialClauseComponent();
       if (specialClauseComponent == null) {
@@ -2763,6 +2775,10 @@ public class IoTDBSqlVisitor extends IoTDBSqlParserBaseVisitor<Operator> {
       ((ShowTimeSeriesOperator) operator).setOffset(offset);
     } else if (operator instanceof ShowDevicesOperator) {
       ((ShowDevicesOperator) operator).setOffset(offset);
+    } else if (operator instanceof ShowChildNodesOperator) {
+      ((ShowChildNodesOperator) operator).setOffset(offset);
+    } else if (operator instanceof ShowChildPathsOperator) {
+      ((ShowChildPathsOperator) operator).setOffset(offset);
     } else {
       SpecialClauseComponent specialClauseComponent = queryOp.getSpecialClauseComponent();
       if (specialClauseComponent == null) {
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
index 64015b72df..fb6c0505d2 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
@@ -2540,4 +2540,46 @@ public class MManagerBasicTest {
 
     assertEquals(0, manager.getMeasurementMNode(path).getOffset());
   }
+
+  @Test
+  public void testShowChildWithLimitAndOffset() throws Exception {
+    MManager manager = IoTDB.metaManager;
+
+    PartialPath path = new PartialPath("root.sg.d1.s");
+    CreateTimeSeriesPlan plan =
+        new CreateTimeSeriesPlan(
+            path,
+            TSDataType.valueOf("INT32"),
+            TSEncoding.valueOf("RLE"),
+            compressionType,
+            null,
+            null,
+            null,
+            null);
+    manager.createTimeseries(plan);
+
+    plan.setPath(new PartialPath("root.sg.d2.s"));
+    manager.createTimeseries(plan);
+
+    plan.setPath(new PartialPath("root.sg.d3.s"));
+    manager.createTimeseries(plan);
+
+    Set<String> result = manager.getChildNodePathInNextLevel(new PartialPath("root.**"), 1, 1);
+    Assert.assertEquals(1, result.size());
+    Assert.assertTrue(result.contains("root.sg.d2"));
+
+    result = manager.getChildNodePathInNextLevel(new PartialPath("root.**"), 3, 3);
+    Assert.assertEquals(3, result.size());
+    Assert.assertTrue(result.contains("root.sg.d1.s"));
+    Assert.assertTrue(result.contains("root.sg.d2.s"));
+    Assert.assertTrue(result.contains("root.sg.d3.s"));
+
+    result = manager.getChildNodePathInNextLevel(new PartialPath("root.sg"), 1, 1);
+    Assert.assertEquals(1, result.size());
+    Assert.assertTrue(result.contains("root.sg.d2"));
+
+    result = manager.getChildNodeNameInNextLevel(new PartialPath("root.sg"), 1, 0);
+    Assert.assertEquals(1, result.size());
+    Assert.assertTrue(result.contains("d1"));
+  }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java b/server/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java
index d6de9bfcda..b8f05dc019 100644
--- a/server/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java
@@ -443,15 +443,15 @@ public class MTreeTest {
           null);
 
       // getChildNodeByPath
-      Set<String> result1 = root.getChildNodeNameInNextLevel(new PartialPath("root.a.d0"));
-      Set<String> result2 = root.getChildNodeNameInNextLevel(new PartialPath("root.a"));
-      Set<String> result3 = root.getChildNodeNameInNextLevel(new PartialPath("root"));
+      Set<String> result1 = root.getChildNodeNameInNextLevel(new PartialPath("root.a.d0"), 0, 0);
+      Set<String> result2 = root.getChildNodeNameInNextLevel(new PartialPath("root.a"), 0, 0);
+      Set<String> result3 = root.getChildNodeNameInNextLevel(new PartialPath("root"), 0, 0);
       assertEquals(new HashSet<>(Arrays.asList("s0", "s1")), result1);
       assertEquals(new HashSet<>(Arrays.asList("d0", "d5")), result2);
       assertEquals(new HashSet<>(Collections.singletonList("a")), result3);
 
       // if child node is nll   will return  null HashSet
-      Set<String> result5 = root.getChildNodeNameInNextLevel(new PartialPath("root.a.d5"));
+      Set<String> result5 = root.getChildNodeNameInNextLevel(new PartialPath("root.a.d5"), 0, 0);
       assertEquals(result5, new HashSet<>(Arrays.asList()));
     } catch (MetadataException e1) {
       e1.printStackTrace();