mirror of https://github.com/apache/druid.git
QueryCountStatsMonitor: emit query/count (#6473)
Let `QueryCountStatsMonitor` emit `query/count`, then I can monitor QPS of my services, or I have to count it by myself.
This commit is contained in:
parent
b662fe84c5
commit
f5f4171a45
|
@ -35,6 +35,7 @@ Available Metrics
|
|||
|`query/node/ttfb`|Time to first byte. Milliseconds elapsed until broker starts receiving the response from individual historical/realtime nodes.|id, status, server.|< 1s|
|
||||
|`query/node/backpressure`|Milliseconds that the channel to this node has spent suspended due to backpressure.|id, status, server.| |
|
||||
|`query/intervalChunk/time`|Only emitted if interval chunking is enabled. Milliseconds required to query an interval chunk.|id, status, chunkInterval (if interval chunking is enabled).|< 1s|
|
||||
|`query/count`|number of total queries|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/success/count`|number of queries successfully processed|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`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 or timeout|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|
@ -49,6 +50,7 @@ Available Metrics
|
|||
|`segment/scan/pending`|Number of segments in queue waiting to be scanned.||Close to 0|
|
||||
|`query/segmentAndCache/time`|Milliseconds taken to query individual segment or hit the cache (if it is enabled on the historical node).|id, segment.|several hundred milliseconds|
|
||||
|`query/cpu/time`|Microseconds of CPU time taken to complete a query|Common: dataSource, type, interval, hasFilters, duration, context, remoteAddress, id. Aggregation Queries: numMetrics, numComplexMetrics. GroupBy: numDimensions. TopN: threshold, dimension.|Varies|
|
||||
|`query/count`|number of total queries|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/success/count`|number of queries successfully processed|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`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 or timeout|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|
@ -60,6 +62,7 @@ Available Metrics
|
|||
|`query/time`|Milliseconds taken to complete a query.|Common: dataSource, type, interval, hasFilters, duration, context, remoteAddress, id. Aggregation Queries: numMetrics, numComplexMetrics. GroupBy: numDimensions. TopN: threshold, dimension.|< 1s|
|
||||
|`query/wait/time`|Milliseconds spent waiting for a segment to be scanned.|id, segment.|several hundred milliseconds|
|
||||
|`segment/scan/pending`|Number of segments in queue waiting to be scanned.||Close to 0|
|
||||
|`query/count`|number of total queries|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/success/count`|number of queries successfully processed|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`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 or timeout|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|
|
|
@ -44,11 +44,16 @@ public class QueryCountStatsMonitor extends AbstractMonitor
|
|||
public boolean doMonitor(ServiceEmitter emitter)
|
||||
{
|
||||
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
|
||||
final long successfulQueryCount = statsProvider.getSuccessfulQueryCount();
|
||||
final long failedQueryCount = statsProvider.getFailedQueryCount();
|
||||
final long interruptedQueryCount = statsProvider.getInterruptedQueryCount();
|
||||
Map<String, Long> diff = keyedDiff.to(
|
||||
"queryCountStats",
|
||||
ImmutableMap.of("query/success/count", statsProvider.getSuccessfulQueryCount(),
|
||||
"query/failed/count", statsProvider.getFailedQueryCount(),
|
||||
"query/interrupted/count", statsProvider.getInterruptedQueryCount()
|
||||
ImmutableMap.of(
|
||||
"query/count", successfulQueryCount + failedQueryCount + interruptedQueryCount,
|
||||
"query/success/count", successfulQueryCount,
|
||||
"query/failed/count", failedQueryCount,
|
||||
"query/interrupted/count", interruptedQueryCount
|
||||
)
|
||||
);
|
||||
if (diff != null) {
|
||||
|
|
Loading…
Reference in New Issue