Skip to content

Emit disk spill and merge buffer utilisation metrics for GroupBy queries#17360

Merged
LakshSingla merged 18 commits intoapache:masterfrom
findingrish:spill-metrics
Nov 22, 2024
Merged

Emit disk spill and merge buffer utilisation metrics for GroupBy queries#17360
LakshSingla merged 18 commits intoapache:masterfrom
findingrish:spill-metrics

Conversation

@findingrish
Copy link
Copy Markdown
Contributor

@findingrish findingrish commented Oct 16, 2024

This change is to emit following metrics as part of GroupByStatsMonitor monitor,
mergeBuffer/used -> Number of merge buffers used.
mergeBuffer/acquisitionTimeNs -> Total time required to acquire merge buffer.
mergeBuffer/acquisition -> Number of queries that acquired a batch of merge buffers.
groupBy/spilledQueries -> Number of queries that spilled onto the disk.
groupBy/spilledBytes-> Spilled bytes on the disk.
groupBy/mergeDictionarySize -> Size of the merging dictionary.

Testing

  1. Added UT to verify that GroupByStatsMonitor is reporting the new metrics.
  2. Added UT for GroupByStatsProvider.
  3. Tested metric emission in a local druid cluster for a group by query.
{"feed":"metrics","metric":"mergeBuffer/usedCount","service":"druid/historical","host":"localhost:8083","version":"32.0.0-SNAPSHOT","value":0,"timestamp":"2024-10-24T13:09:48.884Z"}
{"feed":"metrics","metric":"mergeBuffer/acquisitionCount","service":"druid/historical","host":"localhost:8083","version":"32.0.0-SNAPSHOT","value":1,"timestamp":"2024-10-24T13:09:48.884Z"} 
{"feed":"metrics","metric":"mergeBuffer/acquisitionTimeNs","service":"druid/historical","host":"localhost:8083","version":"32.0.0-SNAPSHOT","value":67541,"timestamp":"2024-10-24T13:09:48.884Z"} 
{"feed":"metrics","metric":"groupBy/spilledQueries","service":"druid/historical","host":"localhost:8083","version":"32.0.0-SNAPSHOT","value":1,"timestamp":"2024-10-24T13:09:48.884Z"}
{"feed":"metrics","metric":"groupBy/spilledBytes","service":"druid/historical","host":"localhost:8083","version":"32.0.0-SNAPSHOT","value":367977,"timestamp":"2024-10-24T13:09:48.884Z"}
{"feed":"metrics","metric":"groupBy/mergeDictionarySize","service":"druid/historical","host":"localhost:8083","version":"32.0.0-SNAPSHOT","value":296,"timestamp":"2024-10-24T13:09:48.884Z"}
{"feed":"metrics","metric":"mergeBuffer/pendingRequests","service":"druid/historical","host":"localhost:8083","version":"32.0.0-SNAPSHOT","value":0,"timestamp":"2024-10-24T13:09:48.883Z"}

{"feed":"metrics","metric":"mergeBuffer/pendingRequests","service":"druid/broker","host":"localhost:8082","version":"32.0.0-SNAPSHOT","value":0,"timestamp":"2024-10-24T13:08:49.711Z"}
{"feed":"metrics","metric":"mergeBuffer/usedCount","service":"druid/broker","host":"localhost:8082","version":"32.0.0-SNAPSHOT","value":0,"timestamp":"2024-10-24T13:08:49.711Z"}
{"feed":"metrics","metric":"mergeBuffer/acquisitionCount","service":"druid/broker","host":"localhost:8082","version":"32.0.0-SNAPSHOT","value":1,"timestamp":"2024-10-24T13:08:49.711Z"}
{"feed":"metrics","metric":"mergeBuffer/acquisitionTimeNs","service":"druid/broker","host":"localhost:8082","version":"32.0.0-SNAPSHOT","value":2545250,"timestamp":"2024-10-24T13:08:49.711Z"}
{"feed":"metrics","metric":"groupBy/mergeDictionarySize","service":"druid/broker","host":"localhost:8082","version":"32.0.0-SNAPSHOT","value":20190,"timestamp":"2024-10-24T13:08:49.712Z"}

Release Notes

Going forward GroupByStatsMonitor monitor should be used for emitting mergeBuffer/pendingRequests metric instead of QueryCountStatsMonitor. In a subsequent release this metric will not be emitted from QueryCountStatsMonitor.

temporaryStorages.add(temporaryStorage);
}

public long getSpilledBytes()
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

What if the registered "limited temporary storage" is not closed between two invocations of the getSpilledBytes(). Would the code double-count the stored bytes? Perhaps that is the intended behavior. If that's the case, we should figure out the intent behind the metric, and take a call on whether we want to do that or not.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Regardless of this, I think that there should be metric (either this one after repurposing or a new one) which indicates the total size of the spilled data, per query. This will allow the admins to estimate whether the queries need a larger merge buffer and by how much. WDYT?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Would the code double-count the stored bytes? Perhaps that is the intended behavior.

Yes, that is the intention. It basically is reporting the amount of bytes spilled when the metric is emitted.

Regardless of this, I think that there should be metric (either this one after repurposing or a new one) which indicates the total size of the spilled data, per query. This will allow the admins to estimate whether the queries need a larger merge buffer and by how much.

Makes sense, but this would be reported at the end of every query?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Yes, the query can update the monitor at the end


private final ConcurrentLinkedQueue<LimitedTemporaryStorage> temporaryStorages;

@Inject
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

nit: We shouldn't need this

Comment on lines +36 to +37
private final AtomicLong resourceAcquisitionTimeNs = new AtomicLong(0);
private final AtomicLong resourceAcquisitionCount = new AtomicLong(0);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Suggested change
private final AtomicLong resourceAcquisitionTimeNs = new AtomicLong(0);
private final AtomicLong resourceAcquisitionCount = new AtomicLong(0);
private final AtomicLong mergeBufferAcquisitionTimeNs = new AtomicLong(0);
private final AtomicLong mergeBufferAcquisitionCount = new AtomicLong(0);

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Making them atomic seems redundant given that both need to be updated together, and guaranteeing atomicity over a single variable is not useful. The current code does not rely on the AtomicLong part of the variables since they are already guarded by the provider's lock.

resourceAcquisitionCount.incrementAndGet();
}

public synchronized long getAndResetGroupByResourceAcquisitionStats()
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This doesn't seem correct. We are reporting the average as a metric. A better way would be to report the sum, as well as the count. This will allow a better weighted-average than directly reporting the average from the monitor. While we are amortizing a lot of metrics by emitting it from the monitor, I think it's better to report count and sum separately.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I couldn't find a way wherein I could report the sum and count together (so that any metric aggregation platform could automatically compute the averages).
Are you suggesting to report two different metrics groupBy/acquisitionTime & groupBy/acquisitionCount?

Comment thread docs/configuration/index.md Outdated
|`org.apache.druid.server.metrics.HistoricalMetricsMonitor`|Reports statistics on Historical services. Available only on Historical services.|
|`org.apache.druid.server.metrics.SegmentStatsMonitor` | **EXPERIMENTAL** Reports statistics about segments on Historical services. Available only on Historical services. Not to be used when lazy loading is configured.|
|`org.apache.druid.server.metrics.QueryCountStatsMonitor`|Reports how many queries have been successful/failed/interrupted.|
|`org.apache.druid.server.metrics.QueryCountStatsMonitor`|Reports how many queries have been successful/failed/interrupted. It also reports stats for group by queries.|
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Suggested change
|`org.apache.druid.server.metrics.QueryCountStatsMonitor`|Reports how many queries have been successful/failed/interrupted. It also reports stats for group by queries.|
|`org.apache.druid.server.metrics.QueryCountStatsMonitor`|Reports how many queries have been successful/failed/interrupted. It also reports statistics for the group by queries.|

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Should it be made as a separate monitor? I don't see any downside of merging the QueryCountStatsMonitor with the group by statistics, but at the same time, I don't think there's any benefit to it.
However, merging group-by statistics with "query count stats" feels incongruent to me.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I don't have strong opinions either, the only reason I merged them was an existing mergeBuffer metric in this monitor. I think a separate monitor would be better due to the added flexibility.

/**
* @return number of used buffers from the pool
*/
long getUsedBufferCount();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

BlockingPool is generic. It shouldn't reference buffers.

Suggested change
long getUsedBufferCount();
long getUsedResourcesCount();

}
finally {
pendingRequests.incrementAndGet();
pendingRequests.decrementAndGet();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

nice catch 💯

return average;
}

public void registerTemporaryStorage(LimitedTemporaryStorage temporaryStorage)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

How often does this method need to be called per group by query, and what does this rely upon? I feel that there's an inbuilt expense with tracking the spilled bytes this way. I think it is still fine, but I wonder if a more efficient way would be as follows:

  1. Have a running counter of the size occupied
  2. LimitedTemporaryStorage implements closeable. Modify wherever the LimitedTemporaryStorage is getting closed to subtract the value of the storage from the counter. There can be a set inclusion check where a hashset of the opened temporary storages being tracked by the running counter are stored, and the code confirms that the removed temporary storage was indeed being tracked by the counter.

Note: My approach seems more convoluted, so if there isn't much performance downside of the current version, I think its fine as is.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

there's an inbuilt expense with tracking the spilled bytes this way

What is the expense you are referring to? The temporary storage object is added to a queue and cost of adding an element should be in most cases should be O(1).

LimitedTemporaryStorage implements closeable. Modify wherever the LimitedTemporaryStorage is getting closed to subtract the value of the storage from the counter. There can be a set inclusion check where a hashset of the opened temporary storages being tracked by the running counter are stored, and the code confirms that the removed temporary storage was indeed being tracked by the counter.

This approach also requires keeping track of the temporary storages thus incurring similar overhead as the existing implementation. Additionally, it requires incrementing the "running counter" whenever bytes are written out to the disk.

Copy link
Copy Markdown
Contributor

@LakshSingla LakshSingla Oct 18, 2024

Choose a reason for hiding this comment

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

What is the expense you are referring to

Iterating while calculating the stored bytes.

keeping track of the temporary storages

That is true, but calculating the stored bytes won't require an iteration over the stored memories.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Iterating while calculating the stored bytes.

I understand. Given that the monitor will run at a fixed interval (every minute or so), is the overhead significant?
I am weighing the implementation complexity against the performance impact.

Comment thread docs/configuration/index.md Outdated
|`org.apache.druid.server.metrics.TaskSlotCountStatsMonitor`|Reports metrics about task slot usage per emission period.|
|`org.apache.druid.server.metrics.WorkerTaskCountStatsMonitor`|Reports how many ingestion tasks are currently running/pending/waiting, the number of successful/failed tasks, and metrics about task slot usage for the reporting worker, per emission period. Only supported by Middle Manager node types.|
|`org.apache.druid.server.metrics.ServiceStatusMonitor`|Reports a heartbeat for the service.|
|`org.apache.druid.server.metrics.GroupByStatsMonitor`|Report statistics for groupBy queries like disk spill, merge buffer usage. |
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

nit : disk spill can be worded better.

Suggested change
|`org.apache.druid.server.metrics.GroupByStatsMonitor`|Report statistics for groupBy queries like disk spill, merge buffer usage. |
|`org.apache.druid.server.metrics.GroupByStatsMonitor`|Reports metrics for groupBy queries like disk and merge buffer utilized by them. |

Comment thread docs/operations/metrics.md Outdated
|`subquery/fallback/unknownReason/count`|Number of subqueries which cannot be materialized as frames due other reasons.|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
|`query/rowLimit/exceeded/count`|Number of queries whose inlined subquery results exceeded the given row limit|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
|`query/byteLimit/exceeded/count`|Number of queries whose inlined subquery results exceeded the given byte limit|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `GroupByStatsMonitor` module is included.|Should be 0.|
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Recommended value doesn't seem correct. It is fine if the queries are waiting on the merge buffer even during normal operation. Can mention, ideally 0, but it is fine if not (after rewording).

Comment thread docs/operations/metrics.md Outdated
|`query/byteLimit/exceeded/count`|Number of queries whose inlined subquery results exceeded the given byte limit|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `GroupByStatsMonitor` module is included.|Should be 0.|
|`mergeBuffer/usedCount`|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/acquisitionCount`|Number of times groupBy queries acquired merge buffers.|This metric is only available if the `GroupByStatsMonitor` module is included.|Depends on the number of groupBy queries needing merge buffers.|
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Number of times groupBy queries acquired merge buffers.|

Is it cumulative or per query? A query should acquire merge buffer only once. I think if we reword the metric name or the documentation, it will be clear to the users.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

It is cumulative, I have renamed it to mergeBuffer/queries.

Comment thread docs/operations/metrics.md Outdated
|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `GroupByStatsMonitor` module is included.|Should be 0.|
|`mergeBuffer/usedCount`|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/acquisitionCount`|Number of times groupBy queries acquired merge buffers.|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.|Should be as low as possible.|
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Suggested change
|`mergeBuffer/acquisitionTimeNs`|Total time in nanoseconds to acquire merge buffer for groupBy queries.|This metric is only available if the `GroupByStatsMonitor` module is included.|Should be as low as possible.|
|`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.|

Comment thread docs/operations/metrics.md Outdated
Comment on lines +112 to +118
|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `GroupByStatsMonitor` module is included.|Should be 0.|
|`mergeBuffer/usedCount`|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/acquisitionTimeNs`|Total time in nanoseconds to acquire merge buffer for groupBy queries.|This metric is only available if the `GroupByStatsMonitor` module is included.|Should be as low as possible.|
|`mergeBuffer/acquisitionCount`|Number of times groupBy queries acquired merge buffers.|This metric is only available if the `GroupByStatsMonitor` module is included.|Depends on the number of groupBy queries needing merge buffers.|
|`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/mergeDictionarySize`|Size of on-heap merge dictionary in bytes.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies|
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I wonder if there's a way to reference all these in a single place instead. It seems wasteful that two copies of the same documentation are maintained. LMK if there are any prior metrics that can do that.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Moved them to a new section.

}
}

private void verifyMetrics(long queries, boolean skipMergeDictionary)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

What is skipMergeDictionary and when is it required?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Renamed, this is basically to skip verifying the merge dictionary size metric, since some of the queries aggregate on non-string dimensions.

Comment on lines +12452 to +12454
if (config.equals(V2_SMALL_BUFFER_CONFIG)) {
verifyMetrics(1, true);
}
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Why are the assertions being done on a single config?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This is the most suited config to verify all the metrics. For example, in config with larger buffer size it is not possible to verify disk spill metrics. I am moved this check into the method and renamed the method.

return aggregateStatsContainer.reset();
}

public static class AggregateStats
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Why are some stats AggregateStats, and some PerQuery? Wouldn't the monitor aggregate them all?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

PerQueryStats is created per query and is closed at the end of the query, AggregateStats is "reset" after each monitor cycle and all of the PerQueryStats are accumulated into AggregateStats when the query is closed.
Monitor simply reports the aggregate metrics.

/**
* Metrics collector for groupBy queries like spilled bytes, merge buffer acquistion time, dictionary size.
*/
@LazySingleton
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Should it be registered in a Module if this is a LazySingleton? (I am not very familiar, just curious if this is working as expected)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Yes, it works as expected. Guice should automatically detect classes marked with this annotation and bind them in singleton scope.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Thanks for confirming!

Comment on lines +52 to +59
public synchronized void closeQuery(QueryResourceId resourceId)
{
if (resourceId == null || !perQueryStats.containsKey(resourceId)) {
return;
}
PerQueryStats container = perQueryStats.remove(resourceId);
aggregateStatsContainer.addQueryStats(container);
}
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This is not intuitive. The callers won't remember calling closeQuery(). Is it possible to mold it as a Closer/Closable? PerQueryStats can extend Closeable, and the close can do that.

It is still not good since it doesn't seem cool to close "Stats" in first place. Perhaps renaming PerQueryStats to PerQueryStatsContainer would also help.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This is not intuitive. The callers won't remember calling closeQuery()

Even if it implements closable, it would require to be explicitly registered with the query closer?
I am doing the same thing in GroupByQueryQueryToolChest

closer.register(() -> groupByStatsProvider.closeQuery(query.context().getQueryResourceId()));

LMK what do you think?

Copy link
Copy Markdown
Contributor

@LakshSingla LakshSingla Nov 6, 2024

Choose a reason for hiding this comment

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

it would require to be explicitly registered with the query closer

Yes. Implementing the interface makes it easier for the developer to call the closer. I have had experience in the past where an implicit contract was not apparent which led to resource leak. Not a blocker.

Comment on lines +54 to +59
emitter.emit(builder.setMetric("mergeBuffer/usedCount", mergeBufferPool.getUsedResourcesCount()));

GroupByStatsProvider.AggregateStats statsContainer = groupByStatsProvider.getStatsSince();

if (statsContainer.getMergeBufferAcquisitionCount() > 0) {
emitter.emit(builder.setMetric("mergeBuffer/acquisitionCount", statsContainer.getMergeBufferAcquisitionCount()));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

usedCount and acquisitionCount are difficult to disambiguate. Can we rename them?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Renamed mergeBuffer/usedCount -> mergeBuffer/used, mergeBuffer/acquistionCount -> mergeBuffer/queries.

LMK if this is easier to understand?

@LakshSingla
Copy link
Copy Markdown
Contributor

LakshSingla commented Nov 11, 2024

Waiting on a minor change. Will merge this post.

@LakshSingla LakshSingla merged commit 74422b5 into apache:master Nov 22, 2024
@LakshSingla
Copy link
Copy Markdown
Contributor

The minor change was to use query id instead of resource reservation id in the logic, but that is not a user facing thing and can be picked up in subsequent patches. I have merged this.


emitter.emit(builder.setMetric("mergeBuffer/pendingRequests", mergeBufferPool.getPendingRequests()));

emitter.emit(builder.setMetric("mergeBuffer/used", mergeBufferPool.getUsedResourcesCount()));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

@LakshSingla @findingrish Is there a reason why this metric isn’t included in the per-query stats? I think it would be helpful to report it for each query individually.

Right now, the emission of this metric depends on the emission interval. For example, if the interval is 1 minute and a query runs in the middle of that cycle—using and then freeing the merge buffers before the minute ends—we won’t see that the buffers were ever used. So, we could miss important usage information simply because the timing doesn’t align with the metric emission.

Please let me know if I’ve misunderstood how this works.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I enabled the metric on a test cluster, and I found that, despite running many GroupBy queries,mergeBuffer/used always equals 0

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

cc @gianm

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This metric is meant to be a point-in-time measurement, designed to help diagnose situations where merge buffer usage is consistently very high, and causes the pool to be continuously exhausted.

There is another metric mergeBuffer/queries that is the number of queries made that used merge buffers. This one might be more what you want, if you're wanting to see per query metrics.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

@gianm mergeBuffer/queries defines how many queries acquired some merge buffers. But we can't get exact numbers, some queries acquire 1 merge buffer, and others acquire 2 merge buffers.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Yeah, that's a good point. IMO it would make sense to add a new metric like mergeBuffer/count which is the total number of merge buffers allocated. Similar to mergeBuffer/queries, but could increment by more than 1 per query.

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.

7 participants