This is an automated email from the ASF dual-hosted git repository.

leventov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git


The following commit(s) were added to refs/heads/master by this push:
     new f5f4171  QueryCountStatsMonitor: emit query/count (#6473)
f5f4171 is described below

commit f5f4171a45ad9f79d1b627631643c98a1270ff21
Author: QiuMM <[email protected]>
AuthorDate: Fri Oct 19 21:15:02 2018 +0800

    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.
---
 docs/content/operations/metrics.md                            |  3 +++
 .../apache/druid/server/metrics/QueryCountStatsMonitor.java   | 11 ++++++++---
 2 files changed, 11 insertions(+), 3 deletions(-)

diff --git a/docs/content/operations/metrics.md 
b/docs/content/operations/metrics.md
index e08aecb..5144b0a 100644
--- a/docs/content/operations/metrics.md
+++ b/docs/content/operations/metrics.md
@@ -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.||
diff --git 
a/server/src/main/java/org/apache/druid/server/metrics/QueryCountStatsMonitor.java
 
b/server/src/main/java/org/apache/druid/server/metrics/QueryCountStatsMonitor.java
index 6d88dc6..e217e8d 100644
--- 
a/server/src/main/java/org/apache/druid/server/metrics/QueryCountStatsMonitor.java
+++ 
b/server/src/main/java/org/apache/druid/server/metrics/QueryCountStatsMonitor.java
@@ -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) {


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to