-
Notifications
You must be signed in to change notification settings - Fork 3.8k
[Groupby Query Metrics] Add merge buffer tracking #18731
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
21004b4
c935ea6
c781910
7063d09
19f6bc3
0fcb6a0
25f10d2
b6ad3c2
28719eb
59fe03c
e6020a6
507eecd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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| | ||
| |`mergeBuffer/maxBytesUsed`|Maximum number of bytes used by merge buffers for any single groupBy query within the emission period.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies| | ||
| |`groupBy/spilledQueries`|Number of groupBy queries that have spilled onto the disk.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies| | ||
| |`groupBy/spilledBytes`|Number of bytes spilled on the disk by the groupBy queries.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies| | ||
| |`groupBy/maxSpilledBytes`|Maximum number of bytes spilled to disk by any single groupBy query within the emission period.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies| | ||
| |`groupBy/mergeDictionarySize`|Size of on-heap merge dictionary in bytes.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies| | ||
| |`groupBy/maxMergeDictionarySize`|Maximum size of the on-heap merge dictionary in bytes observed for any single groupBy query within the emission period.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies| | ||
|
|
||
| ### Historical | ||
|
|
||
|
|
@@ -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| | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ditto here and below |
||
| |`mergeBuffer/maxBytesUsed`|Maximum number of bytes used by merge buffers for any single groupBy query within the emission period.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies| | ||
| |`groupBy/spilledQueries`|Number of groupBy queries that have spilled onto the disk.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies| | ||
| |`groupBy/spilledBytes`|Number of bytes spilled on the disk by the groupBy queries.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies| | ||
| |`groupBy/maxSpilledBytes`|Maximum number of bytes spilled to disk by any single groupBy query within the emission period.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies| | ||
| |`groupBy/mergeDictionarySize`|Size of on-heap merge dictionary in bytes.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies| | ||
| |`groupBy/maxMergeDictionarySize`|Maximum size of the on-heap merge dictionary in bytes observed for any single groupBy query within the emission period.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies| | ||
|
|
||
| ### Real-time | ||
|
|
||
|
|
@@ -140,9 +150,14 @@ to represent the task ID are deprecated and will be removed in a future release. | |
| |`mergeBuffer/used`|Number of merge buffers used from the merge buffer pool. This metric is only available if the `GroupByStatsMonitor` module is included.|`dataSource`, `taskId`|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.|`dataSource`, `taskId`|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.|`dataSource`, `taskId`|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.|`dataSource`, `taskId`|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.|`dataSource`, `taskId`|Varies| | ||
| |`mergeBuffer/maxBytesUsed`|Maximum number of bytes used by merge buffers for any single groupBy query within the emission period. This metric is only available if the `GroupByStatsMonitor` module is included.|`dataSource`, `taskId`|Varies| | ||
| |`groupBy/spilledQueries`|Number of groupBy queries that have spilled onto the disk. This metric is only available if the `GroupByStatsMonitor` module is included.|`dataSource`, `taskId`|Varies| | ||
| |`groupBy/spilledBytes`|Number of bytes spilled on the disk by the groupBy queries. This metric is only available if the `GroupByStatsMonitor` module is included.|`dataSource`, `taskId`|Varies| | ||
| |`groupBy/maxSpilledBytes`|Maximum number of bytes spilled to disk by any single groupBy query within the emission period. This metric is only available if the `GroupByStatsMonitor` module is included.|`dataSource`, `taskId`|Varies| | ||
| |`groupBy/mergeDictionarySize`|Size of on-heap merge dictionary in bytes. This metric is only available if the `GroupByStatsMonitor` module is included.|`dataSource`, `taskId`|Varies| | ||
| |`groupBy/maxMergeDictionarySize`|Maximum size of the on-heap merge dictionary in bytes observed for any single groupBy query within the emission period. This metric is only available if the `GroupByStatsMonitor` module is included.|`dataSource`, `taskId`|Varies| | ||
|
|
||
| ### Jetty | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,7 +27,8 @@ | |
| import java.util.concurrent.atomic.AtomicLong; | ||
|
|
||
| /** | ||
| * Metrics collector for groupBy queries like spilled bytes, merge buffer acquistion time, dictionary size. | ||
| * Collects groupBy query metrics (spilled bytes, merge buffer usage, dictionary size) per-query, then | ||
| * aggregates them when queries complete. Stats are retrieved and reset periodically via {@link #getStatsSince()}. | ||
| */ | ||
| @LazySingleton | ||
| public class GroupByStatsProvider | ||
|
|
@@ -60,34 +61,67 @@ public synchronized void closeQuery(QueryResourceId resourceId) | |
|
|
||
| public synchronized AggregateStats getStatsSince() | ||
| { | ||
| return aggregateStatsContainer.reset(); | ||
| AggregateStats aggregateStats = new AggregateStats(aggregateStatsContainer); | ||
| aggregateStatsContainer.reset(); | ||
| return aggregateStats; | ||
| } | ||
|
|
||
| public static class AggregateStats | ||
| { | ||
| private long mergeBufferQueries = 0; | ||
| private long mergeBufferAcquisitionTimeNs = 0; | ||
| private long mergeBufferTotalUsage = 0; | ||
| private long maxMergeBufferAcquisitionTimeNs = 0; | ||
| private long maxMergeBufferUsage = 0; | ||
| private long spilledQueries = 0; | ||
| private long spilledBytes = 0; | ||
| private long maxSpilledBytes = 0; | ||
| private long mergeDictionarySize = 0; | ||
| private long maxMergeDictionarySize = 0; | ||
|
|
||
| public AggregateStats() | ||
| { | ||
| } | ||
|
|
||
| public AggregateStats(AggregateStats aggregateStats) | ||
| { | ||
| this( | ||
| aggregateStats.mergeBufferQueries, | ||
| aggregateStats.mergeBufferAcquisitionTimeNs, | ||
| aggregateStats.mergeBufferTotalUsage, | ||
| aggregateStats.maxMergeBufferAcquisitionTimeNs, | ||
| aggregateStats.maxMergeBufferUsage, | ||
| aggregateStats.spilledQueries, | ||
| aggregateStats.spilledBytes, | ||
| aggregateStats.maxSpilledBytes, | ||
| aggregateStats.mergeDictionarySize, | ||
| aggregateStats.maxMergeDictionarySize | ||
| ); | ||
| } | ||
|
|
||
| public AggregateStats( | ||
| long mergeBufferQueries, | ||
| long mergeBufferAcquisitionTimeNs, | ||
| long mergeBufferTotalUsage, | ||
| long maxMergeBufferAcquisitionTimeNs, | ||
| long maxMergeBufferUsage, | ||
| long spilledQueries, | ||
| long spilledBytes, | ||
| long mergeDictionarySize | ||
| long maxSpilledBytes, | ||
| long mergeDictionarySize, | ||
| long maxMergeDictionarySize | ||
| ) | ||
| { | ||
| this.mergeBufferQueries = mergeBufferQueries; | ||
| this.mergeBufferAcquisitionTimeNs = mergeBufferAcquisitionTimeNs; | ||
| this.mergeBufferTotalUsage = mergeBufferTotalUsage; | ||
| this.maxMergeBufferAcquisitionTimeNs = maxMergeBufferAcquisitionTimeNs; | ||
| this.maxMergeBufferUsage = maxMergeBufferUsage; | ||
| this.spilledQueries = spilledQueries; | ||
| this.spilledBytes = spilledBytes; | ||
| this.maxSpilledBytes = maxSpilledBytes; | ||
| this.mergeDictionarySize = mergeDictionarySize; | ||
| this.maxMergeDictionarySize = maxMergeDictionarySize; | ||
| } | ||
|
|
||
| public long getMergeBufferQueries() | ||
|
|
@@ -100,6 +134,21 @@ public long getMergeBufferAcquisitionTimeNs() | |
| return mergeBufferAcquisitionTimeNs; | ||
| } | ||
|
|
||
| public long getMergeBufferTotalUsage() | ||
| { | ||
| return mergeBufferTotalUsage; | ||
| } | ||
|
|
||
| public long getMaxMergeBufferAcquisitionTimeNs() | ||
| { | ||
| return maxMergeBufferAcquisitionTimeNs; | ||
| } | ||
|
|
||
| public long getMaxMergeBufferUsage() | ||
| { | ||
| return maxMergeBufferUsage; | ||
| } | ||
|
|
||
| public long getSpilledQueries() | ||
| { | ||
| return spilledQueries; | ||
|
|
@@ -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 | ||
| ); | ||
|
|
||
|
Comment on lines
-133
to
-143
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Any reason to change this contract? I see the caller now calls
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Actually on second thought, I think the new contract |
||
| this.mergeBufferQueries = 0; | ||
| this.mergeBufferAcquisitionTimeNs = 0; | ||
| this.mergeBufferTotalUsage = 0; | ||
| this.maxMergeBufferAcquisitionTimeNs = 0; | ||
| this.maxMergeBufferUsage = 0; | ||
| this.spilledQueries = 0; | ||
| this.spilledBytes = 0; | ||
| this.maxSpilledBytes = 0; | ||
| this.mergeDictionarySize = 0; | ||
|
|
||
| return aggregateStats; | ||
| this.maxMergeDictionarySize = 0; | ||
| } | ||
| } | ||
|
|
||
| public static class PerQueryStats | ||
| { | ||
| private final AtomicLong mergeBufferAcquisitionTimeNs = new AtomicLong(0); | ||
| private final AtomicLong mergeBufferTotalUsage = new AtomicLong(0); | ||
| private final AtomicLong spilledBytes = new AtomicLong(0); | ||
| private final AtomicLong mergeDictionarySize = new AtomicLong(0); | ||
|
|
||
|
|
@@ -162,6 +224,11 @@ public void mergeBufferAcquisitionTime(long delay) | |
| mergeBufferAcquisitionTimeNs.addAndGet(delay); | ||
| } | ||
|
|
||
| public void mergeBufferTotalUsage(long bytes) | ||
| { | ||
| mergeBufferTotalUsage.addAndGet(bytes); | ||
| } | ||
|
|
||
| public void spilledBytes(long bytes) | ||
| { | ||
| spilledBytes.addAndGet(bytes); | ||
|
|
@@ -177,6 +244,11 @@ public long getMergeBufferAcquisitionTimeNs() | |
| return mergeBufferAcquisitionTimeNs.get(); | ||
| } | ||
|
|
||
| public long getMergeBufferTotalUsage() | ||
| { | ||
| return mergeBufferTotalUsage.get(); | ||
| } | ||
|
|
||
| public long getSpilledBytes() | ||
| { | ||
| return spilledBytes.get(); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: move this after
mergeBuffer/acquisitionTimeNssimilar to the other corresponding max metrics