You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ja...@apache.org on 2021/06/08 10:18:37 UTC

[iotdb] branch LogQueryMemoryControl updated (b244c79 -> cde5fc4)

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

jackietien pushed a change to branch LogQueryMemoryControl
in repository https://gitbox.apache.org/repos/asf/iotdb.git.


    from b244c79  add more logs
     new d2639e6  delete log
     new cde5fc4  add command show slow query

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4   |  9 ++++
 .../apache/iotdb/db/qp/constant/SQLConstant.java   |  1 +
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  | 10 ++++
 ...rgeOperator.java => ShowSlowQueryOperator.java} |  8 ++--
 .../apache/iotdb/db/qp/physical/sys/ShowPlan.java  |  2 +-
 ...MergeStatusPlan.java => ShowSlowQueryPlan.java} |  8 ++--
 .../apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java    |  8 ++++
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |  7 ++-
 .../db/query/control/QueryResourceManager.java     | 55 ++++++++++++++++------
 .../org/apache/iotdb/db/service/TSServiceImpl.java |  1 +
 10 files changed, 82 insertions(+), 27 deletions(-)
 copy server/src/main/java/org/apache/iotdb/db/qp/logical/sys/{MergeOperator.java => ShowSlowQueryOperator.java} (82%)
 copy server/src/main/java/org/apache/iotdb/db/qp/physical/sys/{ShowMergeStatusPlan.java => ShowSlowQueryPlan.java} (85%)

[iotdb] 01/02: delete log

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jackietien pushed a commit to branch LogQueryMemoryControl
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit d2639e6df21648f12bc2c54a96d6ed29defd9609
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Tue Jun 8 15:23:29 2021 +0800

    delete log
---
 .../db/query/control/QueryResourceManager.java     | 37 +++++++++++++---------
 1 file changed, 22 insertions(+), 15 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java b/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
index 26c5fb5..12db0e8 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
@@ -76,6 +76,26 @@ public class QueryResourceManager {
 
   private final Map<Long, Long> queryIdEstimatedMemoryMap;
 
+  private final Map<Long, QueryInfo> queryDebugMap;
+
+  private static class QueryInfo {
+    long startTime;
+    String sql;
+    int deduplicatedPathNum;
+    int fetchSize;
+    long estimatedMemoryUsage;
+
+    public QueryInfo(long startTime, String sql, int deduplicatedPathNum, int fetchSize,
+        long estimatedMemoryUsage) {
+      this.startTime = startTime;
+      this.sql = sql;
+      this.deduplicatedPathNum = deduplicatedPathNum;
+      this.fetchSize = fetchSize;
+      this.estimatedMemoryUsage = estimatedMemoryUsage;
+    }
+  }
+
+
   // current total free memory for reading process(not including the cache memory)
   private final AtomicLong totalFreeMemoryForRead;
 
@@ -88,6 +108,7 @@ public class QueryResourceManager {
     filePathsManager = new QueryFileManager();
     externalSortFileMap = new ConcurrentHashMap<>();
     queryIdEstimatedMemoryMap = new ConcurrentHashMap<>();
+    queryDebugMap = new ConcurrentHashMap<>();
     totalFreeMemoryForRead = new AtomicLong(
         IoTDBDescriptor.getInstance().getConfig().getAllocateMemoryForReadWithoutCache());
   }
@@ -136,16 +157,9 @@ public class QueryResourceManager {
         // apply the memory successfully
         if (totalFreeMemoryForRead.addAndGet(-estimatedMemoryUsage) >= 0) {
           queryIdEstimatedMemoryMap.put(queryId, estimatedMemoryUsage);
-          logger.info(
-              "query id {}, sql is {}, apply the memory successfully, deduplicatedPathNum is {}, fetchSize is {}, used {} memory, left {} memory",
-              queryId, sql, deduplicatedPathNum, fetchSize, estimatedMemoryUsage,
-              totalFreeMemoryForRead.get());
+          queryDebugMap.put(queryId, new QueryInfo(System.currentTimeMillis(), sql, deduplicatedPathNum, fetchSize, estimatedMemoryUsage));
         } else {
           totalFreeMemoryForRead.addAndGet(estimatedMemoryUsage);
-          logger.info(
-              "query id {}, sql is {}, apply the memory failed, deduplicatedPathNum is {}, fetchSize is {}, used {} memory, left {} memory",
-              queryId, sql, deduplicatedPathNum, fetchSize, estimatedMemoryUsage,
-              totalFreeMemoryForRead.get());
         }
       }
     }
@@ -233,13 +247,6 @@ public class QueryResourceManager {
     Long estimatedMemoryUsage = queryIdEstimatedMemoryMap.remove(queryId);
     if (estimatedMemoryUsage != null) {
       totalFreeMemoryForRead.addAndGet(estimatedMemoryUsage);
-      logger.info(
-          "query id {}, put back the memory successfully, memory size is {}, current  memory is {}",
-          queryId, estimatedMemoryUsage, totalFreeMemoryForRead.get());
-    } else {
-      logger.info(
-          "query id {}, put back the memory failed, current  memory is {}",
-          queryId, totalFreeMemoryForRead.get());
     }
 
     // remove usage of opened file paths of current thread

[iotdb] 02/02: add command show slow query

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jackietien pushed a commit to branch LogQueryMemoryControl
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit cde5fc4b6ae171084d04b20f77afe2dfe3769f1a
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Tue Jun 8 18:17:46 2021 +0800

    add command show slow query
---
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4   |  9 ++++++++
 .../apache/iotdb/db/qp/constant/SQLConstant.java   |  1 +
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  | 10 ++++++++
 .../db/qp/logical/sys/ShowSlowQueryOperator.java   | 27 ++++++++++++++++++++++
 .../apache/iotdb/db/qp/physical/sys/ShowPlan.java  |  2 +-
 .../db/qp/physical/sys/ShowSlowQueryPlan.java      | 27 ++++++++++++++++++++++
 .../apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java    |  8 +++++++
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |  7 ++++--
 .../db/query/control/QueryResourceManager.java     | 20 +++++++++++++++-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |  1 +
 10 files changed, 108 insertions(+), 4 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
index 510fcf2..1d9d660 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
@@ -76,6 +76,7 @@ statement
     | SHOW VERSION #showVersion
     | SHOW LATEST? TIMESERIES prefixPath? showWhereClause? limitClause? #showTimeseries
     | SHOW STORAGE GROUP prefixPath? #showStorageGroup
+    | SHOW SLOW QUERY #showSlowQuery
     | SHOW CHILD PATHS prefixPath? #showChildPaths
     | SHOW DEVICES prefixPath? #showDevices
     | SHOW MERGE #showMergeStatus
@@ -1190,6 +1191,14 @@ EXPLAIN
     : E X P L A I N
     ;
 
+SLOW
+    : S L O W
+    ;
+
+QUERY
+    : Q U E R Y
+    ;
+
 //============================
 // End of the keywords list
 //============================
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
index 04b9520..00d65f8 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
@@ -157,6 +157,7 @@ public class SQLConstant {
 
   public static final int TOK_COUNT_DEVICES = 92;
   public static final int TOK_COUNT_STORAGE_GROUP = 93;
+  public static final int TOK_SLOW_QUERY = 94;
 
   public static final Map<Integer, String> tokenSymbol = new HashMap<>();
   public static final Map<Integer, String> tokenNames = new HashMap<>();
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 cc6a173..bcfb190 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
@@ -118,6 +118,7 @@ import org.apache.iotdb.db.qp.physical.sys.ShowTTLPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.TracingPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.query.dataset.AlignByDeviceDataSet;
 import org.apache.iotdb.db.query.dataset.ListDataSet;
 import org.apache.iotdb.db.query.dataset.ShowTimeseriesDataSet;
@@ -395,6 +396,8 @@ public class PlanExecutor implements IPlanExecutor {
         return processShowTimeseries((ShowTimeSeriesPlan) showPlan, context);
       case STORAGE_GROUP:
         return processShowStorageGroup((ShowStorageGroupPlan) showPlan);
+      case SLOW_QUERY:
+        return processShowSlowQuery();
       case DEVICES:
         return processShowDevices((ShowDevicesPlan) showPlan);
       case CHILD_PATH:
@@ -582,6 +585,13 @@ public class PlanExecutor implements IPlanExecutor {
     return listDataSet;
   }
 
+  private QueryDataSet processShowSlowQuery() {
+    QueryResourceManager.getInstance().printLongTimeQueryInfo();
+    return new ListDataSet(
+        Collections.singletonList(new PartialPath(COLUMN_STORAGE_GROUP, false)),
+        Collections.singletonList(TSDataType.TEXT));
+  }
+
   private QueryDataSet processShowTimeseries(ShowTimeSeriesPlan showTimeSeriesPlan,
       QueryContext context) throws MetadataException {
     return new ShowTimeseriesDataSet(showTimeSeriesPlan, context);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowSlowQueryOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowSlowQueryOperator.java
new file mode 100644
index 0000000..2a0288b
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/ShowSlowQueryOperator.java
@@ -0,0 +1,27 @@
+/*
+ * 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.qp.logical.sys;
+
+public class ShowSlowQueryOperator extends ShowOperator {
+
+  public ShowSlowQueryOperator(int tokenIntType) {
+    super(tokenIntType);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPlan.java
index dabb949..bbe25cc 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPlan.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowPlan.java
@@ -51,7 +51,7 @@ public class ShowPlan extends PhysicalPlan {
 
   public enum ShowContentType {
     FLUSH_TASK_INFO, TTL, VERSION, TIMESERIES, STORAGE_GROUP, CHILD_PATH, DEVICES,
-    COUNT_TIMESERIES, COUNT_NODE_TIMESERIES, COUNT_NODES, MERGE_STATUS, COUNT_DEVICES, COUNT_STORAGE_GROUP
+    COUNT_TIMESERIES, COUNT_NODE_TIMESERIES, COUNT_NODES, MERGE_STATUS, COUNT_DEVICES, COUNT_STORAGE_GROUP, SLOW_QUERY
   }
 
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowSlowQueryPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowSlowQueryPlan.java
new file mode 100644
index 0000000..265f29d
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowSlowQueryPlan.java
@@ -0,0 +1,27 @@
+/*
+ * 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.qp.physical.sys;
+
+public class ShowSlowQueryPlan extends ShowPlan {
+
+  public ShowSlowQueryPlan(ShowContentType showContentType) {
+    super(showContentType);
+  }
+}
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 92ed310..ea60b85 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
@@ -19,6 +19,7 @@
 package org.apache.iotdb.db.qp.sql;
 
 import static org.apache.iotdb.db.qp.constant.SQLConstant.TIME_PATH;
+import static org.apache.iotdb.db.qp.constant.SQLConstant.TOK_SLOW_QUERY;
 
 import java.io.File;
 import java.time.ZoneId;
@@ -72,6 +73,7 @@ import org.apache.iotdb.db.qp.logical.sys.ShowChildPathsOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowDevicesOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowMergeStatusOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowOperator;
+import org.apache.iotdb.db.qp.logical.sys.ShowSlowQueryOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowStorageGroupOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowTTLOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowTimeSeriesOperator;
@@ -172,6 +174,7 @@ import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowChildPathsContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowDevicesContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowFlushTaskInfoContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowMergeStatusContext;
+import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowSlowQueryContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowStorageGroupContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowTTLStatementContext;
 import org.apache.iotdb.db.qp.sql.SqlBaseParser.ShowTimeseriesContext;
@@ -641,6 +644,11 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
   }
 
   @Override
+  public Operator visitShowSlowQuery(ShowSlowQueryContext ctx) {
+    return new ShowSlowQueryOperator(TOK_SLOW_QUERY);
+  }
+
+  @Override
   public Operator visitShowChildPaths(ShowChildPathsContext ctx) {
     if (ctx.prefixPath() != null) {
       return new ShowChildPathsOperator(
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 d7863c8..3c0ac71 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
@@ -28,9 +28,9 @@ import java.util.Map;
 import java.util.Set;
 import org.apache.iotdb.db.auth.AuthException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.exception.query.PathNumOverLimitException;
 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.exception.runtime.SQLParserException;
 import org.apache.iotdb.db.metadata.PartialPath;
@@ -100,11 +100,12 @@ import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowMergeStatusPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowPlan.ShowContentType;
+import org.apache.iotdb.db.qp.physical.sys.ShowSlowQueryPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTTLPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan;
-import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.qp.physical.sys.TracingPlan;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
 import org.apache.iotdb.db.service.IoTDB;
 import org.apache.iotdb.db.utils.SchemaUtils;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -236,6 +237,8 @@ public class PhysicalGenerator {
           case SQLConstant.TOK_STORAGE_GROUP:
             return new ShowStorageGroupPlan(
                 ShowContentType.STORAGE_GROUP, ((ShowStorageGroupOperator) operator).getPath());
+          case SQLConstant.TOK_SLOW_QUERY:
+            return new ShowSlowQueryPlan(ShowContentType.SLOW_QUERY);
           case SQLConstant.TOK_DEVICES:
             return new ShowDevicesPlan(
                 ShowContentType.DEVICES, ((ShowDevicesOperator) operator).getPath());
diff --git a/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java b/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
index 12db0e8..22bb96c 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
@@ -58,6 +58,8 @@ public class QueryResourceManager {
   private final AtomicLong queryIdAtom = new AtomicLong();
   private final QueryFileManager filePathsManager;
   private static final Logger logger = LoggerFactory.getLogger(QueryResourceManager.class);
+  private static final Logger DEBUG_LOGGER = LoggerFactory.getLogger("QUERY_DEBUG");
+
   // record the total number and size of chunks for each query id
   private Map<Long, Integer> chunkNumMap = new ConcurrentHashMap<>();
   // chunk size represents the number of time-value points in the chunk
@@ -79,6 +81,7 @@ public class QueryResourceManager {
   private final Map<Long, QueryInfo> queryDebugMap;
 
   private static class QueryInfo {
+
     long startTime;
     String sql;
     int deduplicatedPathNum;
@@ -157,7 +160,9 @@ public class QueryResourceManager {
         // apply the memory successfully
         if (totalFreeMemoryForRead.addAndGet(-estimatedMemoryUsage) >= 0) {
           queryIdEstimatedMemoryMap.put(queryId, estimatedMemoryUsage);
-          queryDebugMap.put(queryId, new QueryInfo(System.currentTimeMillis(), sql, deduplicatedPathNum, fetchSize, estimatedMemoryUsage));
+          queryDebugMap.put(queryId,
+              new QueryInfo(System.currentTimeMillis(), sql, deduplicatedPathNum, fetchSize,
+                  estimatedMemoryUsage));
         } else {
           totalFreeMemoryForRead.addAndGet(estimatedMemoryUsage);
         }
@@ -245,6 +250,7 @@ public class QueryResourceManager {
 
     // put back the memory usage
     Long estimatedMemoryUsage = queryIdEstimatedMemoryMap.remove(queryId);
+    queryDebugMap.remove(queryId);
     if (estimatedMemoryUsage != null) {
       totalFreeMemoryForRead.addAndGet(estimatedMemoryUsage);
     }
@@ -253,6 +259,18 @@ public class QueryResourceManager {
     filePathsManager.removeUsedFilesForQuery(queryId);
   }
 
+  public void printLongTimeQueryInfo() {
+    long currentTime = System.currentTimeMillis();
+    queryDebugMap.forEach((k, v) -> {
+      // print query that has been running 5 minutes
+      if (currentTime - v.startTime > 300_000) {
+        logger.info(
+            "query id {}, sql is {}, deduplicatedPathNum is {}, fetchSize is {}, used {} memory",
+            k, v.sql, v.deduplicatedPathNum, v.fetchSize, v.estimatedMemoryUsage);
+      }
+    });
+  }
+
   private static class QueryTokenManagerHelper {
 
     private static final QueryResourceManager INSTANCE = new QueryResourceManager();
diff --git a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index 71677d5..93a39c3 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -800,6 +800,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
       case TIMESERIES:
         return StaticResps.SHOW_TIMESERIES_RESP;
       case STORAGE_GROUP:
+      case SLOW_QUERY:
         return StaticResps.SHOW_STORAGE_GROUP;
       case CHILD_PATH:
         return StaticResps.SHOW_CHILD_PATHS;