gortiz commented on code in PR #16576:
URL: https://github.com/apache/pinot/pull/16576#discussion_r2336633904


##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultiStageOperator.java:
##########
@@ -220,6 +229,25 @@ protected Map<String, Plan.ExplainNode.AttributeValue> 
getExplainAttributes() {
     return Collections.emptyMap();
   }
 
+  private long getGcTimeMillis() {
+    if (!isCollectGcStats(_context.getOpChainMetadata())) {
+      return -1;
+    }
+    long totalGCTime = 0;
+    List<GarbageCollectorMXBean> gcBeans = 
ManagementFactory.getGarbageCollectorMXBeans();
+    for (GarbageCollectorMXBean gcBean : gcBeans) {
+      long gcTime = gcBean.getCollectionTime();
+      if (gcTime > 0) {
+        totalGCTime += gcTime;
+      }
+    }
+    return totalGCTime;
+  }
+
+  private static boolean isCollectGcStats(Map<String, String> opChainMetadata) 
{

Review Comment:
   Removed



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/pipeline/PipelineBreakerOperator.java:
##########
@@ -152,7 +154,25 @@ protected StatMap<?> copyStatMaps() {
 
   public enum StatKey implements StatMap.Key {
     EXECUTION_TIME_MS(StatMap.Type.LONG),
-    EMITTED_ROWS(StatMap.Type.LONG);
+    EMITTED_ROWS(StatMap.Type.LONG),
+    /**
+     * Allocated memory in bytes for this operator or its children in the same 
stage.
+     */
+    ALLOCATED_MEMORY_BYTES(StatMap.Type.LONG) {
+      @Override
+      public boolean includeDefaultInJson() {
+        return true;
+      }
+    },
+    /**
+     * Time spent on GC while this operator or its children in the same stage 
were running.
+     */
+    GC_TIME_MS(StatMap.Type.LONG) {
+      @Override
+      public boolean includeDefaultInJson() {
+        return true;
+      }
+    };

Review Comment:
   I think this was an unintended change, probably introduced by the tooling 
(the ide or copilot) when autocompleting.



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/LeafOperator.java:
##########
@@ -714,15 +716,32 @@ public long merge(long value1, long value2) {
       public String getStatName() {
         return "responseSerializationCpuTimeNs";
       }
-    };
+    },
+    /**
+     * Allocated memory in bytes for this operator or its children in the same 
stage.
+     */
+    ALLOCATED_MEMORY_BYTES(StatMap.Type.LONG, null),
+    /**
+     * Time spent on GC while this operator or its children in the same stage 
were running.
+     */
+    GC_TIME_MS(StatMap.Type.LONG, null);
+    // IMPORTANT: When adding new StatKeys, make sure to either create the 
same key in BrokerResponseNativeV2.KeyStat or
+    //  call the constructor that accepts a String as last argument and set it 
to null.
+    //  Otherwise the constructor will fail with an IllegalArgumentException 
which will not be caught and will
+    //  propagate to the caller, causing the query to timeout.

Review Comment:
   Because the exception won't be caught, it will end up killing the task that 
executed the opchain. This means that this opchain will never send an EOS 
block, which in turn means that the parent tasks will never finish. 
Consequently, the query will end up timing out.
   
   I was in fact affected by this issue when testing this PR, and I didn't find 
a way to fail in a better way, so I decided to add this commen,t hoping this 
will help future contributors (or a future me :laughing: )



##########
pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultiStageOperator.java:
##########
@@ -220,6 +229,25 @@ protected Map<String, Plan.ExplainNode.AttributeValue> 
getExplainAttributes() {
     return Collections.emptyMap();
   }
 
+  private long getGcTimeMillis() {

Review Comment:
   Moved to ThreadResourceUsageProvider



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to