abhishekrb19 commented on code in PR #18731:
URL: https://github.com/apache/druid/pull/18731#discussion_r2678248662


##########
processing/src/test/java/org/apache/druid/query/groupby/GroupByStatsProviderTest.java:
##########
@@ -63,6 +66,7 @@ public void testMetricCollection()
     aggregateStats = statsProvider.getStatsSince();
     Assert.assertEquals(2, aggregateStats.getMergeBufferQueries());
     Assert.assertEquals(1800L, 
aggregateStats.getMergeBufferAcquisitionTimeNs());
+    Assert.assertEquals(150L, aggregateStats.getMergeBufferTotalUsage());

Review Comment:
   Should we also add verify the new maxMergeBufferAcquisitionTimeNs, 
maxMergeBufferUsage, etc here?



##########
processing/src/test/java/org/apache/druid/query/groupby/GroupByStatsProviderTest.java:
##########
@@ -45,6 +46,7 @@ public void testMetricCollection()
 
     stats2.mergeBufferAcquisitionTime(500);
     stats2.mergeBufferAcquisitionTime(600);
+    stats1.mergeBufferTotalUsage(100);

Review Comment:
   nit: did you intend to mix stats1 operation with stats2? If not, please move 
this above these stats2 block



##########
processing/src/main/java/org/apache/druid/query/groupby/GroupByStatsProvider.java:
##########
@@ -110,50 +159,63 @@ public long getSpilledBytes()
       return spilledBytes;
     }
 
+    public long getMaxSpilledBytes()
+    {
+      return maxSpilledBytes;
+    }
+
     public long getMergeDictionarySize()
     {
       return mergeDictionarySize;
     }
 
+    public long getMaxMergeDictionarySize()
+    {
+      return maxMergeDictionarySize;
+    }
+
     public void addQueryStats(PerQueryStats perQueryStats)
     {
       if (perQueryStats.getMergeBufferAcquisitionTimeNs() > 0) {
         mergeBufferQueries++;
         mergeBufferAcquisitionTimeNs += 
perQueryStats.getMergeBufferAcquisitionTimeNs();
+        maxMergeBufferAcquisitionTimeNs = Math.max(
+            maxMergeBufferAcquisitionTimeNs,
+            perQueryStats.getMergeBufferAcquisitionTimeNs()
+        );
+        mergeBufferTotalUsage += perQueryStats.getMergeBufferTotalUsage();
+        maxMergeBufferUsage = Math.max(maxMergeBufferUsage, 
perQueryStats.getMergeBufferTotalUsage());
       }
 
       if (perQueryStats.getSpilledBytes() > 0) {
         spilledQueries++;
         spilledBytes += perQueryStats.getSpilledBytes();
+        maxSpilledBytes = Math.max(maxSpilledBytes, 
perQueryStats.getSpilledBytes());
       }
 
       mergeDictionarySize += perQueryStats.getMergeDictionarySize();
+      maxMergeDictionarySize = Math.max(maxMergeDictionarySize, 
perQueryStats.getMergeDictionarySize());
     }
 
-    public AggregateStats reset()
+    public void reset()
     {
-      AggregateStats aggregateStats =
-          new AggregateStats(
-              mergeBufferQueries,
-              mergeBufferAcquisitionTimeNs,
-              spilledQueries,
-              spilledBytes,
-              mergeDictionarySize
-          );
-

Review Comment:
   Any reason to change this contract? I see the caller now calls 
`AggregateStats()` explicitly before `reset()` to save the state, I think the 
old approach was better.



##########
processing/src/test/java/org/apache/druid/query/groupby/GroupByStatsProviderTest.java:
##########
@@ -63,6 +66,7 @@ public void testMetricCollection()
     aggregateStats = statsProvider.getStatsSince();
     Assert.assertEquals(2, aggregateStats.getMergeBufferQueries());
     Assert.assertEquals(1800L, 
aggregateStats.getMergeBufferAcquisitionTimeNs());
+    Assert.assertEquals(150L, aggregateStats.getMergeBufferTotalUsage());

Review Comment:
   Also, I think it's worth adding a test to validate these max metrics 
explicitly. For example, three resources r1, r2 and r3 with different spilled 
bytes, acquisition times, etc.



##########
docs/operations/metrics.md:
##########
@@ -113,9 +118,14 @@ Most metric values reset each emission period, as 
specified in `druid.monitoring
 |`mergeBuffer/used`|Number of merge buffers used from the merge buffer 
pool.|This metric is only available if the `GroupByStatsMonitor` module is 
included.|Depends on the number of groupBy queries needing merge buffers.|
 |`mergeBuffer/queries`|Number of groupBy queries that acquired a batch of 
buffers from the merge buffer pool.|This metric is only available if the 
`GroupByStatsMonitor` module is included.|Depends on the number of groupBy 
queries needing merge buffers.|
 |`mergeBuffer/acquisitionTimeNs`|Total time in nanoseconds to acquire merge 
buffer for groupBy queries.|This metric is only available if the 
`GroupByStatsMonitor` module is included.|Varies|
+|`mergeBuffer/bytesUsed`|Number of bytes used by merge buffers to process 
groupBy queries.|This metric is only available if the `GroupByStatsMonitor` 
module is included.|Varies|
+|`mergeBuffer/maxAcquisitionTimeNs`|Maximum time in nanoseconds to acquire 
merge buffer for any single groupBy query within the emission period.|This 
metric is only available if the `GroupByStatsMonitor` module is 
included.|Varies|

Review Comment:
   ditto here and below



##########
docs/operations/metrics.md:
##########
@@ -89,9 +89,14 @@ Most metric values reset each emission period, as specified 
in `druid.monitoring
 |`mergeBuffer/used`|Number of merge buffers used from the merge buffer 
pool.|This metric is only available if the `GroupByStatsMonitor` module is 
included.|Depends on the number of groupBy queries needing merge buffers.|
 |`mergeBuffer/queries`|Number of groupBy queries that acquired a batch of 
buffers from the merge buffer pool.|This metric is only available if the 
`GroupByStatsMonitor` module is included.|Depends on the number of groupBy 
queries needing merge buffers.|
 |`mergeBuffer/acquisitionTimeNs`|Total time in nanoseconds to acquire merge 
buffer for groupBy queries.|This metric is only available if the 
`GroupByStatsMonitor` module is included.|Varies|
+|`mergeBuffer/bytesUsed`|Number of bytes used by merge buffers to process 
groupBy queries.|This metric is only available if the `GroupByStatsMonitor` 
module is included.|Varies|
+|`mergeBuffer/maxAcquisitionTimeNs`|Maximum time in nanoseconds to acquire 
merge buffer for any single groupBy query within the emission period.|This 
metric is only available if the `GroupByStatsMonitor` module is 
included.|Varies|

Review Comment:
   nit: move this after `mergeBuffer/acquisitionTimeNs` similar to the other 
corresponding max metrics



##########
server/src/test/java/org/apache/druid/server/metrics/GroupByStatsMonitorTest.java:
##########
@@ -128,11 +134,17 @@ public void testMonitorWithServiceDimensions()
     verifyMetricValue(emitter, "mergeBuffer/used", dimFilters, 0L);
     verifyMetricValue(emitter, "mergeBuffer/queries", dimFilters, 1L);
     verifyMetricValue(emitter, "mergeBuffer/acquisitionTimeNs", dimFilters, 
100L);
+    verifyMetricValue(emitter, "mergeBuffer/bytesUsed", dimFilters, 200L);
+    verifyMetricValue(emitter, "mergeBuffer/maxAcquisitionTimeNs", dimFilters, 
100L);
+    verifyMetricValue(emitter, "mergeBuffer/maxBytesUsed", dimFilters, 200L);
     verifyMetricValue(emitter, "groupBy/spilledQueries", dimFilters, 2L);
     verifyMetricValue(emitter, "groupBy/spilledBytes", dimFilters, 200L);
+    verifyMetricValue(emitter, "groupBy/maxSpilledBytes", dimFilters, 200L);
     verifyMetricValue(emitter, "groupBy/mergeDictionarySize", dimFilters, 
300L);
+    verifyMetricValue(emitter, "groupBy/maxMergeDictionarySize", dimFilters, 
300L);

Review Comment:
    I think it will be nice to have a separate test that mimics multiple 
resource IDs - something that doesn't use the static `AggregateStats 
getStatsSince()`  defined in the test setup.  The existing tests can continue 
to the setup for convenience.
   
   A new test could leverage the provider's methods on different resource IDs, 
similar to what the `GroupByMergingQueryRunner` would actually do:
   - `groupByStatsProvider.getPerQueryStatsContainer()` 
   -  `groupByStatsProvider.closeQuery()`
   
   
   Please let me know what you think.
   



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