diff --git a/docs/content/operations/metrics.md b/docs/content/operations/metrics.md index 24afd51bced..a3964d4a102 100644 --- a/docs/content/operations/metrics.md +++ b/docs/content/operations/metrics.md @@ -30,6 +30,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/node/time`|Milliseconds taken to query individual historical/realtime nodes.|id, status, server.|< 1s| +|`query/node/bytes`|bytes returned from querying individual historical/realtime nodes.|id, status, server.| | |`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/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| diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index d2417623548..92cbefc5487 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -278,6 +278,7 @@ public class DirectDruidClient implements QueryRunner byteCount.get() / (0.0001 * (stopTime - responseStartTime)) ); emitter.emit(builder.build("query/node/time", stopTime - requestStartTime)); + emitter.emit(builder.build("query/node/bytes", byteCount.get())); synchronized (done) { try { // An empty byte array is put at the end to give the SequenceInputStream.close() as something to close out