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:39 UTC

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

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;