Add metrics for Query Count statistics (#3470)

* Add metrics for Query Count statistics

This PR adds a new metrics monitor “QueryCountStatsMonitor” which emits
three new metrics -
1) query/success/count - number of successful queries
2) query/failed/count - number of failed queries
3) query/interrupted/count - number of interrupted/timedout queries

fix bindings

* make fields final

* fix imports

* AsyncQueryForwardingServlet implement QueryStatsProvider

* remove unused import
This commit is contained in:
Nishant 2016-12-19 23:17:58 +05:30 committed by Fangjin Yang
parent 8eee259629
commit 35160e5595
12 changed files with 180 additions and 47 deletions

View File

@ -169,6 +169,7 @@ The following monitors are available:
|`com.metamx.metrics.JvmMonitor`|Reports JVM-related statistics.|
|`io.druid.segment.realtime.RealtimeMetricsMonitor`|Reports statistics on Realtime nodes.|
|`io.druid.server.metrics.EventReceiverFirehoseMonitor`|Reports how many events have been queued in the EventReceiverFirehose.|
|`io.druid.server.metrics.QueryCountStatsMonitor`|Reports how many queries have been successful/failed/interrupted.|
### Emitting Metrics

View File

@ -34,6 +34,9 @@ Available Metrics
|`query/node/bytes`|number of 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|
|`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.||
### Historical
@ -45,6 +48,9 @@ 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/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.||
### Real-time
@ -53,6 +59,9 @@ 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/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.||
### Jetty

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;
import com.google.inject.Provider;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
@ -34,6 +35,7 @@ import io.druid.query.DruidMetrics;
import io.druid.query.Query;
import io.druid.query.QueryToolChestWarehouse;
import io.druid.server.log.RequestLogger;
import io.druid.server.metrics.QueryCountStatsProvider;
import io.druid.server.router.QueryHostFinder;
import io.druid.server.router.Router;
import org.eclipse.jetty.client.HttpClient;
@ -56,11 +58,12 @@ import java.net.URISyntaxException;
import java.net.URLDecoder;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
/**
* This class does async query processing and should be merged with QueryResource at some point
*/
public class AsyncQueryForwardingServlet extends AsyncProxyServlet
public class AsyncQueryForwardingServlet extends AsyncProxyServlet implements QueryCountStatsProvider
{
private static final EmittingLogger log = new EmittingLogger(AsyncQueryForwardingServlet.class);
@Deprecated // use SmileMediaTypes.APPLICATION_JACKSON_SMILE
@ -72,6 +75,9 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet
private static final int CANCELLATION_TIMEOUT_MILLIS = 500;
private static final int MAX_QUEUED_CANCELLATIONS = 64;
private final AtomicLong successfulQueryCount = new AtomicLong();
private final AtomicLong failedQueryCount = new AtomicLong();
private final AtomicLong interruptedQueryCount = new AtomicLong();
private static void handleException(HttpServletResponse response, ObjectMapper objectMapper, Exception exception)
throws IOException
@ -100,6 +106,7 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet
private HttpClient broadcastClient;
@Inject
public AsyncQueryForwardingServlet(
QueryToolChestWarehouse warehouse,
@Json ObjectMapper jsonMapper,
@ -191,6 +198,7 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet
}
);
}
interruptedQueryCount.incrementAndGet();
}
} else if (isQueryEndpoint && HttpMethod.POST.is(request.getMethod())) {
// query request
@ -317,6 +325,24 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet
return new MetricsEmittingProxyResponseListener(request, response, query, start);
}
@Override
public long getSuccessfulQueryCount()
{
return successfulQueryCount.get();
}
@Override
public long getFailedQueryCount()
{
return failedQueryCount.get();
}
@Override
public long getInterruptedQueryCount()
{
return interruptedQueryCount.get();
}
private class MetricsEmittingProxyResponseListener extends ProxyResponseListener
{
@ -345,11 +371,16 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet
{
final long requestTime = System.currentTimeMillis() - start;
try {
boolean success = result.isSucceeded();
if (success) {
successfulQueryCount.incrementAndGet();
} else {
failedQueryCount.incrementAndGet();
}
emitter.emit(
DruidMetrics.makeQueryTimeMetric(warehouse.getToolChest(query), jsonMapper, query, req.getRemoteAddr())
.build("query/time", requestTime)
);
requestLogger.log(
new RequestLogLine(
new DateTime(),
@ -360,12 +391,14 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet
"query/time",
requestTime,
"success",
result.isSucceeded()
success
&& result.getResponse().getStatus() == javax.ws.rs.core.Response.Status.OK.getStatusCode()
)
)
)
);
}
catch (Exception e) {
log.error(e, "Unable to log query [%s]!", query);
@ -379,6 +412,7 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet
{
try {
final String errorMessage = failure.getMessage();
failedQueryCount.incrementAndGet();
requestLogger.log(
new RequestLogLine(
new DateTime(),

View File

@ -46,6 +46,7 @@ import io.druid.query.QueryToolChest;
import io.druid.query.QueryToolChestWarehouse;
import io.druid.server.initialization.ServerConfig;
import io.druid.server.log.RequestLogger;
import io.druid.server.metrics.QueryCountStatsProvider;
import io.druid.server.security.Access;
import io.druid.server.security.Action;
import io.druid.server.security.AuthConfig;
@ -73,11 +74,12 @@ import java.io.OutputStream;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicLong;
/**
*/
@Path("/druid/v2/")
public class QueryResource
public class QueryResource implements QueryCountStatsProvider
{
protected static final EmittingLogger log = new EmittingLogger(QueryResource.class);
@Deprecated // use SmileMediaTypes.APPLICATION_JACKSON_SMILE
@ -85,6 +87,7 @@ public class QueryResource
protected static final int RESPONSE_CTX_HEADER_LEN_LIMIT = 7 * 1024;
protected final QueryToolChestWarehouse warehouse;
protected final ServerConfig config;
protected final ObjectMapper jsonMapper;
@ -94,6 +97,9 @@ public class QueryResource
protected final RequestLogger requestLogger;
protected final QueryManager queryManager;
protected final AuthConfig authConfig;
private final AtomicLong successfulQueryCount = new AtomicLong();
private final AtomicLong failedQueryCount = new AtomicLong();
private final AtomicLong interruptedQueryCount = new AtomicLong();
@Inject
public QueryResource(
@ -250,7 +256,7 @@ public class QueryResource
os.flush(); // Some types of OutputStream suppress flush errors in the .close() method.
os.close();
successfulQueryCount.incrementAndGet();
final long queryTime = System.currentTimeMillis() - start;
emitter.emit(
DruidMetrics.makeQueryTimeMetric(theToolChest, jsonMapper, theQuery, req.getRemoteAddr())
@ -308,6 +314,7 @@ public class QueryResource
catch (QueryInterruptedException e) {
try {
log.warn(e, "Exception while processing queryId [%s]", queryId);
interruptedQueryCount.incrementAndGet();
final long queryTime = System.currentTimeMillis() - start;
emitter.emit(
DruidMetrics.makeQueryTimeMetric(toolChest, jsonMapper, query, req.getRemoteAddr())
@ -347,6 +354,7 @@ public class QueryResource
: query.toString();
log.warn(e, "Exception occurred on request [%s]", queryString);
failedQueryCount.incrementAndGet();
try {
final long queryTime = System.currentTimeMillis() - start;
@ -437,4 +445,22 @@ public class QueryResource
.build();
}
}
@Override
public long getSuccessfulQueryCount()
{
return successfulQueryCount.get();
}
@Override
public long getFailedQueryCount()
{
return failedQueryCount.get();
}
@Override
public long getInterruptedQueryCount()
{
return interruptedQueryCount.get();
}
}

View File

@ -0,0 +1,62 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.server.metrics;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import com.metamx.metrics.AbstractMonitor;
import com.metamx.metrics.KeyedDiff;
import java.util.Map;
public class QueryCountStatsMonitor extends AbstractMonitor
{
private final KeyedDiff keyedDiff = new KeyedDiff();
private final QueryCountStatsProvider statsProvider;
@Inject
public QueryCountStatsMonitor(
QueryCountStatsProvider statsProvider
)
{
this.statsProvider = statsProvider;
}
@Override
public boolean doMonitor(ServiceEmitter emitter)
{
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
Map<String, Long> diff = keyedDiff.to(
"queryCountStats",
ImmutableMap.of("query/success/count", statsProvider.getSuccessfulQueryCount(),
"query/failed/count", statsProvider.getFailedQueryCount(),
"query/interrupted/count", statsProvider.getInterruptedQueryCount()
)
);
if (diff != null) {
for (Map.Entry<String, Long> diffEntry : diff.entrySet()) {
emitter.emit(builder.build(diffEntry.getKey(), diffEntry.getValue()));
}
}
return true;
}
}

View File

@ -0,0 +1,28 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.server.metrics;
public interface QueryCountStatsProvider
{
public long getSuccessfulQueryCount();
public long getFailedQueryCount();
public long getInterruptedQueryCount();
}

View File

@ -49,6 +49,7 @@ import io.druid.server.coordination.broker.DruidBroker;
import io.druid.server.http.BrokerResource;
import io.druid.server.initialization.jetty.JettyServerInitializer;
import io.druid.server.metrics.MetricsModule;
import io.druid.server.metrics.QueryCountStatsProvider;
import io.druid.server.router.TieredBrokerConfig;
import io.druid.sql.guice.SqlModule;
import org.eclipse.jetty.server.Server;
@ -100,7 +101,9 @@ public class CliBroker extends ServerRunnable
binder.bind(QuerySegmentWalker.class).to(ClientQuerySegmentWalker.class).in(LazySingleton.class);
binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class);
Jerseys.addResource(binder, BrokerQueryResource.class);
binder.bind(QueryCountStatsProvider.class).to(BrokerQueryResource.class).in(LazySingleton.class);
Jerseys.addResource(binder, BrokerResource.class);
Jerseys.addResource(binder, ClientInfoResource.class);
LifecycleModule.register(binder, BrokerQueryResource.class);

View File

@ -37,6 +37,7 @@ import io.druid.guice.NodeTypeConfig;
import io.druid.java.util.common.logger.Logger;
import io.druid.query.QuerySegmentWalker;
import io.druid.query.lookup.LookupModule;
import io.druid.server.metrics.QueryCountStatsProvider;
import io.druid.server.QueryResource;
import io.druid.server.coordination.ServerManager;
import io.druid.server.coordination.ZkCoordinator;
@ -82,6 +83,7 @@ public class CliHistorical extends ServerRunnable
binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig("historical"));
binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class);
binder.bind(QueryCountStatsProvider.class).to(QueryResource.class).in(LazySingleton.class);
Jerseys.addResource(binder, QueryResource.class);
Jerseys.addResource(binder, HistoricalResource.class);
LifecycleModule.register(binder, QueryResource.class);

View File

@ -84,6 +84,7 @@ import io.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider;
import io.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierConfig;
import io.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierFactory;
import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
import io.druid.server.metrics.QueryCountStatsProvider;
import io.druid.server.QueryResource;
import io.druid.server.initialization.jetty.ChatHandlerServerModule;
import io.druid.server.initialization.jetty.JettyServerInitializer;
@ -201,6 +202,7 @@ public class CliPeon extends GuiceRunnable
binder.bind(CoordinatorClient.class).in(LazySingleton.class);
binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class);
binder.bind(QueryCountStatsProvider.class).to(QueryResource.class).in(LazySingleton.class);
Jerseys.addResource(binder, QueryResource.class);
LifecycleModule.register(binder, QueryResource.class);
binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(nodeType));

View File

@ -25,7 +25,6 @@ import com.google.inject.Module;
import com.google.inject.Provides;
import com.google.inject.TypeLiteral;
import com.google.inject.name.Names;
import io.airlift.airline.Command;
import io.druid.curator.discovery.DiscoveryModule;
import io.druid.curator.discovery.ServerDiscoveryFactory;
@ -38,7 +37,9 @@ import io.druid.guice.annotations.Self;
import io.druid.guice.http.JettyHttpClientModule;
import io.druid.java.util.common.logger.Logger;
import io.druid.query.lookup.LookupModule;
import io.druid.server.AsyncQueryForwardingServlet;
import io.druid.server.initialization.jetty.JettyServerInitializer;
import io.druid.server.metrics.QueryCountStatsProvider;
import io.druid.server.router.CoordinatorRuleManager;
import io.druid.server.router.QueryHostFinder;
import io.druid.server.router.Router;
@ -91,6 +92,7 @@ public class CliRouter extends ServerRunnable
.toProvider(TieredBrokerSelectorStrategiesProvider.class)
.in(LazySingleton.class);
binder.bind(QueryCountStatsProvider.class).to(AsyncQueryForwardingServlet.class).in(LazySingleton.class);
binder.bind(JettyServerInitializer.class).to(RouterJettyServerInitializer.class).in(LazySingleton.class);
LifecycleModule.register(binder, Server.class);

View File

@ -19,23 +19,13 @@
package io.druid.cli;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import com.google.inject.Injector;
import com.google.inject.Provider;
import com.google.inject.servlet.GuiceFilter;
import com.metamx.emitter.service.ServiceEmitter;
import io.druid.guice.annotations.Json;
import io.druid.guice.annotations.Smile;
import io.druid.guice.http.DruidHttpClientConfig;
import io.druid.query.QueryToolChestWarehouse;
import io.druid.server.AsyncQueryForwardingServlet;
import io.druid.server.initialization.jetty.JettyServerInitUtils;
import io.druid.server.initialization.jetty.JettyServerInitializer;
import io.druid.server.log.RequestLogger;
import io.druid.server.router.QueryHostFinder;
import io.druid.server.router.Router;
import org.eclipse.jetty.client.HttpClient;
import org.eclipse.jetty.server.Handler;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.server.handler.HandlerList;
@ -47,35 +37,17 @@ import org.eclipse.jetty.servlet.ServletHolder;
*/
public class RouterJettyServerInitializer implements JettyServerInitializer
{
private final QueryToolChestWarehouse warehouse;
private final ObjectMapper jsonMapper;
private final ObjectMapper smileMapper;
private final QueryHostFinder hostFinder;
private final Provider<HttpClient> httpClientProvider;
private final AsyncQueryForwardingServlet asyncQueryForwardingServlet;
private final DruidHttpClientConfig httpClientConfig;
private final ServiceEmitter emitter;
private final RequestLogger requestLogger;
@Inject
public RouterJettyServerInitializer(
QueryToolChestWarehouse warehouse,
@Json ObjectMapper jsonMapper,
@Smile ObjectMapper smileMapper,
QueryHostFinder hostFinder,
@Router Provider<HttpClient> httpClientProvider,
DruidHttpClientConfig httpClientConfig,
ServiceEmitter emitter,
RequestLogger requestLogger
AsyncQueryForwardingServlet asyncQueryForwardingServlet
)
{
this.warehouse = warehouse;
this.jsonMapper = jsonMapper;
this.smileMapper = smileMapper;
this.hostFinder = hostFinder;
this.httpClientProvider = httpClientProvider;
this.httpClientConfig = httpClientConfig;
this.emitter = emitter;
this.requestLogger = requestLogger;
this.asyncQueryForwardingServlet = asyncQueryForwardingServlet;
}
@Override
@ -85,16 +57,6 @@ public class RouterJettyServerInitializer implements JettyServerInitializer
root.addServlet(new ServletHolder(new DefaultServlet()), "/*");
final AsyncQueryForwardingServlet asyncQueryForwardingServlet = new AsyncQueryForwardingServlet(
warehouse,
jsonMapper,
smileMapper,
hostFinder,
httpClientProvider,
httpClientConfig,
emitter,
requestLogger
);
asyncQueryForwardingServlet.setTimeout(httpClientConfig.getReadTimeout().getMillis());
ServletHolder sh = new ServletHolder(asyncQueryForwardingServlet);
//NOTE: explicit maxThreads to workaround https://tickets.puppetlabs.com/browse/TK-152

View File

@ -39,6 +39,7 @@ import io.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider;
import io.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierConfig;
import io.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierFactory;
import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory;
import io.druid.server.metrics.QueryCountStatsProvider;
import io.druid.server.QueryResource;
import io.druid.server.initialization.jetty.JettyServerInitializer;
import org.eclipse.jetty.server.Server;
@ -102,6 +103,7 @@ public class RealtimeModule implements Module
binder.bind(QuerySegmentWalker.class).to(RealtimeManager.class).in(ManageLifecycle.class);
binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig("realtime"));
binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class);
binder.bind(QueryCountStatsProvider.class).to(QueryResource.class).in(LazySingleton.class);
Jerseys.addResource(binder, QueryResource.class);
LifecycleModule.register(binder, QueryResource.class);
LifecycleModule.register(binder, Server.class);