Skip to content

Conversation

@GWphua
Copy link
Contributor

@GWphua GWphua commented Nov 10, 2025

Fixes #17902

Huge thanks to @gianm for the implementation tip in the issue!

Description

Tracking merge buffer usage

  • Usage of a direct byte buffer is done under AbstractBufferHashGrouper and its implementations.
  1. Each direct byte buffer uses a ByteBufferHashTable along with an offset tracker.
  2. Usage is calculated by tracking the maximum capacity of the byte buffer in ByteBufferHashTable, and maximum offset size calculated throughout the query's lifecycle.

Incoprated a helpful suggestion by @aho135 : since the size of the hash tables are ever-changing, it makes sense to conduct calculations by taking the maximum values across queries -- so operators can have a better understanding of how the size of merge buffers can be configured.

Here's an example of the current SUM implementations, vs the MAX implementation The latter helps to tell us that we should probably configure merge buffer sizes to 2G for this case:
image

Release note

GroupByStatsMonitor now provides metrics "mergeBuffer/bytesUsed", and max metrics for merge buffer acquisition time, bytes used, spilled bytes, and merge dictionary size.


Key changed/added classes in this PR
  • GroupByStatsProvider
  • Groupers + underlying ByteBuffer table/lists.

This PR has:

  • been self-reviewed.
  • a release note entry in the PR description.
  • added unit tests or modified existing tests to cover new code paths, ensuring the threshold for code coverage is met.
  • been tested in a test Druid cluster.

Possible further enhancements

While building this PR, I have come across some problems which we can further enhance in the future:

Nested Group-bys

The current metric is great, but will not report accurately for nested group-by's. (Do correct me on this if I'm mistaken though!)

As far as I know, nested groupby limits the merge buffers usage count to 2, meaning the merge buffer will be re-used. IIUC, every ConcurrentGrouper (if concurrency is enabled) / SpillingGrouper (if concurrency disabled) is created and closed multiple times, and hence a per-query metric will likely over-report the merge buffer usage.

Simplify Memory Management

Right now we need to configure the following for each queryable service:

  1. size of merge buffer
  2. number of merge buffer
  3. direct memory = (numProcessingThreads + numMergeBuffer + 1) * mergeBufferSizeBytes

It will be great if we can simplify the calculations down to simply configuring direct memory, and we can manage a memory pool instead. This allows for more flexibility (unused memory allocated for merge buffers may be used by processing threads instead).

@GWphua GWphua requested a review from gianm November 10, 2025 03:18
if (perQueryStats.getMergeBufferAcquisitionTimeNs() > 0) {
mergeBufferQueries++;
mergeBufferAcquisitionTimeNs += perQueryStats.getMergeBufferAcquisitionTimeNs();
mergeBufferTotalUsage += perQueryStats.getMergeBufferTotalUsage();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@GWphua Instead of summing here, what do you think about taking the max? Then the metric emitted would be the max merge buffer usage of a single query in that emission period. This would be a good signal for operators on whether they need to tweak the mergeBuffer size.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I was thinking about this.

There are other metrics where taking MAX will also make sense --
spilledBytes --> How much storage would be good to configure?
dictionarySize --> How large can the merge dictionary size get?

I am considering adding another metric (maxSpilledBytes, maxDictionarySize, maxSpilledBytes). What do you think?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah agreed. I do think it makes sense to have it for those 3 metrics

Even for mergeBuffer/acquisitionTimeNs I think there's value in having the max, as it gives operators a signal on whether to increase numMergeBuffers

@aho135
Copy link
Contributor

aho135 commented Nov 24, 2025

Thanks for adding those max metrics @GWphua!

What do you think about adding sqlQueryId as a dimension only for the MAX metrics? I think this would be useful for understanding how much the query execution time was affected by the mergeBufferAcquisition. Can also do this in a follow-up PR if you think it's useful.

@GWphua
Copy link
Contributor Author

GWphua commented Nov 25, 2025

Hi @aho135

Thanks for the review! I also find that it will be very helpful to emit metrics for each query, so we know which query will take up alot of resources. In our version of Druid, we simply appended each of the PerQueryStat to the statsMap in QueryLifecycle#emitLogsAndMetrics, but I feel its quite a hacky way of doing it. sqlQueryId as a dimension in the GroupByStatsMonitor will definitely help.

Alternatively, we can look into migrating the groupBy query metrics in GroupByStatsMonitor to GroupByQueryMetrics, which should emit metrics for each GroupBy query. In that way, this can make the MAX and SUM metrics redundant as we can now emit metrics for each query.

We can do more of this in a seperate PR.

@aho135
Copy link
Contributor

aho135 commented Nov 25, 2025

Hi @aho135

Thanks for the review! I also find that it will be very helpful to emit metrics for each query, so we know which query will take up alot of resources. In our version of Druid, we simply appended each of the PerQueryStat to the statsMap in QueryLifecycle#emitLogsAndMetrics, but I feel its quite a hacky way of doing it. sqlQueryId as a dimension in the GroupByStatsMonitor will definitely help.

Alternatively, we can look into migrating the groupBy query metrics in GroupByStatsMonitor to GroupByQueryMetrics, which should emit metrics for each GroupBy query. In that way, this can make the MAX and SUM metrics redundant as we can now emit metrics for each query.

We can do more of this in a seperate PR.

Sounds good @GWphua I was thinking on very similar lines to emit these from GroupByQueryMetrics

I have a first draft on this: aho135@9f82091
Lmk if you have any thoughts on this. Thanks!

@GWphua
Copy link
Contributor Author

GWphua commented Nov 26, 2025

Hi @aho135, since the scope of adding GroupByQueryMetrics is out of this PR, I have created #18781 to allow us to further discuss it there.

I have a first draft on this: aho135@9f82091
Lmk if you have any thoughts on this. Thanks!

I have a draft for GroupByQueryMetrics before creating this PR, and my draft is a direct extension of your implementation shared. I think I will try and create a PR with that draft soon. I was actually hoping to get this PR merged, before sharing the draft, because that draft is done as a follow-up to this PR.

@GWphua
Copy link
Contributor Author

GWphua commented Dec 31, 2025

Hi @gianm, would appreciate it if I receive a review/feedback on this PR. Thanks!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Add metric for merge buffers usage in bytes

2 participants