-
Notifications
You must be signed in to change notification settings - Fork 3.7k
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
Emit disk spill and merge buffer utilisation metrics for GroupBy queries #17360
Changes from 10 commits
6fcfcf5
f67ca4d
555f487
4397397
9e71dd1
4b0d3e9
4e03093
ad03d0c
ead93e8
9f2d5a9
cc2d93e
680aebe
6757215
2dd8ed2
2a45d30
2795634
b199d57
518c2a7
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 |
---|---|---|
|
@@ -62,7 +62,6 @@ Most metric values reset each emission period, as specified in `druid.monitoring | |
|`query/failed/count`|Number of failed queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.| | | ||
|`query/interrupted/count`|Number of queries interrupted due to cancellation.|This metric is only available if the `QueryCountStatsMonitor` module is included.| | | ||
|`query/timeout/count`|Number of timed out queries.|This metric is only available if the `QueryCountStatsMonitor` 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 `QueryCountStatsMonitor` module is included.| | | ||
|`query/segments/count`|This metric is not enabled by default. See the `QueryMetrics` Interface for reference regarding enabling this metric. Number of segments that will be touched by the query. In the broker, it makes a plan to distribute the query to realtime tasks and historicals based on a snapshot of segment distribution state. If there are some segments moved after this snapshot is created, certain historicals and realtime tasks can report those segments as missing to the broker. The broker will resend the query to the new servers that serve those segments after move. In this case, those segments can be counted more than once in this metric.||Varies| | ||
|`query/priority`|Assigned lane and priority, only if Laning strategy is enabled. Refer to [Laning strategies](../configuration/index.md#laning-strategies)|`lane`, `dataSource`, `type`|0| | ||
|`sqlQuery/time`|Milliseconds taken to complete a SQL query.|`id`, `nativeQueryIds`, `dataSource`, `remoteAddress`, `success`, `engine`|< 1s| | ||
|
@@ -86,6 +85,12 @@ Most metric values reset each emission period, as specified in `druid.monitoring | |
|`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.| | ||
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. 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). |
||
|`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 nanos 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.| | ||
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.
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. 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. It is cumulative, I have renamed it to |
||
|`groupBy/spilledQueries`|Number of groupBy queries that have spilled onto the disk.|This metric is only available if the `GroupByStatsMonitor` module is included.|Depends on the memory configuration for groupBy queries.| | ||
|`groupBy/spilledBytes`|Number of bytes spilled on the disk by the groupBy queries.|This metric is only available if the `GroupByStatsMonitor` module is included.|Depends on the memory configuration for groupBy queries.| | ||
|
||
### Historical | ||
|
||
|
@@ -103,7 +108,12 @@ Most metric values reset each emission period, as specified in `druid.monitoring | |
|`query/failed/count`|Number of failed queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.|| | ||
|`query/interrupted/count`|Number of queries interrupted due to cancellation.|This metric is only available if the `QueryCountStatsMonitor` module is included.|| | ||
|`query/timeout/count`|Number of timed out queries.|This metric is only available if the `QueryCountStatsMonitor` 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 `QueryCountStatsMonitor` 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/acquisitionTimeNs`|Total time in nanos 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.|Depends on the memory configuration for groupBy queries.| | ||
|`groupBy/spilledBytes`|Number of bytes spilled on the disk by the groupBy queries.|This metric is only available if the `GroupByStatsMonitor` module is included.|Depends on the memory configuration for groupBy queries.| | ||
|
||
### Real-time | ||
|
||
|
@@ -119,7 +129,12 @@ Most metric values reset each emission period, as specified in `druid.monitoring | |
|`query/failed/count`|Number of failed queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.|| | ||
|`query/interrupted/count`|Number of queries interrupted due to cancellation.|This metric is only available if the `QueryCountStatsMonitor` module is included.|| | ||
|`query/timeout/count`|Number of timed out queries.|This metric is only available if the `QueryCountStatsMonitor` 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 `QueryCountStatsMonitor` 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/acquisitionTimeNs`|Total time in nanos 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.|Depends on the memory configuration for groupBy queries.| | ||
|`groupBy/spilledBytes`|Number of bytes spilled on the disk by the groupBy queries.|This metric is only available if the `GroupByStatsMonitor` module is included.|Depends on the memory configuration for groupBy queries.| | ||
|
||
### Jetty | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -119,7 +119,7 @@ public List<ReferenceCountingResourceHolder<T>> takeBatch(final int elementNum) | |
throw new RuntimeException(e); | ||
} | ||
finally { | ||
pendingRequests.incrementAndGet(); | ||
pendingRequests.decrementAndGet(); | ||
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. nice catch 💯 |
||
} | ||
} | ||
|
||
|
@@ -129,6 +129,12 @@ public long getPendingRequests() | |
return pendingRequests.get(); | ||
} | ||
|
||
@Override | ||
public long getUsedResourcesCount() | ||
{ | ||
return maxSize - objects.size(); | ||
} | ||
|
||
private List<T> pollObjects(int elementNum) throws InterruptedException | ||
{ | ||
final List<T> list = new ArrayList<>(elementNum); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -61,4 +61,10 @@ public long getPendingRequests() | |
{ | ||
return 0; | ||
} | ||
|
||
@Override | ||
public long getUsedResourcesCount() | ||
{ | ||
return 0; | ||
} | ||
} |
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 : disk spill can be worded better.