You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2020/06/22 11:24:57 UTC

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #1399: [IOTDB-736] Query performance tracing

qiaojialin commented on a change in pull request #1399:
URL: https://github.com/apache/incubator-iotdb/pull/1399#discussion_r443442334



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
##########
@@ -223,6 +226,20 @@ public void enterFlush(FlushContext ctx) {
     initializedOperator = flushOperator;
   }
 
+  @Override
+  public void enterTracingOn(TracingOnContext ctx) {
+    super.enterTracingOn(ctx);
+    IoTDBDescriptor.getInstance().getConfig().setEnablePerformanceTracing(true);

Review comment:
       I suggest adding a parameter boolean in TracingOperator and TracingPlan, then set the config in PlanExecutor.

##########
File path: server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
##########
@@ -524,10 +546,30 @@ public TSExecuteStatementResp executeQueryStatement(TSExecuteStatementReq req) {
    *             AuthorPlan
    */
   private TSExecuteStatementResp internalExecuteQueryStatement(String statement,
-      long statementId, PhysicalPlan plan, int fetchSize, String username) {
+      long statementId, PhysicalPlan plan, int fetchSize, String username) throws IOException {
     auditLogger.info("Session {} execute Query: {}", currSessionId.get(), statement);
     long startTime = System.currentTimeMillis();
     long queryId = -1;
+    if (plan instanceof QueryPlan && config.isEnablePerformanceTracing()) {
+      File performanceDir = SystemFileFactory.INSTANCE.getFile(config.getPerformanceDir());

Review comment:
       put all these performance related to a class TracingManager, which manage a writer, log file.

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
##########
@@ -185,6 +184,11 @@
    */
   private String syncDir = "data" + File.separator + "system" + File.separator + "sync";
 
+  /**
+   * Performance tracing directory, stores performance tracing files
+   */
+  private String performanceDir = "data" + File.separator + "system" + File.separator + "performance";

Review comment:
       put this to logs folder or data/performance is better

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
##########
@@ -125,24 +138,23 @@ private IoTDBConfigCheck() {
     systemProperties.put(PARTITION_INTERVAL_STRING, String.valueOf(partitionInterval));
     systemProperties.put(TSFILE_FILE_SYSTEM_STRING, tsfileFileSystem);
     systemProperties.put(ENABLE_PARTITION_STRING, String.valueOf(enablePartition));
+    systemProperties.put(ENABLE_PERFORMANCE_TRACING, String.valueOf(enablePerformanceTracing));
     systemProperties.put(TAG_ATTRIBUTE_SIZE_STRING, tagAttributeTotalSize);
     systemProperties.put(MAX_DEGREE_OF_INDEX_STRING, maxDegreeOfIndexNode);
   }
 
 
   /**
    * check configuration in system.properties when starting IoTDB
-   *
+   * <p>
    * When init: create system.properties directly
-   *
-   * When upgrading the system.properties:
-   * (1) create system.properties.tmp
-   * (2) delete system.properties
-   * (2) rename system.properties.tmp to system.properties
+   * <p>
+   * When upgrading the system.properties: (1) create system.properties.tmp (2) delete

Review comment:
       format all changes of this class back..

##########
File path: server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
##########
@@ -251,6 +257,13 @@ private void unpackAllOverlappedTimeSeriesMetadataToCachedChunkMetadata(
       unpackOneTimeSeriesMetadata(firstTimeSeriesMetadata);
       firstTimeSeriesMetadata = null;
     }
+
+    // try to calculate the total number of chunk and time-value points in chunk
+    if (IoTDBDescriptor.getInstance().getConfig().isEnablePerformanceTracing()) {
+      totalChunkNum += cachedChunkMetadata.size();

Review comment:
       This method may be called many times before the cachedChunkMetadata is all consumed. You can put this tracing in unpackOneTimeSeriesMetadata, i.e., when we call FileLoaderUtils.

##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1233,6 +1237,20 @@ public QueryDataSource query(String deviceId, String measurementId, QueryContext
       if (filePathsManager != null) {
         filePathsManager.addUsedFilesForQuery(context.getQueryId(), dataSource);
       }
+
+      // exclude repetitive tsFile to calculate the number of it
+      if (config.isEnablePerformanceTracing()) {
+        if (seqFile == null) {
+          seqFile = new HashSet<>(seqResources);

Review comment:
       If I query multiple times, when do you clear this?

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
##########
@@ -80,11 +87,16 @@
 
   private static final String ERROR_LOG = "Wrong %s, please set as: %s !";
 
+  private static final String ENABLE_PERFORMANCE_TRACING = "enable_performance_tracing";

Review comment:
       this is not an invariant parameter, no need to put it here

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
##########
@@ -125,24 +138,23 @@ private IoTDBConfigCheck() {
     systemProperties.put(PARTITION_INTERVAL_STRING, String.valueOf(partitionInterval));
     systemProperties.put(TSFILE_FILE_SYSTEM_STRING, tsfileFileSystem);
     systemProperties.put(ENABLE_PARTITION_STRING, String.valueOf(enablePartition));
+    systemProperties.put(ENABLE_PERFORMANCE_TRACING, String.valueOf(enablePerformanceTracing));

Review comment:
       do not modify this class

##########
File path: server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
##########
@@ -223,6 +226,20 @@ public void enterFlush(FlushContext ctx) {
     initializedOperator = flushOperator;
   }
 
+  @Override
+  public void enterTracingOn(TracingOnContext ctx) {
+    super.enterTracingOn(ctx);
+    IoTDBDescriptor.getInstance().getConfig().setEnablePerformanceTracing(true);

Review comment:
       I suggest adding a parameter boolean in TracingOperator and TracingPlan, then set the config in PlanExecutor.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org