From 2305ed0f1a9f030703335b54e75f1a61b478b3e3 Mon Sep 17 00:00:00 2001 From: leventov Date: Mon, 20 Feb 2017 21:06:40 -0600 Subject: [PATCH 01/17] QueryMetrics: abstraction layer of query metrics emitting --- .../query/scan/ScanQueryQueryToolChest.java | 17 +- .../query/scan/MultiSegmentScanQueryTest.java | 5 +- .../druid/query/scan/ScanQueryRunnerTest.java | 5 +- .../druid/query/CPUTimeMetricQueryRunner.java | 21 +- .../io/druid/query/DefaultQueryMetrics.java | 219 ++++++++++++++++++ .../query/DefaultQueryMetricsFactory.java | 73 ++++++ .../java/io/druid/query/DruidMetrics.java | 58 +---- .../druid/query/FluentQueryRunnerBuilder.java | 13 +- .../query/IntervalChunkingQueryRunner.java | 12 +- .../query/MetricsEmittingQueryRunner.java | 56 +++-- .../main/java/io/druid/query/QueryMetric.java | 52 +++++ .../java/io/druid/query/QueryMetrics.java | 159 +++++++++++++ .../io/druid/query/QueryMetricsFactory.java | 43 ++++ .../java/io/druid/query/QueryToolChest.java | 9 +- .../DataSourceMetadataQueryRunnerFactory.java | 8 +- .../DataSourceQueryQueryToolChest.java | 20 +- .../groupby/DefaultGroupByQueryMetrics.java | 61 +++++ .../query/groupby/GroupByQueryMetrics.java | 45 ++++ .../groupby/GroupByQueryQueryToolChest.java | 37 ++- .../SegmentMetadataQueryQueryToolChest.java | 20 +- .../search/SearchQueryQueryToolChest.java | 23 +- .../select/SelectQueryQueryToolChest.java | 27 ++- .../TimeBoundaryQueryQueryToolChest.java | 26 ++- .../DefaultTimeseriesQueryMetrics.java | 54 +++++ .../timeseries/TimeseriesQueryMetrics.java | 40 ++++ .../TimeseriesQueryQueryToolChest.java | 30 +-- .../query/topn/DefaultTopNQueryMetrics.java | 68 ++++++ .../io/druid/query/topn/TopNQueryMetrics.java | 50 ++++ .../query/topn/TopNQueryQueryToolChest.java | 38 +-- .../DataSourceMetadataQueryTest.java | 7 +- .../io/druid/client/DirectDruidClient.java | 27 +-- .../io/druid/guice/QueryToolChestModule.java | 3 + .../appenderator/SinkQuerySegmentWalker.java | 28 +-- .../server/AsyncQueryForwardingServlet.java | 32 ++- .../io/druid/server/BrokerQueryResource.java | 15 +- .../java/io/druid/server/QueryResource.java | 72 +++--- .../server/coordination/ServerManager.java | 42 +--- .../AsyncQueryForwardingServletTest.java | 7 +- .../io/druid/server/QueryResourceTest.java | 13 +- .../coordination/ServerManagerTest.java | 10 +- 40 files changed, 1227 insertions(+), 318 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/DefaultQueryMetrics.java create mode 100644 processing/src/main/java/io/druid/query/DefaultQueryMetricsFactory.java create mode 100644 processing/src/main/java/io/druid/query/QueryMetric.java create mode 100644 processing/src/main/java/io/druid/query/QueryMetrics.java create mode 100644 processing/src/main/java/io/druid/query/QueryMetricsFactory.java create mode 100644 processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java create mode 100644 processing/src/main/java/io/druid/query/groupby/GroupByQueryMetrics.java create mode 100644 processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java create mode 100644 processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetrics.java create mode 100644 processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNQueryMetrics.java diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java index 222efda0a8e1..0c7e5cb2f730 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java @@ -21,12 +21,13 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.metamx.emitter.service.ServiceMetricEvent; +import com.google.inject.Inject; import io.druid.java.util.common.guava.BaseSequence; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.guava.Sequence; -import io.druid.query.DruidMetrics; import io.druid.query.Query; +import io.druid.query.QueryMetrics; +import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.aggregation.MetricManipulationFn; @@ -39,6 +40,14 @@ public class ScanQueryQueryToolChest extends QueryToolChest mergeResults(final QueryRunner runner) { @@ -74,9 +83,9 @@ public void cleanup(ScanQueryLimitRowIterator iterFromMake) } @Override - public ServiceMetricEvent.Builder makeMetricBuilder(ScanQuery query) + public QueryMetrics> makeMetrics(ScanQuery query) { - return DruidMetrics.makePartialQueryTimeMetric(query); + return queryMetricsFactory.makeMetrics(query); } @Override diff --git a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java index 750a5f428358..d2d021147c55 100644 --- a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java +++ b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java @@ -27,6 +27,7 @@ import io.druid.java.util.common.guava.MergeSequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; @@ -63,7 +64,9 @@ @RunWith(Parameterized.class) public class MultiSegmentScanQueryTest { - private static final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest(); + private static final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest( + DefaultQueryMetricsFactory.instance() + ); private static final QueryRunnerFactory factory = new ScanQueryRunnerFactory( toolChest, diff --git a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java index 94fc1bc5af9f..c18c73e3e50a 100644 --- a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java +++ b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java @@ -27,6 +27,7 @@ import com.google.common.collect.Sets; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequences; +import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.TableDataSource; @@ -95,7 +96,9 @@ public class ScanQueryRunnerTest ); public static final String[] V_0112_0114 = ObjectArrays.concat(V_0112, V_0113, String.class); - private static final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest(); + private static final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest( + DefaultQueryMetricsFactory.instance() + ); @Parameterized.Parameters(name = "{0}") public static Iterable constructorFeeder() throws IOException diff --git a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java index 805c9b2e25ea..9480b4e2aeae 100644 --- a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java +++ b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java @@ -19,11 +19,8 @@ package io.druid.query; -import com.google.common.base.Function; -import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.common.utils.VMUtils; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequence; @@ -36,14 +33,14 @@ public class CPUTimeMetricQueryRunner implements QueryRunner { private final QueryRunner delegate; - private final Function, ServiceMetricEvent.Builder> builderFn; + private final QueryToolChest> queryToolChest; private final ServiceEmitter emitter; private final AtomicLong cpuTimeAccumulator; private final boolean report; private CPUTimeMetricQueryRunner( QueryRunner delegate, - Function, ServiceMetricEvent.Builder> builderFn, + QueryToolChest> queryToolChest, ServiceEmitter emitter, AtomicLong cpuTimeAccumulator, boolean report @@ -53,7 +50,7 @@ private CPUTimeMetricQueryRunner( throw new ISE("Cpu time must enabled"); } this.delegate = delegate; - this.builderFn = builderFn; + this.queryToolChest = queryToolChest; this.emitter = emitter; this.cpuTimeAccumulator = cpuTimeAccumulator == null ? new AtomicLong(0L) : cpuTimeAccumulator; this.report = report; @@ -85,10 +82,10 @@ public RetType wrap(Supplier sequenceProcessing) public void after(boolean isDone, Throwable thrown) throws Exception { if (report) { - final long cpuTime = cpuTimeAccumulator.get(); - if (cpuTime > 0) { - final ServiceMetricEvent.Builder builder = Preconditions.checkNotNull(builderFn.apply(query)); - emitter.emit(builder.build("query/cpu/time", cpuTimeAccumulator.get() / 1000)); + final long cpuTimeNs = cpuTimeAccumulator.get(); + if (cpuTimeNs > 0) { + QueryMetrics> queryMetrics = queryToolChest.makeMetrics(query); + queryMetrics.cpuTime(emitter, cpuTimeNs); } } } @@ -98,7 +95,7 @@ public void after(boolean isDone, Throwable thrown) throws Exception public static QueryRunner safeBuild( QueryRunner delegate, - Function, ServiceMetricEvent.Builder> builderFn, + QueryToolChest> queryToolChest, ServiceEmitter emitter, AtomicLong accumulator, boolean report @@ -107,7 +104,7 @@ public static QueryRunner safeBuild( if (!VMUtils.isThreadCpuTimeEnabled()) { return delegate; } else { - return new CPUTimeMetricQueryRunner<>(delegate, builderFn, emitter, accumulator, report); + return new CPUTimeMetricQueryRunner<>(delegate, queryToolChest, emitter, accumulator, report); } } } diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java new file mode 100644 index 000000000000..46845de19ccb --- /dev/null +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -0,0 +1,219 @@ +/* + * 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.query; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; +import org.joda.time.Interval; + +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public class DefaultQueryMetrics> implements QueryMetrics +{ + protected final ObjectMapper jsonMapper; + protected final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); + + public DefaultQueryMetrics(ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + + @Override + public void query(QueryType query) + { + dataSource(query); + queryType(query); + interval(query); + hasFilters(query); + duration(query); + queryId(query); + } + + @Override + public void dataSource(QueryType query) + { + builder.setDimension(DruidMetrics.DATASOURCE, DataSourceUtil.getMetricName(query.getDataSource())); + } + + @Override + public void queryType(QueryType query) + { + builder.setDimension(DruidMetrics.TYPE, query.getType()); + } + + @Override + public void interval(QueryType query) + { + builder.setDimension( + DruidMetrics.INTERVAL, + Lists.transform( + query.getIntervals(), + new Function() + { + @Override + public String apply(Interval input) + { + return input.toString(); + } + } + ).toArray(new String[query.getIntervals().size()]) + ); + } + + @Override + public void hasFilters(QueryType query) + { + builder.setDimension("hasFilters", String.valueOf(query.hasFilters())); + } + + @Override + public void duration(QueryType query) + { + builder.setDimension("duration", query.getDuration().toString()); + } + + @Override + public void queryId(QueryType query) + { + builder.setDimension(DruidMetrics.ID, Strings.nullToEmpty(query.getId())); + } + + @Override + public void context(QueryType query) + { + try { + builder.setDimension( + "context", + jsonMapper.writeValueAsString( + query.getContext() == null + ? ImmutableMap.of() + : query.getContext() + ) + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + @Override + public void server(String host) + { + builder.setDimension("server", host); + } + + @Override + public void remoteAddress(String removeAddress) + { + builder.setDimension("removeAddress", removeAddress); + } + + @Override + public void userDimensions(Map userDimensions) + { + for (Map.Entry userDimension : userDimensions.entrySet()) { + builder.setDimension(userDimension.getKey(), userDimension.getValue()); + } + } + + @Override + public void status(String status) + { + builder.setDimension(DruidMetrics.STATUS, status); + } + + @Override + public void success(boolean success) + { + builder.setDimension("success", String.valueOf(success)); + } + + @Override + public void queryTime(ServiceEmitter emitter, long timeNs) + { + defaultTimeMetric(emitter, "query/time", timeNs); + } + + @Override + public void queryBytes(ServiceEmitter emitter, long byteCount) + { + emitter.emit(builder.build("query/bytes", byteCount)); + } + + @Override + public void waitTime(ServiceEmitter emitter, long timeNs) + { + defaultTimeMetric(emitter, "query/wait/time", timeNs); + } + + @Override + public void segmentTime(ServiceEmitter emitter, long timeNs) + { + defaultTimeMetric(emitter, "query/segment/time", timeNs); + } + + @Override + public void segmentAndCacheTime(ServiceEmitter emitter, long timeNs) + { + defaultTimeMetric(emitter, "query/segmentAndCache/time", timeNs); + } + + @Override + public void intervalChunkTime(ServiceEmitter emitter, long timeNs) + { + defaultTimeMetric(emitter, "query/intervalChunk/time", timeNs); + } + + @Override + public void cpuTime(ServiceEmitter emitter, long timeNs) + { + emitter.emit(builder.build("query/cpu/time", TimeUnit.NANOSECONDS.toMicros(timeNs))); + } + + @Override + public void nodeTimeToFirstByte(ServiceEmitter emitter, long timeNs) + { + defaultTimeMetric(emitter, "query/node/ttfb", timeNs); + } + + @Override + public void nodeTime(ServiceEmitter emitter, long timeNs) + { + defaultTimeMetric(emitter, "query/node/time", timeNs); + } + + private void defaultTimeMetric(ServiceEmitter emitter, String metricName, long timeNs) + { + emitter.emit(builder.build(metricName, TimeUnit.NANOSECONDS.toMillis(timeNs))); + } + + @Override + public void nodeBytes(ServiceEmitter emitter, long byteCount) + { + emitter.emit(builder.build("query/node/bytes", byteCount)); + } +} diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/DefaultQueryMetricsFactory.java new file mode 100644 index 000000000000..3b787301277e --- /dev/null +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetricsFactory.java @@ -0,0 +1,73 @@ +/* + * 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.query; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import io.druid.query.groupby.DefaultGroupByQueryMetrics; +import io.druid.query.groupby.GroupByQueryMetrics; +import io.druid.query.timeseries.DefaultTimeseriesQueryMetrics; +import io.druid.query.timeseries.TimeseriesQueryMetrics; +import io.druid.query.topn.DefaultTopNQueryMetrics; +import io.druid.query.topn.TopNQueryMetrics; + +public class DefaultQueryMetricsFactory implements QueryMetricsFactory +{ + private static final DefaultQueryMetricsFactory INSTANCE = new DefaultQueryMetricsFactory(new ObjectMapper()); + + public static DefaultQueryMetricsFactory instance() + { + return INSTANCE; + } + + private final ObjectMapper jsonMapper; + + @Inject + public DefaultQueryMetricsFactory(ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + + @Override + public QueryMetrics> makeMetrics(Query query) + { + DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(jsonMapper); + queryMetrics.query(query); + return queryMetrics; + } + + @Override + public TopNQueryMetrics makeTopNQueryMetrics() + { + return new DefaultTopNQueryMetrics(jsonMapper); + } + + @Override + public GroupByQueryMetrics makeGroupByQueryMetrics() + { + return new DefaultGroupByQueryMetrics(jsonMapper); + } + + @Override + public TimeseriesQueryMetrics makeTimeseriesQueryMetrics() + { + return new DefaultTimeseriesQueryMetrics(jsonMapper); + } +} diff --git a/processing/src/main/java/io/druid/query/DruidMetrics.java b/processing/src/main/java/io/druid/query/DruidMetrics.java index 4801f0227e5c..fb1188d2ae6f 100644 --- a/processing/src/main/java/io/druid/query/DruidMetrics.java +++ b/processing/src/main/java/io/druid/query/DruidMetrics.java @@ -20,14 +20,7 @@ package io.druid.query; import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; -import com.google.common.base.Strings; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.aggregation.AggregatorFactory; -import org.joda.time.Interval; import java.util.List; @@ -61,50 +54,21 @@ public static int findNumComplexAggs(List aggs) return retVal; } - public static ServiceMetricEvent.Builder makePartialQueryTimeMetric(Query query) - { - return new ServiceMetricEvent.Builder() - .setDimension(DATASOURCE, DataSourceUtil.getMetricName(query.getDataSource())) - .setDimension(TYPE, query.getType()) - .setDimension( - INTERVAL, - Lists.transform( - query.getIntervals(), - new Function() - { - @Override - public String apply(Interval input) - { - return input.toString(); - } - } - ).toArray(new String[query.getIntervals().size()]) - ) - .setDimension("hasFilters", String.valueOf(query.hasFilters())) - .setDimension("duration", query.getDuration().toString()) - .setDimension(ID, Strings.nullToEmpty(query.getId())); - } - - public static ServiceMetricEvent.Builder makeQueryTimeMetric( + public static QueryMetrics makeRequestMetrics( + final QueryMetricsFactory queryMetricsFactory, final QueryToolChest> toolChest, - final ObjectMapper jsonMapper, final Query query, final String remoteAddr ) throws JsonProcessingException { - final ServiceMetricEvent.Builder baseMetric = toolChest == null - ? makePartialQueryTimeMetric(query) - : toolChest.makeMetricBuilder(query); - - return baseMetric - .setDimension( - "context", - jsonMapper.writeValueAsString( - query.getContext() == null - ? ImmutableMap.of() - : query.getContext() - ) - ) - .setDimension("remoteAddress", remoteAddr); + QueryMetrics> queryMetrics; + if (toolChest != null) { + queryMetrics = toolChest.makeMetrics(query); + } else { + queryMetrics = queryMetricsFactory.makeMetrics(query); + } + queryMetrics.context(query); + queryMetrics.remoteAddress(remoteAddr); + return queryMetrics; } } diff --git a/processing/src/main/java/io/druid/query/FluentQueryRunnerBuilder.java b/processing/src/main/java/io/druid/query/FluentQueryRunnerBuilder.java index fbc1150f3c4a..71bf44451cda 100644 --- a/processing/src/main/java/io/druid/query/FluentQueryRunnerBuilder.java +++ b/processing/src/main/java/io/druid/query/FluentQueryRunnerBuilder.java @@ -19,12 +19,9 @@ package io.druid.query; -import com.google.common.base.Function; import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.java.util.common.guava.Sequence; -import javax.annotation.Nullable; import java.util.Map; import java.util.concurrent.atomic.AtomicLong; @@ -90,15 +87,7 @@ public FluentQueryRunner emitCPUTimeMetric(ServiceEmitter emitter) return from( CPUTimeMetricQueryRunner.safeBuild( baseRunner, - new Function, ServiceMetricEvent.Builder>() - { - @Nullable - @Override - public ServiceMetricEvent.Builder apply(Query tQuery) - { - return toolChest.makeMetricBuilder(tQuery); - } - }, + toolChest, emitter, new AtomicLong(0L), true diff --git a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java index 2e718ec23168..8a6d2cd5f838 100644 --- a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java @@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.granularity.PeriodGranularity; import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.guava.Sequence; @@ -102,16 +101,9 @@ public Sequence apply(Interval singleInterval) toolChest.mergeResults( new MetricsEmittingQueryRunner( emitter, - new Function, ServiceMetricEvent.Builder>() - { - @Override - public ServiceMetricEvent.Builder apply(Query input) - { - return toolChest.makeMetricBuilder(input); - } - }, + toolChest, baseRunner, - "query/intervalChunk/time", + QueryMetric.INTERVAL_CHUNK_TIME, ImmutableMap.of("chunkInterval", singleInterval.toString()) ).withWaitMeasuredFromNow() ), diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index c35547c051df..0015a488ffe1 100644 --- a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java @@ -19,10 +19,8 @@ package io.druid.query; -import com.google.common.base.Function; import com.google.common.base.Supplier; import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.java.util.common.guava.LazySequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.SequenceWrapper; @@ -35,48 +33,48 @@ public class MetricsEmittingQueryRunner implements QueryRunner { private final ServiceEmitter emitter; - private final Function, ServiceMetricEvent.Builder> builderFn; + private final QueryToolChest> queryToolChest; private final QueryRunner queryRunner; - private final long creationTime; - private final String metricName; + private final long creationTimeNs; + private final QueryMetric metric; private final Map userDimensions; private MetricsEmittingQueryRunner( ServiceEmitter emitter, - Function, ServiceMetricEvent.Builder> builderFn, + QueryToolChest> queryToolChest, QueryRunner queryRunner, - long creationTime, - String metricName, + long creationTimeNs, + QueryMetric metric, Map userDimensions ) { this.emitter = emitter; - this.builderFn = builderFn; + this.queryToolChest = queryToolChest; this.queryRunner = queryRunner; - this.creationTime = creationTime; - this.metricName = metricName; + this.creationTimeNs = creationTimeNs; + this.metric = metric; this.userDimensions = userDimensions; } public MetricsEmittingQueryRunner( ServiceEmitter emitter, - Function, ServiceMetricEvent.Builder> builderFn, + QueryToolChest> queryToolChest, QueryRunner queryRunner, - String metricName, + QueryMetric metric, Map userDimensions ) { - this(emitter, builderFn, queryRunner, -1, metricName, userDimensions); + this(emitter, queryToolChest, queryRunner, -1, metric, userDimensions); } public MetricsEmittingQueryRunner withWaitMeasuredFromNow() { - return new MetricsEmittingQueryRunner( + return new MetricsEmittingQueryRunner<>( emitter, - builderFn, + queryToolChest, queryRunner, - System.currentTimeMillis(), - metricName, + System.nanoTime(), + metric, userDimensions ); } @@ -84,11 +82,9 @@ public MetricsEmittingQueryRunner withWaitMeasuredFromNow() @Override public Sequence run(final Query query, final Map responseContext) { - final ServiceMetricEvent.Builder builder = builderFn.apply(query); + final QueryMetrics> queryMetrics = queryToolChest.makeMetrics(query); - for (Map.Entry userDimension : userDimensions.entrySet()) { - builder.setDimension(userDimension.getKey(), userDimension.getValue()); - } + queryMetrics.userDimensions(userDimensions); return Sequences.wrap( // Use LazySequence because want to account execution time of queryRunner.run() (it prepares the underlying @@ -104,27 +100,27 @@ public Sequence get() }), new SequenceWrapper() { - private long startTime; + private long startTimeNs; @Override public void before() { - startTime = System.currentTimeMillis(); + startTimeNs = System.nanoTime(); } @Override public void after(boolean isDone, Throwable thrown) { if (thrown != null) { - builder.setDimension(DruidMetrics.STATUS, "failed"); + queryMetrics.status("failed"); } else if (!isDone) { - builder.setDimension(DruidMetrics.STATUS, "short"); + queryMetrics.status("short"); } - long timeTaken = System.currentTimeMillis() - startTime; - emitter.emit(builder.build(metricName, timeTaken)); + long timeTakenNs = System.nanoTime() - startTimeNs; + metric.emit(queryMetrics, emitter, timeTakenNs); - if (creationTime > 0) { - emitter.emit(builder.build("query/wait/time", startTime - creationTime)); + if (creationTimeNs > 0) { + queryMetrics.waitTime(emitter, startTimeNs - creationTimeNs); } } } diff --git a/processing/src/main/java/io/druid/query/QueryMetric.java b/processing/src/main/java/io/druid/query/QueryMetric.java new file mode 100644 index 000000000000..4ff98c67f72f --- /dev/null +++ b/processing/src/main/java/io/druid/query/QueryMetric.java @@ -0,0 +1,52 @@ +/* + * 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.query; + +import com.metamx.emitter.service.ServiceEmitter; + +/** + * Used in {@link MetricsEmittingQueryRunner} as a binder to a particular metric emitting method of {@link QueryMetrics} + */ +public enum QueryMetric +{ + SEGMENT_TIME { + @Override + public void emit(QueryMetrics metrics, ServiceEmitter emitter, long timeNs) + { + metrics.segmentTime(emitter, timeNs); + } + }, + SEGMENT_AND_CACHE_TIME { + @Override + public void emit(QueryMetrics metrics, ServiceEmitter emitter, long timeNs) + { + metrics.segmentAndCacheTime(emitter, timeNs); + } + }, + INTERVAL_CHUNK_TIME { + @Override + public void emit(QueryMetrics metrics, ServiceEmitter emitter, long timeNs) + { + metrics.intervalChunkTime(emitter, timeNs); + } + }; + + public abstract void emit(QueryMetrics metrics, ServiceEmitter emitter, long value); +} diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java new file mode 100644 index 000000000000..c109159d4c4d --- /dev/null +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -0,0 +1,159 @@ +/* + * 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.query; + +import com.metamx.emitter.service.ServiceEmitter; + +import java.util.Map; + +/** + * Abstraction wrapping {@link com.metamx.emitter.service.ServiceMetricEvent.Builder} and allowing to control what + * metrics are actually emitted, what dimensions do they have, etc. + * + * + * Goals of QueryMetrics + * --------------------- + * 1. Skipping or partial filtering of particular dimensions or metrics entirely. Implementation could leave the body + * of the corresponding empty, or implement random filtering like: + * public void cpuTime(ServiceEmitter emitter, long timeNs) + * { + * if (ThreadLocalRandom.current().nextDouble() < 0.1) { + * super.cpuTime(emitter, timeNs); + * } + * } + * + * 2. Ability to add new dimensions and metrics, possibly expensive to compute, or expensive to process (long string + * values, high cardinality, etc.) and not to affect existing Druid installations, by skipping (see 1.) those + * dimensions and metrics entirely in the default QueryMetrics implementations. Users who need those expensive + * dimensions and metrics, could explicitly emit them in their own QueryMetrics. + * + * 3. Control over the time unit, in which time metrics are emitted. By default (see {@link DefaultQueryMetrics} and + * it's subclasses) it's milliseconds, but if queries are fast, it could be not precise enough. + * + * 4. Control over the dimension and metric names. + * + * + * Types of methods in this interface + * ---------------------------------- + * 1. Methods, pulling some dimensions from the query object. These methods are used to populate the metric before the + * query is run. These methods accept a single `QueryType query` parameter. + * 2. Methods for general dimensions, which become known in the process of the query execution or after the query is + * completed. + * 3. Methods for emitting metrics, they accept {@link ServiceEmitter} as the first parameter. + * + * + * Implementors expectations + * ------------------------- + * QueryMetrics is expected to be changed often, in every Druid release. Users who create their custom implementations + * of QueryMetrics should be ready to fix the code of their QueryMetrics (implement new methods) when they update Druid. + * Broken builds of custom extensions, containing custom QueryMetrics is the way to notify users that Druid core "wants" + * to emit new dimension or metric, and the user handles them manually: if the new dimension or metric is useful and not + * very expensive to process and store then emit, skip (see above Goals, 1.) otherwise. + * + * + * Adding new methods to QueryMetrics + * ---------------------------------- + * 1. When adding a new method for emitting a dimension, which could be pulled from the query object, always make them + * accept a single `QueryType query` parameter, letting the implementations to do all the work of carving the dimension + * value out of the query object. + * + * 2. When adding a new method for emitting a dimension, which becomes known in the process of the query execution or + * after the query is completed, design them so that as little work as possible is done for preparing arguments for this + * method, and as much work as possible is done in the implementations of this method, if they decide to emit this + * dimension. + * + * 3. When adding a new method for emitting metrics, make it to accept the metric value in the smallest reasonable + * unit (i. e. nanoseconds for time metrics, bytes for metrics of data size, etc.), allowing the implementations of + * this method to round the value up to more coarse-grained units, if they don't need the maximum precision. + * + * @param + */ +public interface QueryMetrics> +{ + /** + * Pulls all information for the query object into dimensions of future metrics events. + */ + void query(QueryType query); + + /** + * Sets {@link Query#getDataSource()} of the given query as dimension. + */ + void dataSource(QueryType query); + + /** + * Sets {@link Query#getType()} of the given query as dimension. + */ + void queryType(QueryType query); + + /** + * Sets {@link Query#getIntervals()} of the given query as dimension. + */ + void interval(QueryType query); + + /** + * Sets {@link Query#hasFilters()} of the given query as dimension. + */ + void hasFilters(QueryType query); + + /** + * Sets {@link Query#getDuration()} of the given query as dimension. + */ + void duration(QueryType query); + + /** + * Sets {@link Query#getId()} of the given query as dimension. + */ + void queryId(QueryType query); + + /** + * Sets {@link Query#getContext()} of the given query as dimension. + */ + void context(QueryType query); + + void server(String host); + + void remoteAddress(String removeAddress); + + void userDimensions(Map userDimensions); + + void status(String status); + + void success(boolean success); + + void queryTime(ServiceEmitter emitter, long timeNs); + + void queryBytes(ServiceEmitter emitter, long byteCount); + + void waitTime(ServiceEmitter emitter, long timeNs); + + void segmentTime(ServiceEmitter emitter, long timeNs); + + void segmentAndCacheTime(ServiceEmitter emitter, long timeNs); + + void intervalChunkTime(ServiceEmitter emitter, long timeNs); + + void cpuTime(ServiceEmitter emitter, long timeNs); + + void nodeTimeToFirstByte(ServiceEmitter emitter, long timeNs); + + void nodeTime(ServiceEmitter emitter, long timeNs); + + void nodeBytes(ServiceEmitter emitter, long byteCount); +} diff --git a/processing/src/main/java/io/druid/query/QueryMetricsFactory.java b/processing/src/main/java/io/druid/query/QueryMetricsFactory.java new file mode 100644 index 000000000000..5af813b0c70b --- /dev/null +++ b/processing/src/main/java/io/druid/query/QueryMetricsFactory.java @@ -0,0 +1,43 @@ +/* + * 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.query; + +import io.druid.query.groupby.GroupByQueryMetrics; +import io.druid.query.timeseries.TimeseriesQueryMetrics; +import io.druid.query.topn.TopNQueryMetrics; + +/** + * This factory is used for DI of custom {@link QueryMetrics} implementations. + */ +public interface QueryMetricsFactory +{ + /** + * Creates a generic {@link QueryMetrics} for query, which doesn't require a custom QueryMetrics subclass (i. e. not + * a topN, groupBy or timeseries query). This method must call {@link QueryMetrics#query(Query)} with the given query + * on the created QueryMetrics object before returning. + */ + QueryMetrics> makeMetrics(Query query); + + TopNQueryMetrics makeTopNQueryMetrics(); + + GroupByQueryMetrics makeGroupByQueryMetrics(); + + TimeseriesQueryMetrics makeTimeseriesQueryMetrics(); +} diff --git a/processing/src/main/java/io/druid/query/QueryToolChest.java b/processing/src/main/java/io/druid/query/QueryToolChest.java index e62348228fd5..d57a79a3f416 100644 --- a/processing/src/main/java/io/druid/query/QueryToolChest.java +++ b/processing/src/main/java/io/druid/query/QueryToolChest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.timeline.LogicalSegment; @@ -46,16 +45,16 @@ public abstract class QueryToolChest mergeResults(QueryRunner runner); /** - * Creates a builder that is used to generate a metric for this specific query type. This exists - * to allow for query-specific dimensions on metrics. That is, the ToolChest is expected to set some + * Creates a {@link QueryMetrics} object that is used to generate metrics for this specific query type. This exists + * to allow for query-specific dimensions and metrics. That is, the ToolChest is expected to set some * meaningful dimensions for metrics given this query type. Examples might be the topN threshold for * a TopN query or the number of dimensions included for a groupBy query. * * @param query The query that is being processed * - * @return A MetricEvent.Builder that can be used to make metrics for the provided query + * @return A QueryMetrics that can be used to make metrics for the provided query */ - public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); + public abstract QueryMetrics makeMetrics(QueryType query); /** * Creates a Function that can take in a ResultType and return a new ResultType having applied diff --git a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java index 1e3b5c81bcbb..c137f75c9b1c 100644 --- a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java @@ -42,12 +42,16 @@ public class DataSourceMetadataQueryRunnerFactory implements QueryRunnerFactory, DataSourceMetadataQuery> { - private static final DataSourceQueryQueryToolChest toolChest = new DataSourceQueryQueryToolChest(); + private final DataSourceQueryQueryToolChest toolChest; private final QueryWatcher queryWatcher; @Inject - public DataSourceMetadataQueryRunnerFactory(QueryWatcher queryWatcher) + public DataSourceMetadataQueryRunnerFactory( + DataSourceQueryQueryToolChest toolChest, + QueryWatcher queryWatcher + ) { + this.toolChest = toolChest; this.queryWatcher = queryWatcher; } diff --git a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java index 0ec071756c75..3f87050066d2 100644 --- a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java @@ -25,14 +25,14 @@ import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.metamx.emitter.service.ServiceMetricEvent; +import com.google.inject.Inject; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.BySegmentSkippingQueryRunner; import io.druid.query.CacheStrategy; -import io.druid.query.DataSourceUtil; -import io.druid.query.DruidMetrics; import io.druid.query.Query; +import io.druid.query.QueryMetrics; +import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -51,6 +51,14 @@ public class DataSourceQueryQueryToolChest { }; + private final QueryMetricsFactory queryMetricsFactory; + + @Inject + public DataSourceQueryQueryToolChest(QueryMetricsFactory queryMetricsFactory) + { + this.queryMetricsFactory = queryMetricsFactory; + } + @Override public List filterSegments(DataSourceMetadataQuery query, List segments) { @@ -103,11 +111,9 @@ protected Sequence> doRun( } @Override - public ServiceMetricEvent.Builder makeMetricBuilder(DataSourceMetadataQuery query) + public QueryMetrics> makeMetrics(DataSourceMetadataQuery query) { - return DruidMetrics.makePartialQueryTimeMetric(query) - .setDimension("dataSource", DataSourceUtil.getMetricName(query.getDataSource())) - .setDimension("type", query.getType()); + return queryMetricsFactory.makeMetrics(query); } @Override diff --git a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java new file mode 100644 index 000000000000..5d8ee7321fc3 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java @@ -0,0 +1,61 @@ +/* + * 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.query.groupby; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.query.DefaultQueryMetrics; +import io.druid.query.DruidMetrics; + +public class DefaultGroupByQueryMetrics extends DefaultQueryMetrics implements GroupByQueryMetrics +{ + + public DefaultGroupByQueryMetrics(ObjectMapper jsonMapper) + { + super(jsonMapper); + } + + @Override + public void query(GroupByQuery query) + { + super.query(query); + numDimensions(query); + numMetrics(query); + numComplexMetrics(query); + } + + @Override + public void numDimensions(GroupByQuery query) + { + builder.setDimension("numDimensions", String.valueOf(query.getDimensions().size())); + } + + @Override + public void numMetrics(GroupByQuery query) + { + builder.setDimension("numMetrics", String.valueOf(query.getAggregatorSpecs().size())); + } + + @Override + public void numComplexMetrics(GroupByQuery query) + { + int numComplexAggs = DruidMetrics.findNumComplexAggs(query.getAggregatorSpecs()); + builder.setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetrics.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetrics.java new file mode 100644 index 000000000000..2d9ef261d193 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetrics.java @@ -0,0 +1,45 @@ +/* + * 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.query.groupby; + +import io.druid.query.QueryMetrics; + +/** + * Specialization of {@link QueryMetrics} for {@link GroupByQuery}. + */ +public interface GroupByQueryMetrics extends QueryMetrics +{ + /** + * Sets the size of {@link GroupByQuery#getDimensions()} of the given query as dimension. + */ + void numDimensions(GroupByQuery query); + + /** + * Sets the number of metrics of the given groupBy query as dimension. + */ + void numMetrics(GroupByQuery query); + + /** + * Sets the number of "complex" metrics of the given groupBy query as dimension. By default it is assumed that + * "complex" metric is a metric of not long or double type, but it could be redefined in the implementation of this + * method. + */ + void numComplexMetrics(GroupByQuery query); +} diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 57979634becb..1129e0d5ead3 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -31,7 +31,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.inject.Inject; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; @@ -43,10 +42,11 @@ import io.druid.query.BaseQuery; import io.druid.query.CacheStrategy; import io.druid.query.DataSource; -import io.druid.query.DruidMetrics; +import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; import io.druid.query.QueryDataSource; +import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.SubqueryQueryRunner; @@ -88,19 +88,38 @@ public class GroupByQueryQueryToolChest extends QueryToolChest bufferPool; private final IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator; + private final QueryMetricsFactory queryMetricsFactory; - @Inject public GroupByQueryQueryToolChest( Supplier configSupplier, GroupByStrategySelector strategySelector, @Global StupidPool bufferPool, IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator ) + { + this( + configSupplier, + strategySelector, + bufferPool, + intervalChunkingQueryRunnerDecorator, + DefaultQueryMetricsFactory.instance() + ); + } + + @Inject + public GroupByQueryQueryToolChest( + Supplier configSupplier, + GroupByStrategySelector strategySelector, + @Global StupidPool bufferPool, + IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator, + QueryMetricsFactory queryMetricsFactory + ) { this.configSupplier = configSupplier; this.strategySelector = strategySelector; this.bufferPool = bufferPool; this.intervalChunkingQueryRunnerDecorator = intervalChunkingQueryRunnerDecorator; + this.queryMetricsFactory = queryMetricsFactory; } @Override @@ -193,15 +212,11 @@ private Sequence mergeGroupByResults( } @Override - public ServiceMetricEvent.Builder makeMetricBuilder(GroupByQuery query) + public GroupByQueryMetrics makeMetrics(GroupByQuery query) { - return DruidMetrics.makePartialQueryTimeMetric(query) - .setDimension("numDimensions", String.valueOf(query.getDimensions().size())) - .setDimension("numMetrics", String.valueOf(query.getAggregatorSpecs().size())) - .setDimension( - "numComplexMetrics", - String.valueOf(DruidMetrics.findNumComplexAggs(query.getAggregatorSpecs())) - ); + GroupByQueryMetrics queryMetrics = queryMetricsFactory.makeGroupByQueryMetrics(); + queryMetrics.query(query); + return queryMetrics; } @Override diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 0cb17be84436..36f9b22bc3a0 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -31,7 +31,6 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.common.guava.CombiningSequence; import io.druid.common.utils.JodaUtils; import io.druid.data.input.impl.TimestampSpec; @@ -40,8 +39,10 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.nary.BinaryFn; import io.druid.query.CacheStrategy; -import io.druid.query.DruidMetrics; +import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.Query; +import io.druid.query.QueryMetrics; +import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.ResultMergeQueryRunner; @@ -78,13 +79,18 @@ public SegmentAnalysis apply(SegmentAnalysis analysis) }; private final SegmentMetadataQueryConfig config; + private final QueryMetricsFactory queryMetricsFactory; + + public SegmentMetadataQueryQueryToolChest(SegmentMetadataQueryConfig config) + { + this(config, DefaultQueryMetricsFactory.instance()); + } @Inject - public SegmentMetadataQueryQueryToolChest( - SegmentMetadataQueryConfig config - ) + public SegmentMetadataQueryQueryToolChest(SegmentMetadataQueryConfig config, QueryMetricsFactory queryMetricsFactory) { this.config = config; + this.queryMetricsFactory = queryMetricsFactory; } @Override @@ -147,9 +153,9 @@ public SegmentAnalysis apply(SegmentAnalysis arg1, SegmentAnalysis arg2) } @Override - public ServiceMetricEvent.Builder makeMetricBuilder(SegmentMetadataQuery query) + public QueryMetrics> makeMetrics(SegmentMetadataQuery query) { - return DruidMetrics.makePartialQueryTimeMetric(query); + return queryMetricsFactory.makeMetrics(query); } @Override diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index 4a0103226301..489976cd803b 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -29,7 +29,6 @@ import com.google.common.collect.Ordering; import com.google.common.primitives.Ints; import com.google.inject.Inject; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequence; @@ -37,9 +36,11 @@ import io.druid.java.util.common.guava.nary.BinaryFn; import io.druid.query.BaseQuery; import io.druid.query.CacheStrategy; -import io.druid.query.DruidMetrics; +import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; +import io.druid.query.QueryMetrics; +import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -72,17 +73,27 @@ public class SearchQueryQueryToolChest extends QueryToolChest, Result, Result< } @Override - public ServiceMetricEvent.Builder makeMetricBuilder(SearchQuery query) + public QueryMetrics> makeMetrics(SearchQuery query) { - return DruidMetrics.makePartialQueryTimeMetric(query); + return queryMetricsFactory.makeMetrics(query); } @Override diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java index f65918846966..4a35cfdeb21e 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -30,16 +30,17 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.granularity.QueryGranularity; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.nary.BinaryFn; import io.druid.query.CacheStrategy; -import io.druid.query.DruidMetrics; +import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; +import io.druid.query.QueryMetrics; +import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -77,15 +78,27 @@ public class SelectQueryQueryToolChest extends QueryToolChest, Result, Result< } @Override - public ServiceMetricEvent.Builder makeMetricBuilder(SelectQuery query) + public QueryMetrics> makeMetrics(SelectQuery query) { - return DruidMetrics.makePartialQueryTimeMetric(query); + return queryMetricsFactory.makeMetrics(query); } @Override diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 03da955bb2d0..0e1b315daa77 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -25,14 +25,15 @@ import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.metamx.emitter.service.ServiceMetricEvent; +import com.google.inject.Inject; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.BySegmentSkippingQueryRunner; import io.druid.query.CacheStrategy; -import io.druid.query.DataSourceUtil; -import io.druid.query.DruidMetrics; +import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.Query; +import io.druid.query.QueryMetrics; +import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -58,6 +59,19 @@ public class TimeBoundaryQueryQueryToolChest { }; + private final QueryMetricsFactory queryMetricsFactory; + + public TimeBoundaryQueryQueryToolChest() + { + this(DefaultQueryMetricsFactory.instance()); + } + + @Inject + public TimeBoundaryQueryQueryToolChest(QueryMetricsFactory queryMetricsFactory) + { + this.queryMetricsFactory = queryMetricsFactory; + } + @Override public List filterSegments(TimeBoundaryQuery query, List segments) { @@ -107,11 +121,9 @@ protected Sequence> doRun( } @Override - public ServiceMetricEvent.Builder makeMetricBuilder(TimeBoundaryQuery query) + public QueryMetrics> makeMetrics(TimeBoundaryQuery query) { - return DruidMetrics.makePartialQueryTimeMetric(query) - .setDimension(DruidMetrics.DATASOURCE, DataSourceUtil.getMetricName(query.getDataSource())) - .setDimension(DruidMetrics.TYPE, query.getType()); + return queryMetricsFactory.makeMetrics(query); } @Override diff --git a/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java new file mode 100644 index 000000000000..addaac5a76f5 --- /dev/null +++ b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java @@ -0,0 +1,54 @@ +/* + * 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.query.timeseries; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.query.DefaultQueryMetrics; +import io.druid.query.DruidMetrics; + +public class DefaultTimeseriesQueryMetrics extends DefaultQueryMetrics + implements TimeseriesQueryMetrics +{ + public DefaultTimeseriesQueryMetrics(ObjectMapper jsonMapper) + { + super(jsonMapper); + } + + @Override + public void query(TimeseriesQuery query) + { + super.query(query); + numMetrics(query); + numComplexMetrics(query); + } + + @Override + public void numMetrics(TimeseriesQuery query) + { + builder.setDimension("numMetrics", String.valueOf(query.getAggregatorSpecs().size())); + } + + @Override + public void numComplexMetrics(TimeseriesQuery query) + { + int numComplexAggs = DruidMetrics.findNumComplexAggs(query.getAggregatorSpecs()); + builder.setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); + } +} diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetrics.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetrics.java new file mode 100644 index 000000000000..db7553525dc6 --- /dev/null +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetrics.java @@ -0,0 +1,40 @@ +/* + * 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.query.timeseries; + +import io.druid.query.QueryMetrics; + +/** + * Specialization of {@link QueryMetrics} for {@link TimeseriesQuery}. + */ +public interface TimeseriesQueryMetrics extends QueryMetrics +{ + /** + * Sets the number of metrics of the given timeseries query as dimension. + */ + void numMetrics(TimeseriesQuery query); + + /** + * Sets the number of "complex" metrics of the given timeseries query as dimension. By default it is assumed that + * "complex" metric is a metric of not long or double type, but it could be redefined in the implementation of this + * method. + */ + void numComplexMetrics(TimeseriesQuery query); +} diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index a399af557e76..307bf6ccef9b 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -25,14 +25,14 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.inject.Inject; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.granularity.QueryGranularity; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.nary.BinaryFn; import io.druid.query.CacheStrategy; -import io.druid.query.DruidMetrics; +import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; +import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -64,11 +64,21 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest, Result, } @Override - public ServiceMetricEvent.Builder makeMetricBuilder(TimeseriesQuery query) + public TimeseriesQueryMetrics makeMetrics(TimeseriesQuery query) { - return DruidMetrics.makePartialQueryTimeMetric(query) - .setDimension( - "numMetrics", - String.valueOf(query.getAggregatorSpecs().size()) - ) - .setDimension( - "numComplexMetrics", - String.valueOf(DruidMetrics.findNumComplexAggs(query.getAggregatorSpecs())) - ); + TimeseriesQueryMetrics queryMetrics = queryMetricsFactory.makeTimeseriesQueryMetrics(); + queryMetrics.query(query); + return queryMetrics; } @Override diff --git a/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java new file mode 100644 index 000000000000..2be769b6c1fd --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java @@ -0,0 +1,68 @@ +/* + * 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.query.topn; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.query.DefaultQueryMetrics; +import io.druid.query.DruidMetrics; + +public class DefaultTopNQueryMetrics extends DefaultQueryMetrics implements TopNQueryMetrics +{ + + public DefaultTopNQueryMetrics(ObjectMapper jsonMapper) + { + super(jsonMapper); + } + + @Override + public void query(TopNQuery query) + { + super.query(query); + threshold(query); + dimension(query); + numMetrics(query); + numComplexMetrics(query); + } + + @Override + public void threshold(TopNQuery query) + { + builder.setDimension("threshold", String.valueOf(query.getThreshold())); + } + + @Override + public void dimension(TopNQuery query) + { + builder.setDimension("dimension", query.getDimensionSpec().getDimension()); + } + + @Override + public void numMetrics(TopNQuery query) + { + builder.setDimension("numMetrics", String.valueOf(query.getAggregatorSpecs().size())); + } + + @Override + public void numComplexMetrics(TopNQuery query) + { + int numComplexAggs = DruidMetrics.findNumComplexAggs(query.getAggregatorSpecs()); + builder.setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryMetrics.java b/processing/src/main/java/io/druid/query/topn/TopNQueryMetrics.java new file mode 100644 index 000000000000..a3492f717429 --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryMetrics.java @@ -0,0 +1,50 @@ +/* + * 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.query.topn; + +import io.druid.query.QueryMetrics; + +/** + * Specialization of {@link QueryMetrics} for {@link TopNQuery}. + */ +public interface TopNQueryMetrics extends QueryMetrics +{ + /** + * Sets {@link TopNQuery#getThreshold()} of the given query as dimension. + */ + void threshold(TopNQuery query); + + /** + * Sets {@link TopNQuery#getDimensionSpec()}.{@link io.druid.query.dimension.DimensionSpec#getDimension() + * getDimension()} of the given query as dimension. + */ + void dimension(TopNQuery query); + + /** + * Sets the number of metrics of the given topN query as dimension. + */ + void numMetrics(TopNQuery query); + + /** + * Sets the number of "complex" metrics of the given topN query as dimension. By default it is assumed that "complex" + * metric is a metric of not long or double type, but it could be redefined in the implementation of this method. + */ + void numComplexMetrics(TopNQuery query); +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index 6f876de37e25..443f9f684396 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -26,7 +26,6 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.inject.Inject; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.granularity.QueryGranularity; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequence; @@ -35,9 +34,10 @@ import io.druid.query.BaseQuery; import io.druid.query.BySegmentResultValue; import io.druid.query.CacheStrategy; -import io.druid.query.DruidMetrics; +import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; +import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -68,18 +68,29 @@ public class TopNQueryQueryToolChest extends QueryToolChest OBJECT_TYPE_REFERENCE = new TypeReference() { }; - private final TopNQueryConfig config; + private final TopNQueryConfig config; private final IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator; + private final QueryMetricsFactory queryMetricsFactory; - @Inject public TopNQueryQueryToolChest( TopNQueryConfig config, IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator ) + { + this(config, intervalChunkingQueryRunnerDecorator, DefaultQueryMetricsFactory.instance()); + } + + @Inject + public TopNQueryQueryToolChest( + TopNQueryConfig config, + IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator, + QueryMetricsFactory queryMetricsFactory + ) { this.config = config; this.intervalChunkingQueryRunnerDecorator = intervalChunkingQueryRunnerDecorator; + this.queryMetricsFactory = queryMetricsFactory; } protected static String[] extractFactoryName(final List aggregatorFactories) @@ -140,22 +151,11 @@ protected BinaryFn, Result, Result segments = new DataSourceQueryQueryToolChest().filterSegments( + DefaultQueryMetricsFactory queryMetricsFactory = DefaultQueryMetricsFactory.instance(); + DataSourceQueryQueryToolChest toolChest = new DataSourceQueryQueryToolChest(queryMetricsFactory); + List segments = toolChest + .filterSegments( null, Arrays.asList( new LogicalSegment() diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index 358f2685f819..7eb3f5f02acd 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -36,14 +36,12 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.http.client.HttpClient; import com.metamx.http.client.Request; import com.metamx.http.client.response.ClientResponse; import com.metamx.http.client.response.HttpResponseHandler; import com.metamx.http.client.response.StatusResponseHandler; import com.metamx.http.client.response.StatusResponseHolder; - import io.druid.java.util.common.IAE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.RE; @@ -56,6 +54,7 @@ import io.druid.query.BySegmentResultValueClass; import io.druid.query.Query; import io.druid.query.QueryInterruptedException; +import io.druid.query.QueryMetrics; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; @@ -83,6 +82,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -161,14 +161,14 @@ public Sequence run(final Query query, final Map context) try { log.debug("Querying queryId[%s] url[%s]", query.getId(), url); - final long requestStartTime = System.currentTimeMillis(); + final long requestStartTimeNs = System.nanoTime(); - final ServiceMetricEvent.Builder builder = toolChest.makeMetricBuilder(query); - builder.setDimension("server", host); + final QueryMetrics> queryMetrics = toolChest.makeMetrics(query); + queryMetrics.server(host); final HttpResponseHandler responseHandler = new HttpResponseHandler() { - private long responseStartTime; + private long responseStartTimeNs; private final AtomicLong byteCount = new AtomicLong(0); private final BlockingQueue queue = new LinkedBlockingQueue<>(); private final AtomicBoolean done = new AtomicBoolean(false); @@ -177,8 +177,8 @@ public Sequence run(final Query query, final Map context) public ClientResponse handleResponse(HttpResponse response) { log.debug("Initial response from url[%s] for queryId[%s]", url, query.getId()); - responseStartTime = System.currentTimeMillis(); - emitter.emit(builder.build("query/node/ttfb", responseStartTime - requestStartTime)); + responseStartTimeNs = System.nanoTime(); + queryMetrics.nodeTimeToFirstByte(emitter, responseStartTimeNs - requestStartTimeNs); try { final String responseContext = response.headers().get("X-Druid-Response-Context"); @@ -267,17 +267,18 @@ public ClientResponse handleChunk( @Override public ClientResponse done(ClientResponse clientResponse) { - long stopTime = System.currentTimeMillis(); + long stopTimeNs = System.nanoTime(); + long nodeTimeNs = stopTimeNs - responseStartTimeNs; log.debug( "Completed queryId[%s] request to url[%s] with %,d bytes returned in %,d millis [%,f b/s].", query.getId(), url, byteCount.get(), - stopTime - responseStartTime, - byteCount.get() / (0.0001 * (stopTime - responseStartTime)) + TimeUnit.NANOSECONDS.toMillis(nodeTimeNs), + byteCount.get() / TimeUnit.NANOSECONDS.toSeconds(nodeTimeNs) ); - emitter.emit(builder.build("query/node/time", stopTime - requestStartTime)); - emitter.emit(builder.build("query/node/bytes", byteCount.get())); + queryMetrics.nodeTime(emitter, nodeTimeNs); + queryMetrics.nodeBytes(emitter, byteCount.get()); synchronized (done) { try { // An empty byte array is put at the end to give the SequenceInputStream.close() as something to close out diff --git a/server/src/main/java/io/druid/guice/QueryToolChestModule.java b/server/src/main/java/io/druid/guice/QueryToolChestModule.java index 4c07cdbf0f8d..a20969e035d9 100644 --- a/server/src/main/java/io/druid/guice/QueryToolChestModule.java +++ b/server/src/main/java/io/druid/guice/QueryToolChestModule.java @@ -23,8 +23,10 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.multibindings.MapBinder; +import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.MapQueryToolChestWarehouse; import io.druid.query.Query; +import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; import io.druid.query.datasourcemetadata.DataSourceMetadataQuery; @@ -77,6 +79,7 @@ public void configure(Binder binder) } binder.bind(QueryToolChestWarehouse.class).to(MapQueryToolChestWarehouse.class); + binder.bind(QueryMetricsFactory.class).to(DefaultQueryMetricsFactory.class); JsonConfigProvider.bind(binder, "druid.query.groupBy", GroupByQueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.search", SearchQueryConfig.class); diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 4b1ccef95532..6cd266b129a2 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -27,7 +27,6 @@ import com.google.common.util.concurrent.MoreExecutors; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.client.CachingQueryRunner; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; @@ -48,6 +47,7 @@ import io.druid.query.QueryToolChest; import io.druid.query.ReportTimelineMissingSegmentQueryRunner; import io.druid.query.SegmentDescriptor; +import io.druid.query.QueryMetric; import io.druid.query.TableDataSource; import io.druid.query.spec.SpecificSegmentQueryRunner; import io.druid.query.spec.SpecificSegmentSpec; @@ -60,7 +60,6 @@ import io.druid.timeline.partition.PartitionHolder; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.Closeable; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicLong; @@ -166,15 +165,6 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final } final QueryToolChest> toolChest = factory.getToolchest(); - final Function, ServiceMetricEvent.Builder> builderFn = - new Function, ServiceMetricEvent.Builder>() - { - @Override - public ServiceMetricEvent.Builder apply(@Nullable Query input) - { - return toolChest.makeMetricBuilder(query); - } - }; final boolean skipIncrementalSegment = query.getContextValue(CONTEXT_SKIP_INCREMENTAL_SEGMENT, false); final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); @@ -262,7 +252,7 @@ public QueryRunner apply(final FireHydrant hydrant) ) ) ), - builderFn, + toolChest, sinkSegmentIdentifier, cpuTimeAccumulator ), @@ -273,7 +263,7 @@ public QueryRunner apply(final FireHydrant hydrant) ) ) ), - builderFn, + toolChest, emitter, cpuTimeAccumulator, true @@ -286,7 +276,7 @@ public QueryRunner apply(final FireHydrant hydrant) */ private QueryRunner withPerSinkMetrics( final QueryRunner sinkRunner, - final Function, ServiceMetricEvent.Builder> builderFn, + final QueryToolChest> queryToolChest, final String sinkSegmentIdentifier, final AtomicLong cpuTimeAccumulator ) @@ -301,18 +291,18 @@ private QueryRunner withPerSinkMetrics( return CPUTimeMetricQueryRunner.safeBuild( new MetricsEmittingQueryRunner<>( emitter, - builderFn, + queryToolChest, new MetricsEmittingQueryRunner<>( emitter, - builderFn, + queryToolChest, sinkRunner, - "query/segment/time", + QueryMetric.SEGMENT_TIME, dims ), - "query/segmentAndCache/time", + QueryMetric.SEGMENT_AND_CACHE_TIME, dims ).withWaitMeasuredFromNow(), - builderFn, + queryToolChest, emitter, cpuTimeAccumulator, false diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index be1a2cdad60e..7f8f885315b4 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -33,6 +33,8 @@ import io.druid.guice.http.DruidHttpClientConfig; import io.druid.query.DruidMetrics; import io.druid.query.Query; +import io.druid.query.QueryMetrics; +import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryToolChestWarehouse; import io.druid.server.log.RequestLogger; import io.druid.server.metrics.QueryCountStatsProvider; @@ -103,6 +105,7 @@ private static void handleException(HttpServletResponse response, ObjectMapper o private final DruidHttpClientConfig httpClientConfig; private final ServiceEmitter emitter; private final RequestLogger requestLogger; + private final QueryMetricsFactory queryMetricsFactory; private HttpClient broadcastClient; @@ -115,7 +118,8 @@ public AsyncQueryForwardingServlet( @Router Provider httpClientProvider, DruidHttpClientConfig httpClientConfig, ServiceEmitter emitter, - RequestLogger requestLogger + RequestLogger requestLogger, + QueryMetricsFactory queryMetricsFactory ) { this.warehouse = warehouse; @@ -126,6 +130,7 @@ public AsyncQueryForwardingServlet( this.httpClientConfig = httpClientConfig; this.emitter = emitter; this.requestLogger = requestLogger; + this.queryMetricsFactory = queryMetricsFactory; } @Override @@ -278,7 +283,7 @@ protected Response.Listener newProxyResponseListener( { final Query query = (Query) request.getAttribute(QUERY_ATTRIBUTE); if (query != null) { - return newMetricsEmittingProxyResponseListener(request, response, query, System.currentTimeMillis()); + return newMetricsEmittingProxyResponseListener(request, response, query, System.nanoTime()); } else { return super.newProxyResponseListener(request, response); } @@ -331,10 +336,10 @@ private Response.Listener newMetricsEmittingProxyResponseListener( HttpServletRequest request, HttpServletResponse response, Query query, - long start + long startNs ) { - return new MetricsEmittingProxyResponseListener(request, response, query, start); + return new MetricsEmittingProxyResponseListener(request, response, query, startNs); } @Override @@ -361,13 +366,13 @@ private class MetricsEmittingProxyResponseListener extends ProxyResponseListener private final HttpServletRequest req; private final HttpServletResponse res; private final Query query; - private final long start; + private final long startNs; public MetricsEmittingProxyResponseListener( HttpServletRequest request, HttpServletResponse response, Query query, - long start + long startNs ) { super(request, response); @@ -375,13 +380,13 @@ public MetricsEmittingProxyResponseListener( this.req = request; this.res = response; this.query = query; - this.start = start; + this.startNs = startNs; } @Override public void onComplete(Result result) { - final long requestTime = System.currentTimeMillis() - start; + final long requestTimeNs = System.nanoTime() - startNs; try { boolean success = result.isSucceeded(); if (success) { @@ -389,10 +394,13 @@ public void onComplete(Result result) } else { failedQueryCount.incrementAndGet(); } - emitter.emit( - DruidMetrics.makeQueryTimeMetric(warehouse.getToolChest(query), jsonMapper, query, req.getRemoteAddr()) - .build("query/time", requestTime) + QueryMetrics queryMetrics = DruidMetrics.makeRequestMetrics( + queryMetricsFactory, + warehouse.getToolChest(query), + query, + req.getRemoteAddr() ); + queryMetrics.queryTime(emitter, requestTimeNs); requestLogger.log( new RequestLogLine( new DateTime(), @@ -401,7 +409,7 @@ public void onComplete(Result result) new QueryStats( ImmutableMap.of( "query/time", - requestTime, + TimeUnit.NANOSECONDS.toMillis(requestTimeNs), "success", success && result.getResponse().getStatus() == javax.ws.rs.core.Response.Status.OK.getStatusCode() diff --git a/server/src/main/java/io/druid/server/BrokerQueryResource.java b/server/src/main/java/io/druid/server/BrokerQueryResource.java index 033b372ec625..2a38f17e8684 100644 --- a/server/src/main/java/io/druid/server/BrokerQueryResource.java +++ b/server/src/main/java/io/druid/server/BrokerQueryResource.java @@ -29,6 +29,7 @@ import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.query.Query; +import io.druid.query.QueryMetricsFactory; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChestWarehouse; import io.druid.server.http.security.StateResourceFilter; @@ -67,10 +68,22 @@ public BrokerQueryResource( RequestLogger requestLogger, QueryManager queryManager, AuthConfig authConfig, + QueryMetricsFactory queryMetricsFactory, TimelineServerView brokerServerView ) { - super(warehouse, config, jsonMapper, smileMapper, texasRanger, emitter, requestLogger, queryManager, authConfig); + super( + warehouse, + config, + jsonMapper, + smileMapper, + texasRanger, + emitter, + requestLogger, + queryManager, + authConfig, + queryMetricsFactory + ); this.brokerServerView = brokerServerView; } diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 2cbd1b361a4c..fbdb7e6765ce 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -41,6 +41,8 @@ import io.druid.query.Query; import io.druid.query.QueryContextKeys; import io.druid.query.QueryInterruptedException; +import io.druid.query.QueryMetrics; +import io.druid.query.QueryMetricsFactory; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; @@ -74,6 +76,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; /** @@ -97,6 +100,7 @@ public class QueryResource implements QueryCountStatsProvider protected final RequestLogger requestLogger; protected final QueryManager queryManager; protected final AuthConfig authConfig; + private final QueryMetricsFactory queryMetricsFactory; private final AtomicLong successfulQueryCount = new AtomicLong(); private final AtomicLong failedQueryCount = new AtomicLong(); private final AtomicLong interruptedQueryCount = new AtomicLong(); @@ -111,7 +115,8 @@ public QueryResource( ServiceEmitter emitter, RequestLogger requestLogger, QueryManager queryManager, - AuthConfig authConfig + AuthConfig authConfig, + QueryMetricsFactory queryMetricsFactory ) { this.warehouse = warehouse; @@ -123,6 +128,7 @@ public QueryResource( this.requestLogger = requestLogger; this.queryManager = queryManager; this.authConfig = authConfig; + this.queryMetricsFactory = queryMetricsFactory; } @DELETE @@ -168,7 +174,7 @@ public Response doPost( @Context final HttpServletRequest req // used to get request content-type, remote address and AuthorizationInfo ) throws IOException { - final long start = System.currentTimeMillis(); + final long startNs = System.nanoTime(); Query query = null; QueryToolChest toolChest = null; String queryId = null; @@ -247,25 +253,31 @@ public void write(OutputStream outputStream) throws IOException, WebApplicationE 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()) - .setDimension("success", "true") - .build("query/time", queryTime) - ); - emitter.emit( - DruidMetrics.makeQueryTimeMetric(theToolChest, jsonMapper, theQuery, req.getRemoteAddr()) - .build("query/bytes", os.getCount()) + final long queryTimeNs = System.nanoTime() - startNs; + QueryMetrics queryMetrics = DruidMetrics.makeRequestMetrics( + queryMetricsFactory, + theToolChest, + theQuery, + req.getRemoteAddr() ); + queryMetrics.success(true); + queryMetrics.queryTime(emitter, queryTimeNs); + + DruidMetrics.makeRequestMetrics( + queryMetricsFactory, + theToolChest, + theQuery, + req.getRemoteAddr() + ).queryBytes(emitter, os.getCount()); requestLogger.log( new RequestLogLine( - new DateTime(start), + new DateTime(TimeUnit.NANOSECONDS.toMillis(startNs)), req.getRemoteAddr(), theQuery, new QueryStats( ImmutableMap.of( - "query/time", queryTime, + "query/time", TimeUnit.NANOSECONDS.toMillis(queryTimeNs), "query/bytes", os.getCount(), "success", true ) @@ -309,21 +321,24 @@ public void write(OutputStream outputStream) throws IOException, WebApplicationE 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()) - .setDimension("success", "false") - .build("query/time", queryTime) + final long queryTimeNs = System.nanoTime() - startNs; + QueryMetrics queryMetrics = DruidMetrics.makeRequestMetrics( + queryMetricsFactory, + toolChest, + query, + req.getRemoteAddr() ); + queryMetrics.success(false); + queryMetrics.queryTime(emitter, queryTimeNs); requestLogger.log( new RequestLogLine( - new DateTime(start), + new DateTime(TimeUnit.NANOSECONDS.toMillis(startNs)), req.getRemoteAddr(), query, new QueryStats( ImmutableMap.of( "query/time", - queryTime, + TimeUnit.NANOSECONDS.toMillis(queryTimeNs), "success", false, "interrupted", @@ -351,20 +366,23 @@ public void write(OutputStream outputStream) throws IOException, WebApplicationE failedQueryCount.incrementAndGet(); try { - final long queryTime = System.currentTimeMillis() - start; - emitter.emit( - DruidMetrics.makeQueryTimeMetric(toolChest, jsonMapper, query, req.getRemoteAddr()) - .setDimension("success", "false") - .build("query/time", queryTime) + final long queryTimeNs = System.nanoTime() - startNs; + QueryMetrics queryMetrics = DruidMetrics.makeRequestMetrics( + queryMetricsFactory, + toolChest, + query, + req.getRemoteAddr() ); + queryMetrics.success(false); + queryMetrics.queryTime(emitter, queryTimeNs); requestLogger.log( new RequestLogLine( - new DateTime(start), + new DateTime(TimeUnit.NANOSECONDS.toMillis(startNs)), req.getRemoteAddr(), query, new QueryStats(ImmutableMap.of( "query/time", - queryTime, + TimeUnit.NANOSECONDS.toMillis(queryTimeNs), "success", false, "exception", diff --git a/server/src/main/java/io/druid/server/coordination/ServerManager.java b/server/src/main/java/io/druid/server/coordination/ServerManager.java index 9360c4e2171d..7fde82b2c54f 100644 --- a/server/src/main/java/io/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/io/druid/server/coordination/ServerManager.java @@ -27,7 +27,6 @@ import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.client.CachingQueryRunner; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; @@ -52,6 +51,7 @@ import io.druid.query.ReferenceCountingSegmentQueryRunner; import io.druid.query.ReportTimelineMissingSegmentQueryRunner; import io.druid.query.SegmentDescriptor; +import io.druid.query.QueryMetric; import io.druid.query.TableDataSource; import io.druid.query.spec.SpecificSegmentQueryRunner; import io.druid.query.spec.SpecificSegmentSpec; @@ -255,7 +255,6 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable> toolChest = factory.getToolchest(); - final Function, ServiceMetricEvent.Builder> builderFn = getBuilderFn(toolChest); final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); DataSource dataSource = query.getDataSource(); @@ -312,7 +311,6 @@ public QueryRunner apply(PartitionChunk input) holder.getVersion(), input.getChunkNumber() ), - builderFn, cpuTimeAccumulator ); } @@ -327,7 +325,7 @@ public QueryRunner apply(PartitionChunk input) toolChest.mergeResults(factory.mergeRunners(exec, queryRunners)), toolChest ), - builderFn, + toolChest, emitter, cpuTimeAccumulator, true @@ -362,7 +360,6 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable(); } - final Function, ServiceMetricEvent.Builder> builderFn = getBuilderFn(toolChest); final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); FunctionalIterable> queryRunners = FunctionalIterable @@ -390,7 +387,7 @@ public Iterable> apply(SegmentDescriptor input) final ReferenceCountingSegment adapter = chunk.getObject(); return Arrays.asList( - buildAndDecorateQueryRunner(factory, toolChest, adapter, input, builderFn, cpuTimeAccumulator) + buildAndDecorateQueryRunner(factory, toolChest, adapter, input, cpuTimeAccumulator) ); } } @@ -401,7 +398,7 @@ public Iterable> apply(SegmentDescriptor input) toolChest.mergeResults(factory.mergeRunners(exec, queryRunners)), toolChest ), - builderFn, + toolChest, emitter, cpuTimeAccumulator, true @@ -413,7 +410,6 @@ private QueryRunner buildAndDecorateQueryRunner( final QueryToolChest> toolChest, final ReferenceCountingSegment adapter, final SegmentDescriptor segmentDescriptor, - final Function, ServiceMetricEvent.Builder> builderFn, final AtomicLong cpuTimeAccumulator ) { @@ -422,7 +418,7 @@ private QueryRunner buildAndDecorateQueryRunner( new SpecificSegmentQueryRunner( new MetricsEmittingQueryRunner( emitter, - builderFn, + toolChest, new BySegmentQueryRunner( adapter.getIdentifier(), adapter.getDataInterval().getStart(), @@ -434,44 +430,24 @@ private QueryRunner buildAndDecorateQueryRunner( toolChest, new MetricsEmittingQueryRunner( emitter, - new Function, ServiceMetricEvent.Builder>() - { - @Override - public ServiceMetricEvent.Builder apply(@Nullable final Query input) - { - return toolChest.makeMetricBuilder(input); - } - }, + toolChest, new ReferenceCountingSegmentQueryRunner(factory, adapter, segmentDescriptor), - "query/segment/time", + QueryMetric.SEGMENT_TIME, ImmutableMap.of("segment", adapter.getIdentifier()) ), cachingExec, cacheConfig ) ), - "query/segmentAndCache/time", + QueryMetric.SEGMENT_AND_CACHE_TIME, ImmutableMap.of("segment", adapter.getIdentifier()) ).withWaitMeasuredFromNow(), segmentSpec ), - builderFn, + toolChest, emitter, cpuTimeAccumulator, false ); } - - private static Function, ServiceMetricEvent.Builder> getBuilderFn(final QueryToolChest> toolChest) - { - return new Function, ServiceMetricEvent.Builder>() - { - @Nullable - @Override - public ServiceMetricEvent.Builder apply(@Nullable Query input) - { - return toolChest.makeMetricBuilder(input); - } - }; - } } diff --git a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java index b2d19a28baed..bee48a1c8271 100644 --- a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java +++ b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java @@ -41,6 +41,7 @@ import io.druid.guice.http.DruidHttpClientConfig; import io.druid.initialization.Initialization; import io.druid.java.util.common.lifecycle.Lifecycle; +import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.MapQueryToolChestWarehouse; import io.druid.query.Query; import io.druid.query.QueryToolChest; @@ -215,10 +216,11 @@ public Collection getAllHosts() } }; + ObjectMapper jsonMapper = injector.getInstance(ObjectMapper.class); ServletHolder holder = new ServletHolder( new AsyncQueryForwardingServlet( new MapQueryToolChestWarehouse(ImmutableMap., QueryToolChest>of()), - injector.getInstance(ObjectMapper.class), + jsonMapper, injector.getInstance(Key.get(ObjectMapper.class, Smile.class)), hostFinder, injector.getProvider(org.eclipse.jetty.client.HttpClient.class), @@ -231,7 +233,8 @@ public void log(RequestLogLine requestLogLine) throws IOException { // noop } - } + }, + new DefaultQueryMetricsFactory(jsonMapper) ) { @Override diff --git a/server/src/test/java/io/druid/server/QueryResourceTest.java b/server/src/test/java/io/druid/server/QueryResourceTest.java index 0f023b1bc2cf..b127cb458171 100644 --- a/server/src/test/java/io/druid/server/QueryResourceTest.java +++ b/server/src/test/java/io/druid/server/QueryResourceTest.java @@ -29,6 +29,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.MapQueryToolChestWarehouse; import io.druid.query.Query; import io.druid.query.QueryRunner; @@ -138,7 +139,8 @@ public void setup() new NoopServiceEmitter(), new NoopRequestLogger(), queryManager, - new AuthConfig() + new AuthConfig(), + new DefaultQueryMetricsFactory(jsonMapper) ); } @@ -212,7 +214,8 @@ public Access isAuthorized( new NoopServiceEmitter(), new NoopRequestLogger(), queryManager, - new AuthConfig(true) + new AuthConfig(true), + new DefaultQueryMetricsFactory(jsonMapper) ); Response response = queryResource.doPost( @@ -282,7 +285,8 @@ public Access isAuthorized( new NoopServiceEmitter(), new NoopRequestLogger(), queryManager, - new AuthConfig(true) + new AuthConfig(true), + new DefaultQueryMetricsFactory(jsonMapper) ); final String queryString = "{\"queryType\":\"timeBoundary\", \"dataSource\":\"allow\"," @@ -378,7 +382,8 @@ public Access isAuthorized( new NoopServiceEmitter(), new NoopRequestLogger(), queryManager, - new AuthConfig(true) + new AuthConfig(true), + new DefaultQueryMetricsFactory(jsonMapper) ); final String queryString = "{\"queryType\":\"timeBoundary\", \"dataSource\":\"allow\"," diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index b24ed2ba4287..5e85ce579feb 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -20,6 +20,7 @@ package io.druid.server.coordination; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Throwables; @@ -28,8 +29,6 @@ import com.google.common.collect.Lists; import com.google.common.util.concurrent.MoreExecutors; import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.service.ServiceMetricEvent; - import io.druid.client.cache.CacheConfig; import io.druid.client.cache.LocalCacheProvider; import io.druid.granularity.QueryGranularities; @@ -44,9 +43,11 @@ import io.druid.java.util.common.guava.YieldingAccumulator; import io.druid.java.util.common.guava.YieldingSequenceBase; import io.druid.query.ConcatQueryRunner; +import io.druid.query.DefaultQueryMetrics; import io.druid.query.Druids; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryMetrics; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -66,7 +67,6 @@ import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; - import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -575,9 +575,9 @@ public QueryRunner mergeResults(QueryRunner runner) } @Override - public ServiceMetricEvent.Builder makeMetricBuilder(QueryType query) + public QueryMetrics> makeMetrics(QueryType query) { - return new ServiceMetricEvent.Builder(); + return new DefaultQueryMetrics<>(new ObjectMapper()); } @Override From 3ce08ce289b7d300f6d356b9ff0fc8e2cb36e842 Mon Sep 17 00:00:00 2001 From: leventov Date: Tue, 21 Feb 2017 00:34:38 -0600 Subject: [PATCH 02/17] Minor fixes --- .../main/java/io/druid/query/CPUTimeMetricQueryRunner.java | 3 +-- processing/src/main/java/io/druid/query/QueryMetrics.java | 4 ++-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java index 9480b4e2aeae..fe410c7581f2 100644 --- a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java +++ b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java @@ -84,8 +84,7 @@ public void after(boolean isDone, Throwable thrown) throws Exception if (report) { final long cpuTimeNs = cpuTimeAccumulator.get(); if (cpuTimeNs > 0) { - QueryMetrics> queryMetrics = queryToolChest.makeMetrics(query); - queryMetrics.cpuTime(emitter, cpuTimeNs); + queryToolChest.makeMetrics(query).cpuTime(emitter, cpuTimeNs); } } } diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index c109159d4c4d..393b0d6f2f18 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -31,7 +31,7 @@ * Goals of QueryMetrics * --------------------- * 1. Skipping or partial filtering of particular dimensions or metrics entirely. Implementation could leave the body - * of the corresponding empty, or implement random filtering like: + * of the corresponding method empty, or implement random filtering like: * public void cpuTime(ServiceEmitter emitter, long timeNs) * { * if (ThreadLocalRandom.current().nextDouble() < 0.1) { @@ -88,7 +88,7 @@ public interface QueryMetrics> { /** - * Pulls all information for the query object into dimensions of future metrics events. + * Pulls all information from the query object into dimensions of future metrics. */ void query(QueryType query); From c1967b370549f027012bf62af3d81cd15f44ce0b Mon Sep 17 00:00:00 2001 From: leventov Date: Tue, 21 Feb 2017 15:29:36 -0600 Subject: [PATCH 03/17] QueryMetrics.emit() for bulk emit and improve Javadoc --- .../druid/query/CPUTimeMetricQueryRunner.java | 2 +- .../io/druid/query/DefaultQueryMetrics.java | 59 ++++++++++------- .../query/MetricsEmittingQueryRunner.java | 5 +- .../main/java/io/druid/query/QueryMetric.java | 16 +++-- .../java/io/druid/query/QueryMetrics.java | 63 +++++++++++++++---- .../io/druid/client/DirectDruidClient.java | 7 ++- .../server/AsyncQueryForwardingServlet.java | 2 +- .../java/io/druid/server/QueryResource.java | 8 +-- 8 files changed, 108 insertions(+), 54 deletions(-) diff --git a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java index fe410c7581f2..a50f407c5ab8 100644 --- a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java +++ b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java @@ -84,7 +84,7 @@ public void after(boolean isDone, Throwable thrown) throws Exception if (report) { final long cpuTimeNs = cpuTimeAccumulator.get(); if (cpuTimeNs > 0) { - queryToolChest.makeMetrics(query).cpuTime(emitter, cpuTimeNs); + queryToolChest.makeMetrics(query).cpuTime(cpuTimeNs).emit(emitter); } } } diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index 46845de19ccb..5c84a28f6045 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -29,6 +29,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import org.joda.time.Interval; +import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -36,6 +37,7 @@ public class DefaultQueryMetrics> implements QueryMet { protected final ObjectMapper jsonMapper; protected final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); + protected final Map metrics = new HashMap<>(); public DefaultQueryMetrics(ObjectMapper jsonMapper) { @@ -153,67 +155,80 @@ public void success(boolean success) } @Override - public void queryTime(ServiceEmitter emitter, long timeNs) + public QueryMetrics queryTime(long timeNs) { - defaultTimeMetric(emitter, "query/time", timeNs); + return defaultTimeMetric("query/time", timeNs); } @Override - public void queryBytes(ServiceEmitter emitter, long byteCount) + public QueryMetrics queryBytes(long byteCount) { - emitter.emit(builder.build("query/bytes", byteCount)); + metrics.put("query/bytes", byteCount); + return this; } @Override - public void waitTime(ServiceEmitter emitter, long timeNs) + public QueryMetrics waitTime(long timeNs) { - defaultTimeMetric(emitter, "query/wait/time", timeNs); + return defaultTimeMetric("query/wait/time", timeNs); } @Override - public void segmentTime(ServiceEmitter emitter, long timeNs) + public QueryMetrics segmentTime(long timeNs) { - defaultTimeMetric(emitter, "query/segment/time", timeNs); + return defaultTimeMetric("query/segment/time", timeNs); } @Override - public void segmentAndCacheTime(ServiceEmitter emitter, long timeNs) + public QueryMetrics segmentAndCacheTime(long timeNs) { - defaultTimeMetric(emitter, "query/segmentAndCache/time", timeNs); + return defaultTimeMetric("query/segmentAndCache/time", timeNs); } @Override - public void intervalChunkTime(ServiceEmitter emitter, long timeNs) + public QueryMetrics intervalChunkTime(long timeNs) { - defaultTimeMetric(emitter, "query/intervalChunk/time", timeNs); + return defaultTimeMetric("query/intervalChunk/time", timeNs); } @Override - public void cpuTime(ServiceEmitter emitter, long timeNs) + public QueryMetrics cpuTime(long timeNs) { - emitter.emit(builder.build("query/cpu/time", TimeUnit.NANOSECONDS.toMicros(timeNs))); + metrics.put("query/cpu/time", TimeUnit.NANOSECONDS.toMicros(timeNs)); + return this; } @Override - public void nodeTimeToFirstByte(ServiceEmitter emitter, long timeNs) + public QueryMetrics nodeTimeToFirstByte(long timeNs) { - defaultTimeMetric(emitter, "query/node/ttfb", timeNs); + return defaultTimeMetric("query/node/ttfb", timeNs); } @Override - public void nodeTime(ServiceEmitter emitter, long timeNs) + public QueryMetrics nodeTime(long timeNs) { - defaultTimeMetric(emitter, "query/node/time", timeNs); + return defaultTimeMetric("query/node/time", timeNs); } - private void defaultTimeMetric(ServiceEmitter emitter, String metricName, long timeNs) + private QueryMetrics defaultTimeMetric(String metricName, long timeNs) { - emitter.emit(builder.build(metricName, TimeUnit.NANOSECONDS.toMillis(timeNs))); + metrics.put(metricName, TimeUnit.NANOSECONDS.toMillis(timeNs)); + return this; } @Override - public void nodeBytes(ServiceEmitter emitter, long byteCount) + public QueryMetrics nodeBytes(long byteCount) { - emitter.emit(builder.build("query/node/bytes", byteCount)); + metrics.put("query/node/bytes", byteCount); + return this; + } + + @Override + public void emit(ServiceEmitter emitter) + { + for (Map.Entry metric : metrics.entrySet()) { + emitter.emit(builder.build(metric.getKey(), metric.getValue())); + } + metrics.clear(); } } diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index 0015a488ffe1..d403fa1110de 100644 --- a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java @@ -117,11 +117,12 @@ public void after(boolean isDone, Throwable thrown) queryMetrics.status("short"); } long timeTakenNs = System.nanoTime() - startTimeNs; - metric.emit(queryMetrics, emitter, timeTakenNs); + metric.apply(queryMetrics, timeTakenNs); if (creationTimeNs > 0) { - queryMetrics.waitTime(emitter, startTimeNs - creationTimeNs); + queryMetrics.waitTime(startTimeNs - creationTimeNs); } + queryMetrics.emit(emitter); } } ); diff --git a/processing/src/main/java/io/druid/query/QueryMetric.java b/processing/src/main/java/io/druid/query/QueryMetric.java index 4ff98c67f72f..4eb27940dad2 100644 --- a/processing/src/main/java/io/druid/query/QueryMetric.java +++ b/processing/src/main/java/io/druid/query/QueryMetric.java @@ -19,8 +19,6 @@ package io.druid.query; -import com.metamx.emitter.service.ServiceEmitter; - /** * Used in {@link MetricsEmittingQueryRunner} as a binder to a particular metric emitting method of {@link QueryMetrics} */ @@ -28,25 +26,25 @@ public enum QueryMetric { SEGMENT_TIME { @Override - public void emit(QueryMetrics metrics, ServiceEmitter emitter, long timeNs) + public void apply(QueryMetrics metrics, long timeNs) { - metrics.segmentTime(emitter, timeNs); + metrics.segmentTime(timeNs); } }, SEGMENT_AND_CACHE_TIME { @Override - public void emit(QueryMetrics metrics, ServiceEmitter emitter, long timeNs) + public void apply(QueryMetrics metrics, long timeNs) { - metrics.segmentAndCacheTime(emitter, timeNs); + metrics.segmentAndCacheTime(timeNs); } }, INTERVAL_CHUNK_TIME { @Override - public void emit(QueryMetrics metrics, ServiceEmitter emitter, long timeNs) + public void apply(QueryMetrics metrics, long timeNs) { - metrics.intervalChunkTime(emitter, timeNs); + metrics.intervalChunkTime(timeNs); } }; - public abstract void emit(QueryMetrics metrics, ServiceEmitter emitter, long value); + public abstract void apply(QueryMetrics metrics, long value); } diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index 393b0d6f2f18..a6575d9067aa 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -53,10 +53,12 @@ * Types of methods in this interface * ---------------------------------- * 1. Methods, pulling some dimensions from the query object. These methods are used to populate the metric before the - * query is run. These methods accept a single `QueryType query` parameter. + * query is run. These methods accept a single `QueryType query` parameter. {@link #query(Query)} calls all methods + * of this type, hence pulling all available information from the query object as dimensions. * 2. Methods for general dimensions, which become known in the process of the query execution or after the query is * completed. - * 3. Methods for emitting metrics, they accept {@link ServiceEmitter} as the first parameter. + * 3. Methods to register metrics to be emitted later in bulk via {@link #emit(ServiceEmitter)}. These methods + * return this QueryMetrics object back for chaining. * * * Implementors expectations @@ -67,6 +69,8 @@ * to emit new dimension or metric, and the user handles them manually: if the new dimension or metric is useful and not * very expensive to process and store then emit, skip (see above Goals, 1.) otherwise. * + *

QueryMetrics is designed for use from a single thread, implementations shouldn't care about thread-safety. + * * * Adding new methods to QueryMetrics * ---------------------------------- @@ -137,23 +141,58 @@ public interface QueryMetrics> void success(boolean success); - void queryTime(ServiceEmitter emitter, long timeNs); + /** + * Registers "query time" metric. + */ + QueryMetrics queryTime(long timeNs); + + /** + * Registers "query bytes" metric. + */ + QueryMetrics queryBytes(long byteCount); - void queryBytes(ServiceEmitter emitter, long byteCount); + /** + * Registers "wait time" metric. + */ + QueryMetrics waitTime(long timeNs); - void waitTime(ServiceEmitter emitter, long timeNs); + /** + * Registers "segment time" metric. + */ + QueryMetrics segmentTime(long timeNs); - void segmentTime(ServiceEmitter emitter, long timeNs); + /** + * Registers "segmentAndCache time" metric. + */ + QueryMetrics segmentAndCacheTime(long timeNs); - void segmentAndCacheTime(ServiceEmitter emitter, long timeNs); + /** + * Registers "interval chunk time" metric. + */ + QueryMetrics intervalChunkTime(long timeNs); - void intervalChunkTime(ServiceEmitter emitter, long timeNs); + /** + * Registers "cpu time" metric. + */ + QueryMetrics cpuTime(long timeNs); - void cpuTime(ServiceEmitter emitter, long timeNs); + /** + * Registers "time to first byte" metric. + */ + QueryMetrics nodeTimeToFirstByte(long timeNs); - void nodeTimeToFirstByte(ServiceEmitter emitter, long timeNs); + /** + * Registers "node time" metric. + */ + QueryMetrics nodeTime(long timeNs); - void nodeTime(ServiceEmitter emitter, long timeNs); + /** + * Registers "node bytes" metric. + */ + QueryMetrics nodeBytes(long byteCount); - void nodeBytes(ServiceEmitter emitter, long byteCount); + /** + * Emits all metrics, registered since the last {@code emit()} call on this QueryMetrics object. + */ + void emit(ServiceEmitter emitter); } diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index 7eb3f5f02acd..b2527851989a 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -178,7 +178,7 @@ public ClientResponse handleResponse(HttpResponse response) { log.debug("Initial response from url[%s] for queryId[%s]", url, query.getId()); responseStartTimeNs = System.nanoTime(); - queryMetrics.nodeTimeToFirstByte(emitter, responseStartTimeNs - requestStartTimeNs); + queryMetrics.nodeTimeToFirstByte(responseStartTimeNs - requestStartTimeNs).emit(emitter); try { final String responseContext = response.headers().get("X-Druid-Response-Context"); @@ -277,8 +277,9 @@ public ClientResponse done(ClientResponse clientRespon TimeUnit.NANOSECONDS.toMillis(nodeTimeNs), byteCount.get() / TimeUnit.NANOSECONDS.toSeconds(nodeTimeNs) ); - queryMetrics.nodeTime(emitter, nodeTimeNs); - queryMetrics.nodeBytes(emitter, byteCount.get()); + queryMetrics.nodeTime(nodeTimeNs); + queryMetrics.nodeBytes(byteCount.get()); + queryMetrics.emit(emitter); synchronized (done) { try { // An empty byte array is put at the end to give the SequenceInputStream.close() as something to close out diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 7f8f885315b4..f323e20055f9 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -400,7 +400,7 @@ public void onComplete(Result result) query, req.getRemoteAddr() ); - queryMetrics.queryTime(emitter, requestTimeNs); + queryMetrics.queryTime(requestTimeNs).emit(emitter); requestLogger.log( new RequestLogLine( new DateTime(), diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index fbdb7e6765ce..01dc1ae1bbce 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -261,14 +261,14 @@ public void write(OutputStream outputStream) throws IOException, WebApplicationE req.getRemoteAddr() ); queryMetrics.success(true); - queryMetrics.queryTime(emitter, queryTimeNs); + queryMetrics.queryTime(queryTimeNs).emit(emitter); DruidMetrics.makeRequestMetrics( queryMetricsFactory, theToolChest, theQuery, req.getRemoteAddr() - ).queryBytes(emitter, os.getCount()); + ).queryBytes(os.getCount()).emit(emitter); requestLogger.log( new RequestLogLine( @@ -329,7 +329,7 @@ public void write(OutputStream outputStream) throws IOException, WebApplicationE req.getRemoteAddr() ); queryMetrics.success(false); - queryMetrics.queryTime(emitter, queryTimeNs); + queryMetrics.queryTime(queryTimeNs).emit(emitter); requestLogger.log( new RequestLogLine( new DateTime(TimeUnit.NANOSECONDS.toMillis(startNs)), @@ -374,7 +374,7 @@ public void write(OutputStream outputStream) throws IOException, WebApplicationE req.getRemoteAddr() ); queryMetrics.success(false); - queryMetrics.queryTime(emitter, queryTimeNs); + queryMetrics.queryTime(queryTimeNs).emit(emitter); requestLogger.log( new RequestLogLine( new DateTime(TimeUnit.NANOSECONDS.toMillis(startNs)), From d0a78fb59585d5ad63811f9b15af993a3c565e03 Mon Sep 17 00:00:00 2001 From: leventov Date: Tue, 21 Feb 2017 15:34:43 -0600 Subject: [PATCH 04/17] Fixes --- .../io/druid/query/MetricsEmittingQueryRunner.java | 2 +- .../src/main/java/io/druid/query/QueryMetric.java | 11 ++++++----- .../src/main/java/io/druid/query/QueryMetrics.java | 2 +- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index d403fa1110de..2a0011b341a4 100644 --- a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java @@ -117,7 +117,7 @@ public void after(boolean isDone, Throwable thrown) queryMetrics.status("short"); } long timeTakenNs = System.nanoTime() - startTimeNs; - metric.apply(queryMetrics, timeTakenNs); + metric.register(queryMetrics, timeTakenNs); if (creationTimeNs > 0) { queryMetrics.waitTime(startTimeNs - creationTimeNs); diff --git a/processing/src/main/java/io/druid/query/QueryMetric.java b/processing/src/main/java/io/druid/query/QueryMetric.java index 4eb27940dad2..5cbfbf4efd78 100644 --- a/processing/src/main/java/io/druid/query/QueryMetric.java +++ b/processing/src/main/java/io/druid/query/QueryMetric.java @@ -20,31 +20,32 @@ package io.druid.query; /** - * Used in {@link MetricsEmittingQueryRunner} as a binder to a particular metric emitting method of {@link QueryMetrics} + * Used in {@link MetricsEmittingQueryRunner} as a binder to a particular metric registering method of + * {@link QueryMetrics}. */ public enum QueryMetric { SEGMENT_TIME { @Override - public void apply(QueryMetrics metrics, long timeNs) + public void register(QueryMetrics metrics, long timeNs) { metrics.segmentTime(timeNs); } }, SEGMENT_AND_CACHE_TIME { @Override - public void apply(QueryMetrics metrics, long timeNs) + public void register(QueryMetrics metrics, long timeNs) { metrics.segmentAndCacheTime(timeNs); } }, INTERVAL_CHUNK_TIME { @Override - public void apply(QueryMetrics metrics, long timeNs) + public void register(QueryMetrics metrics, long timeNs) { metrics.intervalChunkTime(timeNs); } }; - public abstract void apply(QueryMetrics metrics, long value); + public abstract void register(QueryMetrics metrics, long value); } diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index a6575d9067aa..a92bc570e0ef 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -83,7 +83,7 @@ * method, and as much work as possible is done in the implementations of this method, if they decide to emit this * dimension. * - * 3. When adding a new method for emitting metrics, make it to accept the metric value in the smallest reasonable + * 3. When adding a new method for registering metrics, make it to accept the metric value in the smallest reasonable * unit (i. e. nanoseconds for time metrics, bytes for metrics of data size, etc.), allowing the implementations of * this method to round the value up to more coarse-grained units, if they don't need the maximum precision. * From 22fcb1072f451a670e154f44b3f9bdc68c610ae6 Mon Sep 17 00:00:00 2001 From: leventov Date: Tue, 21 Feb 2017 15:35:47 -0600 Subject: [PATCH 05/17] Fix --- processing/src/main/java/io/druid/query/QueryMetrics.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index a92bc570e0ef..0d42b6d74c1e 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -32,10 +32,10 @@ * --------------------- * 1. Skipping or partial filtering of particular dimensions or metrics entirely. Implementation could leave the body * of the corresponding method empty, or implement random filtering like: - * public void cpuTime(ServiceEmitter emitter, long timeNs) + * public void cpuTime(long timeNs) * { * if (ThreadLocalRandom.current().nextDouble() < 0.1) { - * super.cpuTime(emitter, timeNs); + * super.cpuTime(timeNs); * } * } * From 08a9ab2acb52dbc79d2c63aa4445b69bda3d712b Mon Sep 17 00:00:00 2001 From: leventov Date: Tue, 21 Feb 2017 15:43:15 -0600 Subject: [PATCH 06/17] Javadoc fixes --- .../src/main/java/io/druid/query/QueryMetrics.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index 0d42b6d74c1e..12ac46aab8e0 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -55,7 +55,7 @@ * 1. Methods, pulling some dimensions from the query object. These methods are used to populate the metric before the * query is run. These methods accept a single `QueryType query` parameter. {@link #query(Query)} calls all methods * of this type, hence pulling all available information from the query object as dimensions. - * 2. Methods for general dimensions, which become known in the process of the query execution or after the query is + * 2. Methods for setting dimensions, which become known in the process of the query execution or after the query is * completed. * 3. Methods to register metrics to be emitted later in bulk via {@link #emit(ServiceEmitter)}. These methods * return this QueryMetrics object back for chaining. @@ -74,14 +74,14 @@ * * Adding new methods to QueryMetrics * ---------------------------------- - * 1. When adding a new method for emitting a dimension, which could be pulled from the query object, always make them + * 1. When adding a new method for setting a dimension, which could be pulled from the query object, always make them * accept a single `QueryType query` parameter, letting the implementations to do all the work of carving the dimension * value out of the query object. * - * 2. When adding a new method for emitting a dimension, which becomes known in the process of the query execution or - * after the query is completed, design them so that as little work as possible is done for preparing arguments for this - * method, and as much work as possible is done in the implementations of this method, if they decide to emit this - * dimension. + * 2. When adding a new method for setting a dimension, which becomes known in the process of the query execution or + * after the query is completed, design it so that as little work as possible is done for preparing arguments for this + * method, and as much work as possible is done in the implementations of this method, if they decide to actually emit + * this dimension. * * 3. When adding a new method for registering metrics, make it to accept the metric value in the smallest reasonable * unit (i. e. nanoseconds for time metrics, bytes for metrics of data size, etc.), allowing the implementations of From 8202d4cae181bc6b775a3157753a96f72c7bb3a6 Mon Sep 17 00:00:00 2001 From: leventov Date: Tue, 21 Feb 2017 16:31:50 -0600 Subject: [PATCH 07/17] Typo --- .../src/main/java/io/druid/query/DefaultQueryMetrics.java | 4 ++-- processing/src/main/java/io/druid/query/QueryMetrics.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index 5c84a28f6045..a3c98bf811f9 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -129,9 +129,9 @@ public void server(String host) } @Override - public void remoteAddress(String removeAddress) + public void remoteAddress(String remoteAddress) { - builder.setDimension("removeAddress", removeAddress); + builder.setDimension("remoteAddress", remoteAddress); } @Override diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index 12ac46aab8e0..d61f4633ebaf 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -133,7 +133,7 @@ public interface QueryMetrics> void server(String host); - void remoteAddress(String removeAddress); + void remoteAddress(String remoteAddress); void userDimensions(Map userDimensions); From a5c81afd8c44ff99369ecc5cb93b281a244aa6e3 Mon Sep 17 00:00:00 2001 From: leventov Date: Tue, 21 Feb 2017 16:32:06 -0600 Subject: [PATCH 08/17] Use DefaultObjectMapper --- .../main/java/io/druid/query/DefaultQueryMetricsFactory.java | 3 ++- .../java/io/druid/server/coordination/ServerManagerTest.java | 3 +-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/DefaultQueryMetricsFactory.java index 3b787301277e..3acaec0b5781 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetricsFactory.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; +import io.druid.jackson.DefaultObjectMapper; import io.druid.query.groupby.DefaultGroupByQueryMetrics; import io.druid.query.groupby.GroupByQueryMetrics; import io.druid.query.timeseries.DefaultTimeseriesQueryMetrics; @@ -30,7 +31,7 @@ public class DefaultQueryMetricsFactory implements QueryMetricsFactory { - private static final DefaultQueryMetricsFactory INSTANCE = new DefaultQueryMetricsFactory(new ObjectMapper()); + private static final DefaultQueryMetricsFactory INSTANCE = new DefaultQueryMetricsFactory(new DefaultObjectMapper()); public static DefaultQueryMetricsFactory instance() { diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 5e85ce579feb..5669ff882cf6 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -20,7 +20,6 @@ package io.druid.server.coordination; import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Throwables; @@ -577,7 +576,7 @@ public QueryRunner mergeResults(QueryRunner runner) @Override public QueryMetrics> makeMetrics(QueryType query) { - return new DefaultQueryMetrics<>(new ObjectMapper()); + return new DefaultQueryMetrics<>(new DefaultObjectMapper()); } @Override From 52c451e991400fc9e020456179258112bd7747c4 Mon Sep 17 00:00:00 2001 From: leventov Date: Mon, 20 Mar 2017 16:51:57 -0600 Subject: [PATCH 09/17] Add tests --- .../io/druid/query/DefaultQueryMetrics.java | 14 +- .../java/io/druid/query/CachingEmitter.java | 56 +++++++ .../druid/query/DefaultQueryMetricsTest.java | 157 ++++++++++++++++++ .../DefaultGroupByQueryMetricsTest.java | 126 ++++++++++++++ .../DefaultTimeseriesQueryMetricsTest.java | 102 ++++++++++++ .../topn/DefaultTopNQueryMetricsTest.java | 108 ++++++++++++ 6 files changed, 550 insertions(+), 13 deletions(-) create mode 100644 processing/src/test/java/io/druid/query/CachingEmitter.java create mode 100644 processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java create mode 100644 processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java create mode 100644 processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java create mode 100644 processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index a3c98bf811f9..31b5441a9446 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -21,10 +21,8 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import org.joda.time.Interval; @@ -72,17 +70,7 @@ public void interval(QueryType query) { builder.setDimension( DruidMetrics.INTERVAL, - Lists.transform( - query.getIntervals(), - new Function() - { - @Override - public String apply(Interval input) - { - return input.toString(); - } - } - ).toArray(new String[query.getIntervals().size()]) + query.getIntervals().stream().map(Interval::toString).toArray(String[]::new) ); } diff --git a/processing/src/test/java/io/druid/query/CachingEmitter.java b/processing/src/test/java/io/druid/query/CachingEmitter.java new file mode 100644 index 000000000000..aa2663904744 --- /dev/null +++ b/processing/src/test/java/io/druid/query/CachingEmitter.java @@ -0,0 +1,56 @@ +/* + * 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.query; + +import com.metamx.emitter.core.Emitter; +import com.metamx.emitter.core.Event; + +import java.io.IOException; + +public class CachingEmitter implements Emitter +{ + private Event lastEmittedEvent; + + public Event getLastEmittedEvent() + { + return lastEmittedEvent; + } + + @Override + public void start() + { + } + + @Override + public void emit(Event event) + { + lastEmittedEvent = event; + } + + @Override + public void flush() throws IOException + { + } + + @Override + public void close() throws IOException + { + } +} diff --git a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java new file mode 100644 index 000000000000..15236e01b8d9 --- /dev/null +++ b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java @@ -0,0 +1,157 @@ +/* + * 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.query; + +import com.google.common.collect.ImmutableSet; +import com.metamx.emitter.service.ServiceEmitter; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.granularity.Granularities; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.ListFilteredDimensionSpec; +import io.druid.query.filter.SelectorDimFilter; +import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNQueryBuilder; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class DefaultQueryMetricsTest +{ + + /** + * Tests that passed a query {@link DefaultQueryMetrics} produces events with a certain set of dimensions, no more, + * no less. + */ + @Test + public void testDefaultQueryMetricsQuery() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(new DefaultObjectMapper()); + TopNQuery query = new TopNQueryBuilder() + .dataSource("xx") + .granularity(Granularities.ALL) + .dimension(new ListFilteredDimensionSpec( + new DefaultDimensionSpec("tags", "tags"), + ImmutableSet.of("t3"), + null + )) + .metric("count") + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators(Collections.singletonList(new CountAggregatorFactory("count"))) + .threshold(5) + .filters(new SelectorDimFilter("tags", "t3", null)) + .build(); + queryMetrics.query(query); + + queryMetrics.queryTime(0).emit(serviceEmitter); + Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals(12, actualEvent.size()); + Assert.assertTrue(actualEvent.containsKey("feed")); + Assert.assertTrue(actualEvent.containsKey("timestamp")); + Assert.assertEquals("", actualEvent.get("host")); + Assert.assertEquals("", actualEvent.get("service")); + Assert.assertEquals("xx", actualEvent.get(DruidMetrics.DATASOURCE)); + Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); + List expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals(); + List expectedStringIntervals = + expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList()); + Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL)); + Assert.assertEquals("true", actualEvent.get("hasFilters")); + Assert.assertEquals(expectedIntervals.get(0).toDuration().toString(), actualEvent.get("duration")); + Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); + Assert.assertEquals("query/time", actualEvent.get("metric")); + Assert.assertEquals(0L, actualEvent.get("value")); + } + + @Test + public void testDefaultQueryMetricsMetricNamesAndUnits() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultQueryMetrics> queryMetrics = new DefaultQueryMetrics<>(new DefaultObjectMapper()); + testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); + } + + public static void testQueryMetricsDefaultMetricNamesAndUnits( + CachingEmitter cachingEmitter, + ServiceEmitter serviceEmitter, + QueryMetrics> queryMetrics + ) + { + queryMetrics.queryTime(1000001).emit(serviceEmitter); + Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals("query/time", actualEvent.get("metric")); + // query/time and most metrics below are measured in milliseconds by default + Assert.assertEquals(1L, actualEvent.get("value")); + + queryMetrics.waitTime(2000001).emit(serviceEmitter); + actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals("query/wait/time", actualEvent.get("metric")); + Assert.assertEquals(2L, actualEvent.get("value")); + + queryMetrics.segmentTime(3000001).emit(serviceEmitter); + actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals("query/segment/time", actualEvent.get("metric")); + Assert.assertEquals(3L, actualEvent.get("value")); + + queryMetrics.segmentAndCacheTime(4000001).emit(serviceEmitter); + actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals("query/segmentAndCache/time", actualEvent.get("metric")); + Assert.assertEquals(4L, actualEvent.get("value")); + + queryMetrics.intervalChunkTime(5000001).emit(serviceEmitter); + actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals("query/intervalChunk/time", actualEvent.get("metric")); + Assert.assertEquals(5L, actualEvent.get("value")); + + queryMetrics.cpuTime(6000001).emit(serviceEmitter); + actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals("query/cpu/time", actualEvent.get("metric")); + // CPU time is measured in microseconds by default + Assert.assertEquals(6000L, actualEvent.get("value")); + + queryMetrics.nodeTimeToFirstByte(7000001).emit(serviceEmitter); + actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals("query/node/ttfb", actualEvent.get("metric")); + Assert.assertEquals(7L, actualEvent.get("value")); + + queryMetrics.nodeTime(8000001).emit(serviceEmitter); + actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals("query/node/time", actualEvent.get("metric")); + Assert.assertEquals(8L, actualEvent.get("value")); + + queryMetrics.queryBytes(9).emit(serviceEmitter); + actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals("query/bytes", actualEvent.get("metric")); + Assert.assertEquals(9L, actualEvent.get("value")); + + queryMetrics.nodeBytes(10).emit(serviceEmitter); + actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals("query/node/bytes", actualEvent.get("metric")); + Assert.assertEquals(10L, actualEvent.get("value")); + } +} diff --git a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java new file mode 100644 index 000000000000..610bae3ea8da --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java @@ -0,0 +1,126 @@ +/* + * 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.query.groupby; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.metamx.emitter.service.ServiceEmitter; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.granularity.PeriodGranularity; +import io.druid.query.CachingEmitter; +import io.druid.query.DefaultQueryMetricsTest; +import io.druid.query.DruidMetrics; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.dimension.ExtractionDimensionSpec; +import io.druid.query.extraction.MapLookupExtractor; +import io.druid.query.filter.SelectorDimFilter; +import io.druid.query.lookup.LookupExtractionFn; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; + +public class DefaultGroupByQueryMetricsTest +{ + + /** + * Tests that passed a query {@link DefaultGroupByQueryMetrics} produces events with a certain set of dimensions, + * no more, no less. + */ + @Test + public void testDefaultGroupByQueryMetricsQuery() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultGroupByQueryMetrics queryMetrics = new DefaultGroupByQueryMetrics(new DefaultObjectMapper()); + GroupByQuery.Builder builder = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-04-02/2011-04-04") + .setDimensions( + Lists.newArrayList( + new ExtractionDimensionSpec( + "quality", + "alias", + new LookupExtractionFn( + new MapLookupExtractor( + ImmutableMap.of( + "mezzanine", + "mezzanine0" + ), + false + ), false, null, true, + false + ) + ) + ) + ) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)) + .setDimFilter(new SelectorDimFilter("quality", "mezzanine", null)) + .setContext(ImmutableMap.of("bySegment", true)); + GroupByQuery query = builder.build(); + queryMetrics.query(query); + + queryMetrics.queryTime(0).emit(serviceEmitter); + Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals(15, actualEvent.size()); + Assert.assertTrue(actualEvent.containsKey("feed")); + Assert.assertTrue(actualEvent.containsKey("timestamp")); + Assert.assertEquals("", actualEvent.get("host")); + Assert.assertEquals("", actualEvent.get("service")); + Assert.assertEquals(QueryRunnerTestHelper.dataSource, actualEvent.get(DruidMetrics.DATASOURCE)); + Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); + Interval expectedInterval = new Interval("2011-04-02/2011-04-04"); + Assert.assertEquals(Collections.singletonList(expectedInterval.toString()), actualEvent.get(DruidMetrics.INTERVAL)); + Assert.assertEquals("true", actualEvent.get("hasFilters")); + Assert.assertEquals(expectedInterval.toDuration().toString(), actualEvent.get("duration")); + Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); + + // GroupBy-specific dimensions + Assert.assertEquals("1", actualEvent.get("numDimensions")); + Assert.assertEquals("2", actualEvent.get("numMetrics")); + Assert.assertEquals("0", actualEvent.get("numComplexMetrics")); + + // Metric + Assert.assertEquals("query/time", actualEvent.get("metric")); + Assert.assertEquals(0L, actualEvent.get("value")); + } + + @Test + public void testDefaultGroupByQueryMetricsMetricNamesAndUnits() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultGroupByQueryMetrics queryMetrics = new DefaultGroupByQueryMetrics(new DefaultObjectMapper()); + DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); + } +} diff --git a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java new file mode 100644 index 000000000000..a50eeb3ddf68 --- /dev/null +++ b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java @@ -0,0 +1,102 @@ +/* + * 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.query.timeseries; + +import com.metamx.emitter.service.ServiceEmitter; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.CachingEmitter; +import io.druid.query.DefaultQueryMetricsTest; +import io.druid.query.DruidMetrics; +import io.druid.query.Druids; +import io.druid.query.QueryRunnerTestHelper; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class DefaultTimeseriesQueryMetricsTest +{ + + /** + * Tests that passed a query {@link DefaultTimeseriesQueryMetrics} produces events with a certain set of dimensions, + * no more, no less. + */ + @Test + public void testDefaultTimeseriesQueryMetricsQuery() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultTimeseriesQueryMetrics queryMetrics = new DefaultTimeseriesQueryMetrics(new DefaultObjectMapper()); + TimeseriesQuery query = Druids + .newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + QueryRunnerTestHelper.indexDoubleSum + ) + ) + .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .descending(true) + .build(); + queryMetrics.query(query); + + queryMetrics.queryTime(0).emit(serviceEmitter); + Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals(14, actualEvent.size()); + Assert.assertTrue(actualEvent.containsKey("feed")); + Assert.assertTrue(actualEvent.containsKey("timestamp")); + Assert.assertEquals("", actualEvent.get("host")); + Assert.assertEquals("", actualEvent.get("service")); + Assert.assertEquals(QueryRunnerTestHelper.dataSource, actualEvent.get(DruidMetrics.DATASOURCE)); + Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); + List expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals(); + List expectedStringIntervals = + expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList()); + Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL)); + Assert.assertEquals("false", actualEvent.get("hasFilters")); + Assert.assertEquals(expectedIntervals.get(0).toDuration().toString(), actualEvent.get("duration")); + Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); + + // Timeseries-specific dimensions + Assert.assertEquals("2", actualEvent.get("numMetrics")); + Assert.assertEquals("0", actualEvent.get("numComplexMetrics")); + + // Metric + Assert.assertEquals("query/time", actualEvent.get("metric")); + Assert.assertEquals(0L, actualEvent.get("value")); + } + + @Test + public void testDefaultTimeseriesQueryMetricsMetricNamesAndUnits() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultTimeseriesQueryMetrics queryMetrics = new DefaultTimeseriesQueryMetrics(new DefaultObjectMapper()); + DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); + } +} diff --git a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java new file mode 100644 index 000000000000..c53a981bfc0e --- /dev/null +++ b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java @@ -0,0 +1,108 @@ +/* + * 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.query.topn; + +import com.google.common.collect.ImmutableSet; +import com.metamx.emitter.service.ServiceEmitter; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.granularity.Granularities; +import io.druid.query.CachingEmitter; +import io.druid.query.DefaultQueryMetricsTest; +import io.druid.query.DruidMetrics; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.ListFilteredDimensionSpec; +import io.druid.query.filter.SelectorDimFilter; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class DefaultTopNQueryMetricsTest +{ + + /** + * Tests that passed a query {@link DefaultTopNQueryMetrics} produces events with a certain set of dimensions, + * no more, no less. + */ + @Test + public void testDefaultTopNQueryMetricsQuery() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultTopNQueryMetrics queryMetrics = new DefaultTopNQueryMetrics(new DefaultObjectMapper()); + TopNQuery query = new TopNQueryBuilder() + .dataSource("xx") + .granularity(Granularities.ALL) + .dimension(new ListFilteredDimensionSpec( + new DefaultDimensionSpec("tags", "tags"), + ImmutableSet.of("t3"), + null + )) + .metric("count") + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators(Collections.singletonList(new CountAggregatorFactory("count"))) + .threshold(5) + .filters(new SelectorDimFilter("tags", "t3", null)) + .build(); + queryMetrics.query(query); + + queryMetrics.queryTime(0).emit(serviceEmitter); + Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals(16, actualEvent.size()); + Assert.assertTrue(actualEvent.containsKey("feed")); + Assert.assertTrue(actualEvent.containsKey("timestamp")); + Assert.assertEquals("", actualEvent.get("host")); + Assert.assertEquals("", actualEvent.get("service")); + Assert.assertEquals("xx", actualEvent.get(DruidMetrics.DATASOURCE)); + Assert.assertEquals(query.getType(), actualEvent.get(DruidMetrics.TYPE)); + List expectedIntervals = QueryRunnerTestHelper.fullOnInterval.getIntervals(); + List expectedStringIntervals = + expectedIntervals.stream().map(Interval::toString).collect(Collectors.toList()); + Assert.assertEquals(expectedStringIntervals, actualEvent.get(DruidMetrics.INTERVAL)); + Assert.assertEquals("true", actualEvent.get("hasFilters")); + Assert.assertEquals(expectedIntervals.get(0).toDuration().toString(), actualEvent.get("duration")); + Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); + + // TopN-specific dimensions + Assert.assertEquals("5", actualEvent.get("threshold")); + Assert.assertEquals("tags", actualEvent.get("dimension")); + Assert.assertEquals("1", actualEvent.get("numMetrics")); + Assert.assertEquals("0", actualEvent.get("numComplexMetrics")); + + // Metric + Assert.assertEquals("query/time", actualEvent.get("metric")); + Assert.assertEquals(0L, actualEvent.get("value")); + } + + @Test + public void testDefaultTopNQueryMetricsMetricNamesAndUnits() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultTopNQueryMetrics queryMetrics = new DefaultTopNQueryMetrics(new DefaultObjectMapper()); + DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); + } +} From 24cd047d78dcc49ea4c4033481d7a431133b59d4 Mon Sep 17 00:00:00 2001 From: leventov Date: Wed, 22 Mar 2017 15:00:30 -0600 Subject: [PATCH 10/17] Address PR comments --- .../druid/query/CPUTimeMetricQueryRunner.java | 2 +- .../io/druid/query/DefaultQueryMetrics.java | 20 +++++----- .../query/MetricsEmittingQueryRunner.java | 2 +- .../main/java/io/druid/query/QueryMetric.java | 6 +-- .../java/io/druid/query/QueryMetrics.java | 40 ++++++++++--------- .../io/druid/query/QueryMetricsFactory.java | 2 +- .../java/io/druid/query/QueryToolChest.java | 6 +++ .../druid/query/DefaultQueryMetricsTest.java | 22 +++++----- .../DefaultGroupByQueryMetricsTest.java | 2 +- .../DefaultTimeseriesQueryMetricsTest.java | 2 +- .../topn/DefaultTopNQueryMetricsTest.java | 2 +- .../io/druid/client/DirectDruidClient.java | 6 +-- .../server/AsyncQueryForwardingServlet.java | 2 +- .../java/io/druid/server/QueryResource.java | 8 ++-- 14 files changed, 66 insertions(+), 56 deletions(-) diff --git a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java index a50f407c5ab8..7aefbb7ad472 100644 --- a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java +++ b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java @@ -84,7 +84,7 @@ public void after(boolean isDone, Throwable thrown) throws Exception if (report) { final long cpuTimeNs = cpuTimeAccumulator.get(); if (cpuTimeNs > 0) { - queryToolChest.makeMetrics(query).cpuTime(cpuTimeNs).emit(emitter); + queryToolChest.makeMetrics(query).reportCpuTime(cpuTimeNs).emit(emitter); } } } diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index 31b5441a9446..41b49a0daca0 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -143,57 +143,57 @@ public void success(boolean success) } @Override - public QueryMetrics queryTime(long timeNs) + public QueryMetrics reportQueryTime(long timeNs) { return defaultTimeMetric("query/time", timeNs); } @Override - public QueryMetrics queryBytes(long byteCount) + public QueryMetrics reportQueryBytes(long byteCount) { metrics.put("query/bytes", byteCount); return this; } @Override - public QueryMetrics waitTime(long timeNs) + public QueryMetrics reportWaitTime(long timeNs) { return defaultTimeMetric("query/wait/time", timeNs); } @Override - public QueryMetrics segmentTime(long timeNs) + public QueryMetrics reportSegmentTime(long timeNs) { return defaultTimeMetric("query/segment/time", timeNs); } @Override - public QueryMetrics segmentAndCacheTime(long timeNs) + public QueryMetrics reportSegmentAndCacheTime(long timeNs) { return defaultTimeMetric("query/segmentAndCache/time", timeNs); } @Override - public QueryMetrics intervalChunkTime(long timeNs) + public QueryMetrics reportIntervalChunkTime(long timeNs) { return defaultTimeMetric("query/intervalChunk/time", timeNs); } @Override - public QueryMetrics cpuTime(long timeNs) + public QueryMetrics reportCpuTime(long timeNs) { metrics.put("query/cpu/time", TimeUnit.NANOSECONDS.toMicros(timeNs)); return this; } @Override - public QueryMetrics nodeTimeToFirstByte(long timeNs) + public QueryMetrics reportNodeTimeToFirstByte(long timeNs) { return defaultTimeMetric("query/node/ttfb", timeNs); } @Override - public QueryMetrics nodeTime(long timeNs) + public QueryMetrics reportNodeTime(long timeNs) { return defaultTimeMetric("query/node/time", timeNs); } @@ -205,7 +205,7 @@ private QueryMetrics defaultTimeMetric(String metricName, long timeNs } @Override - public QueryMetrics nodeBytes(long byteCount) + public QueryMetrics reportNodeBytes(long byteCount) { metrics.put("query/node/bytes", byteCount); return this; diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index 2a0011b341a4..94fca2b54389 100644 --- a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java @@ -120,7 +120,7 @@ public void after(boolean isDone, Throwable thrown) metric.register(queryMetrics, timeTakenNs); if (creationTimeNs > 0) { - queryMetrics.waitTime(startTimeNs - creationTimeNs); + queryMetrics.reportWaitTime(startTimeNs - creationTimeNs); } queryMetrics.emit(emitter); } diff --git a/processing/src/main/java/io/druid/query/QueryMetric.java b/processing/src/main/java/io/druid/query/QueryMetric.java index 5cbfbf4efd78..aeb576d71063 100644 --- a/processing/src/main/java/io/druid/query/QueryMetric.java +++ b/processing/src/main/java/io/druid/query/QueryMetric.java @@ -29,21 +29,21 @@ public enum QueryMetric @Override public void register(QueryMetrics metrics, long timeNs) { - metrics.segmentTime(timeNs); + metrics.reportSegmentTime(timeNs); } }, SEGMENT_AND_CACHE_TIME { @Override public void register(QueryMetrics metrics, long timeNs) { - metrics.segmentAndCacheTime(timeNs); + metrics.reportSegmentAndCacheTime(timeNs); } }, INTERVAL_CHUNK_TIME { @Override public void register(QueryMetrics metrics, long timeNs) { - metrics.intervalChunkTime(timeNs); + metrics.reportIntervalChunkTime(timeNs); } }; diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index d61f4633ebaf..6690d89b641e 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -32,10 +32,10 @@ * --------------------- * 1. Skipping or partial filtering of particular dimensions or metrics entirely. Implementation could leave the body * of the corresponding method empty, or implement random filtering like: - * public void cpuTime(long timeNs) + * public void reportCpuTime(long timeNs) * { * if (ThreadLocalRandom.current().nextDouble() < 0.1) { - * super.cpuTime(timeNs); + * super.reportCpuTime(timeNs); * } * } * @@ -49,6 +49,9 @@ * * 4. Control over the dimension and metric names. * + * Here, "control" is provided to the operator of a Druid cluster, who would exercise that control through a + * site-specific extension adding a QueryMetricsFactory impl. + * * * Types of methods in this interface * ---------------------------------- @@ -58,16 +61,17 @@ * 2. Methods for setting dimensions, which become known in the process of the query execution or after the query is * completed. * 3. Methods to register metrics to be emitted later in bulk via {@link #emit(ServiceEmitter)}. These methods - * return this QueryMetrics object back for chaining. + * return this QueryMetrics object back for chaining. Names of these methods start with "report" prefix. * * * Implementors expectations * ------------------------- - * QueryMetrics is expected to be changed often, in every Druid release. Users who create their custom implementations - * of QueryMetrics should be ready to fix the code of their QueryMetrics (implement new methods) when they update Druid. - * Broken builds of custom extensions, containing custom QueryMetrics is the way to notify users that Druid core "wants" - * to emit new dimension or metric, and the user handles them manually: if the new dimension or metric is useful and not - * very expensive to process and store then emit, skip (see above Goals, 1.) otherwise. + * QueryMetrics is expected to be changed often, in every Druid release (including "patch" releases). Users who create + * their custom implementations of QueryMetrics should be ready to fix the code of their QueryMetrics (implement new + * methods) when they update Druid. Broken builds of custom extensions, containing custom QueryMetrics is the way to + * notify users that Druid core "wants" to emit new dimension or metric, and the user handles them manually: if the new + * dimension or metric is useful and not very expensive to process and store then emit, skip (see above Goals, 1.) + * otherwise. * *

QueryMetrics is designed for use from a single thread, implementations shouldn't care about thread-safety. * @@ -144,52 +148,52 @@ public interface QueryMetrics> /** * Registers "query time" metric. */ - QueryMetrics queryTime(long timeNs); + QueryMetrics reportQueryTime(long timeNs); /** * Registers "query bytes" metric. */ - QueryMetrics queryBytes(long byteCount); + QueryMetrics reportQueryBytes(long byteCount); /** * Registers "wait time" metric. */ - QueryMetrics waitTime(long timeNs); + QueryMetrics reportWaitTime(long timeNs); /** * Registers "segment time" metric. */ - QueryMetrics segmentTime(long timeNs); + QueryMetrics reportSegmentTime(long timeNs); /** * Registers "segmentAndCache time" metric. */ - QueryMetrics segmentAndCacheTime(long timeNs); + QueryMetrics reportSegmentAndCacheTime(long timeNs); /** * Registers "interval chunk time" metric. */ - QueryMetrics intervalChunkTime(long timeNs); + QueryMetrics reportIntervalChunkTime(long timeNs); /** * Registers "cpu time" metric. */ - QueryMetrics cpuTime(long timeNs); + QueryMetrics reportCpuTime(long timeNs); /** * Registers "time to first byte" metric. */ - QueryMetrics nodeTimeToFirstByte(long timeNs); + QueryMetrics reportNodeTimeToFirstByte(long timeNs); /** * Registers "node time" metric. */ - QueryMetrics nodeTime(long timeNs); + QueryMetrics reportNodeTime(long timeNs); /** * Registers "node bytes" metric. */ - QueryMetrics nodeBytes(long byteCount); + QueryMetrics reportNodeBytes(long byteCount); /** * Emits all metrics, registered since the last {@code emit()} call on this QueryMetrics object. diff --git a/processing/src/main/java/io/druid/query/QueryMetricsFactory.java b/processing/src/main/java/io/druid/query/QueryMetricsFactory.java index 5af813b0c70b..ecaa74eb441f 100644 --- a/processing/src/main/java/io/druid/query/QueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/QueryMetricsFactory.java @@ -29,7 +29,7 @@ public interface QueryMetricsFactory { /** - * Creates a generic {@link QueryMetrics} for query, which doesn't require a custom QueryMetrics subclass (i. e. not + * Creates a {@link QueryMetrics} for query, which doesn't have predefined QueryMetrics subclass (i. e. not * a topN, groupBy or timeseries query). This method must call {@link QueryMetrics#query(Query)} with the given query * on the created QueryMetrics object before returning. */ diff --git a/processing/src/main/java/io/druid/query/QueryToolChest.java b/processing/src/main/java/io/druid/query/QueryToolChest.java index d57a79a3f416..9cc6276cb5d0 100644 --- a/processing/src/main/java/io/druid/query/QueryToolChest.java +++ b/processing/src/main/java/io/druid/query/QueryToolChest.java @@ -49,6 +49,12 @@ public abstract class QueryToolChestPublic extensions (belonging to the Druid source tree) should use injected {@link QueryMetricsFactory} object + * for creating the QueryMetrics object, returned from this method. + * + *

This method should ensure that {@link QueryMetrics#query(Query)} is called with the given query passed on the + * created QueryMetrics object before returning. * * @param query The query that is being processed * diff --git a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java index 15236e01b8d9..63740e9e730c 100644 --- a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java @@ -67,7 +67,7 @@ public void testDefaultQueryMetricsQuery() .build(); queryMetrics.query(query); - queryMetrics.queryTime(0).emit(serviceEmitter); + queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals(12, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); @@ -102,54 +102,54 @@ public static void testQueryMetricsDefaultMetricNamesAndUnits( QueryMetrics> queryMetrics ) { - queryMetrics.queryTime(1000001).emit(serviceEmitter); + queryMetrics.reportQueryTime(1000001).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals("query/time", actualEvent.get("metric")); // query/time and most metrics below are measured in milliseconds by default Assert.assertEquals(1L, actualEvent.get("value")); - queryMetrics.waitTime(2000001).emit(serviceEmitter); + queryMetrics.reportWaitTime(2000001).emit(serviceEmitter); actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals("query/wait/time", actualEvent.get("metric")); Assert.assertEquals(2L, actualEvent.get("value")); - queryMetrics.segmentTime(3000001).emit(serviceEmitter); + queryMetrics.reportSegmentTime(3000001).emit(serviceEmitter); actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals("query/segment/time", actualEvent.get("metric")); Assert.assertEquals(3L, actualEvent.get("value")); - queryMetrics.segmentAndCacheTime(4000001).emit(serviceEmitter); + queryMetrics.reportSegmentAndCacheTime(4000001).emit(serviceEmitter); actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals("query/segmentAndCache/time", actualEvent.get("metric")); Assert.assertEquals(4L, actualEvent.get("value")); - queryMetrics.intervalChunkTime(5000001).emit(serviceEmitter); + queryMetrics.reportIntervalChunkTime(5000001).emit(serviceEmitter); actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals("query/intervalChunk/time", actualEvent.get("metric")); Assert.assertEquals(5L, actualEvent.get("value")); - queryMetrics.cpuTime(6000001).emit(serviceEmitter); + queryMetrics.reportCpuTime(6000001).emit(serviceEmitter); actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals("query/cpu/time", actualEvent.get("metric")); // CPU time is measured in microseconds by default Assert.assertEquals(6000L, actualEvent.get("value")); - queryMetrics.nodeTimeToFirstByte(7000001).emit(serviceEmitter); + queryMetrics.reportNodeTimeToFirstByte(7000001).emit(serviceEmitter); actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals("query/node/ttfb", actualEvent.get("metric")); Assert.assertEquals(7L, actualEvent.get("value")); - queryMetrics.nodeTime(8000001).emit(serviceEmitter); + queryMetrics.reportNodeTime(8000001).emit(serviceEmitter); actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals("query/node/time", actualEvent.get("metric")); Assert.assertEquals(8L, actualEvent.get("value")); - queryMetrics.queryBytes(9).emit(serviceEmitter); + queryMetrics.reportQueryBytes(9).emit(serviceEmitter); actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals("query/bytes", actualEvent.get("metric")); Assert.assertEquals(9L, actualEvent.get("value")); - queryMetrics.nodeBytes(10).emit(serviceEmitter); + queryMetrics.reportNodeBytes(10).emit(serviceEmitter); actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals("query/node/bytes", actualEvent.get("metric")); Assert.assertEquals(10L, actualEvent.get("value")); diff --git a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java index 610bae3ea8da..f04600c28181 100644 --- a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java @@ -90,7 +90,7 @@ public void testDefaultGroupByQueryMetricsQuery() GroupByQuery query = builder.build(); queryMetrics.query(query); - queryMetrics.queryTime(0).emit(serviceEmitter); + queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals(15, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); diff --git a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java index a50eeb3ddf68..fb03645c0675 100644 --- a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java @@ -65,7 +65,7 @@ public void testDefaultTimeseriesQueryMetricsQuery() .build(); queryMetrics.query(query); - queryMetrics.queryTime(0).emit(serviceEmitter); + queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals(14, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); diff --git a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java index c53a981bfc0e..185795fc1db0 100644 --- a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java @@ -69,7 +69,7 @@ public void testDefaultTopNQueryMetricsQuery() .build(); queryMetrics.query(query); - queryMetrics.queryTime(0).emit(serviceEmitter); + queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); Assert.assertEquals(16, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index b2527851989a..cd6401462a0c 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -178,7 +178,7 @@ public ClientResponse handleResponse(HttpResponse response) { log.debug("Initial response from url[%s] for queryId[%s]", url, query.getId()); responseStartTimeNs = System.nanoTime(); - queryMetrics.nodeTimeToFirstByte(responseStartTimeNs - requestStartTimeNs).emit(emitter); + queryMetrics.reportNodeTimeToFirstByte(responseStartTimeNs - requestStartTimeNs).emit(emitter); try { final String responseContext = response.headers().get("X-Druid-Response-Context"); @@ -277,8 +277,8 @@ public ClientResponse done(ClientResponse clientRespon TimeUnit.NANOSECONDS.toMillis(nodeTimeNs), byteCount.get() / TimeUnit.NANOSECONDS.toSeconds(nodeTimeNs) ); - queryMetrics.nodeTime(nodeTimeNs); - queryMetrics.nodeBytes(byteCount.get()); + queryMetrics.reportNodeTime(nodeTimeNs); + queryMetrics.reportNodeBytes(byteCount.get()); queryMetrics.emit(emitter); synchronized (done) { try { diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index f323e20055f9..5fe67551d122 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -400,7 +400,7 @@ public void onComplete(Result result) query, req.getRemoteAddr() ); - queryMetrics.queryTime(requestTimeNs).emit(emitter); + queryMetrics.reportQueryTime(requestTimeNs).emit(emitter); requestLogger.log( new RequestLogLine( new DateTime(), diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 15ddddadcc62..4d1f2b0a7f87 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -275,14 +275,14 @@ public void write(OutputStream outputStream) throws IOException, WebApplicationE req.getRemoteAddr() ); queryMetrics.success(true); - queryMetrics.queryTime(queryTimeNs).emit(emitter); + queryMetrics.reportQueryTime(queryTimeNs).emit(emitter); DruidMetrics.makeRequestMetrics( queryMetricsFactory, theToolChest, theQuery, req.getRemoteAddr() - ).queryBytes(os.getCount()).emit(emitter); + ).reportQueryBytes(os.getCount()).emit(emitter); requestLogger.log( new RequestLogLine( @@ -348,7 +348,7 @@ public void write(OutputStream outputStream) throws IOException, WebApplicationE req.getRemoteAddr() ); queryMetrics.success(false); - queryMetrics.queryTime(queryTimeNs).emit(emitter); + queryMetrics.reportQueryTime(queryTimeNs).emit(emitter); requestLogger.log( new RequestLogLine( new DateTime(TimeUnit.NANOSECONDS.toMillis(startNs)), @@ -393,7 +393,7 @@ public void write(OutputStream outputStream) throws IOException, WebApplicationE req.getRemoteAddr() ); queryMetrics.success(false); - queryMetrics.queryTime(queryTimeNs).emit(emitter); + queryMetrics.reportQueryTime(queryTimeNs).emit(emitter); requestLogger.log( new RequestLogLine( new DateTime(TimeUnit.NANOSECONDS.toMillis(startNs)), From c163e516b1171ad8ddd2e5bf1e484897f5701935 Mon Sep 17 00:00:00 2001 From: leventov Date: Wed, 22 Mar 2017 16:46:20 -0600 Subject: [PATCH 11/17] Remove QueryMetrics.userDimensions(); Rename QueryMetric.register() to report() --- .../io/druid/query/DefaultQueryMetrics.java | 20 +++++++++++-------- .../query/IntervalChunkingQueryRunner.java | 3 +-- .../query/MetricsEmittingQueryRunner.java | 17 ++++++++-------- .../main/java/io/druid/query/QueryMetric.java | 8 ++++---- .../java/io/druid/query/QueryMetrics.java | 9 +++++---- .../appenderator/SinkQuerySegmentWalker.java | 10 ++++------ .../server/coordination/ServerManager.java | 12 +++++------ 7 files changed, 41 insertions(+), 38 deletions(-) diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index 41b49a0daca0..a19053be8789 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -122,14 +122,6 @@ public void remoteAddress(String remoteAddress) builder.setDimension("remoteAddress", remoteAddress); } - @Override - public void userDimensions(Map userDimensions) - { - for (Map.Entry userDimension : userDimensions.entrySet()) { - builder.setDimension(userDimension.getKey(), userDimension.getValue()); - } - } - @Override public void status(String status) { @@ -142,6 +134,18 @@ public void success(boolean success) builder.setDimension("success", String.valueOf(success)); } + @Override + public void segment(String segmentIdentifier) + { + builder.setDimension("segment", segmentIdentifier); + } + + @Override + public void chunkInterval(Interval interval) + { + builder.setDimension("chunkInterval", interval.toString()); + } + @Override public QueryMetrics reportQueryTime(long timeNs) { diff --git a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java index f4e3287af7a9..961f80c61344 100644 --- a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java @@ -20,7 +20,6 @@ package io.druid.query; import com.google.common.base.Function; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.emitter.service.ServiceEmitter; import io.druid.java.util.common.granularity.PeriodGranularity; @@ -109,7 +108,7 @@ public Sequence apply(Interval singleInterval) toolChest, baseRunner, QueryMetric.INTERVAL_CHUNK_TIME, - ImmutableMap.of("chunkInterval", singleInterval.toString()) + queryMetrics -> queryMetrics.chunkInterval(singleInterval) ).withWaitMeasuredFromNow() ), executor, queryWatcher diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index 94fca2b54389..eb95707ebc1c 100644 --- a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java @@ -27,6 +27,7 @@ import io.druid.java.util.common.guava.Sequences; import java.util.Map; +import java.util.function.Consumer; /** */ @@ -37,7 +38,7 @@ public class MetricsEmittingQueryRunner implements QueryRunner private final QueryRunner queryRunner; private final long creationTimeNs; private final QueryMetric metric; - private final Map userDimensions; + private final Consumer>> applyCustomDimensions; private MetricsEmittingQueryRunner( ServiceEmitter emitter, @@ -45,7 +46,7 @@ private MetricsEmittingQueryRunner( QueryRunner queryRunner, long creationTimeNs, QueryMetric metric, - Map userDimensions + Consumer>> applyCustomDimensions ) { this.emitter = emitter; @@ -53,7 +54,7 @@ private MetricsEmittingQueryRunner( this.queryRunner = queryRunner; this.creationTimeNs = creationTimeNs; this.metric = metric; - this.userDimensions = userDimensions; + this.applyCustomDimensions = applyCustomDimensions; } public MetricsEmittingQueryRunner( @@ -61,10 +62,10 @@ public MetricsEmittingQueryRunner( QueryToolChest> queryToolChest, QueryRunner queryRunner, QueryMetric metric, - Map userDimensions + Consumer>> applyCustomDimensions ) { - this(emitter, queryToolChest, queryRunner, -1, metric, userDimensions); + this(emitter, queryToolChest, queryRunner, -1, metric, applyCustomDimensions); } public MetricsEmittingQueryRunner withWaitMeasuredFromNow() @@ -75,7 +76,7 @@ public MetricsEmittingQueryRunner withWaitMeasuredFromNow() queryRunner, System.nanoTime(), metric, - userDimensions + applyCustomDimensions ); } @@ -84,7 +85,7 @@ public Sequence run(final Query query, final Map responseC { final QueryMetrics> queryMetrics = queryToolChest.makeMetrics(query); - queryMetrics.userDimensions(userDimensions); + applyCustomDimensions.accept(queryMetrics); return Sequences.wrap( // Use LazySequence because want to account execution time of queryRunner.run() (it prepares the underlying @@ -117,7 +118,7 @@ public void after(boolean isDone, Throwable thrown) queryMetrics.status("short"); } long timeTakenNs = System.nanoTime() - startTimeNs; - metric.register(queryMetrics, timeTakenNs); + metric.report(queryMetrics, timeTakenNs); if (creationTimeNs > 0) { queryMetrics.reportWaitTime(startTimeNs - creationTimeNs); diff --git a/processing/src/main/java/io/druid/query/QueryMetric.java b/processing/src/main/java/io/druid/query/QueryMetric.java index aeb576d71063..45fa43e0a7e2 100644 --- a/processing/src/main/java/io/druid/query/QueryMetric.java +++ b/processing/src/main/java/io/druid/query/QueryMetric.java @@ -27,25 +27,25 @@ public enum QueryMetric { SEGMENT_TIME { @Override - public void register(QueryMetrics metrics, long timeNs) + public void report(QueryMetrics metrics, long timeNs) { metrics.reportSegmentTime(timeNs); } }, SEGMENT_AND_CACHE_TIME { @Override - public void register(QueryMetrics metrics, long timeNs) + public void report(QueryMetrics metrics, long timeNs) { metrics.reportSegmentAndCacheTime(timeNs); } }, INTERVAL_CHUNK_TIME { @Override - public void register(QueryMetrics metrics, long timeNs) + public void report(QueryMetrics metrics, long timeNs) { metrics.reportIntervalChunkTime(timeNs); } }; - public abstract void register(QueryMetrics metrics, long value); + public abstract void report(QueryMetrics metrics, long value); } diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index 6690d89b641e..38db803c4756 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -20,8 +20,7 @@ package io.druid.query; import com.metamx.emitter.service.ServiceEmitter; - -import java.util.Map; +import org.joda.time.Interval; /** * Abstraction wrapping {@link com.metamx.emitter.service.ServiceMetricEvent.Builder} and allowing to control what @@ -139,12 +138,14 @@ public interface QueryMetrics> void remoteAddress(String remoteAddress); - void userDimensions(Map userDimensions); - void status(String status); void success(boolean success); + void segment(String segmentIdentifier); + + void chunkInterval(Interval interval); + /** * Registers "query time" metric. */ diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 6cd266b129a2..8fe983e7fa58 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.MoreExecutors; import com.metamx.emitter.EmittingLogger; @@ -39,6 +38,7 @@ import io.druid.query.MetricsEmittingQueryRunner; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryMetric; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -47,7 +47,6 @@ import io.druid.query.QueryToolChest; import io.druid.query.ReportTimelineMissingSegmentQueryRunner; import io.druid.query.SegmentDescriptor; -import io.druid.query.QueryMetric; import io.druid.query.TableDataSource; import io.druid.query.spec.SpecificSegmentQueryRunner; import io.druid.query.spec.SpecificSegmentSpec; @@ -281,9 +280,8 @@ private QueryRunner withPerSinkMetrics( final AtomicLong cpuTimeAccumulator ) { - final ImmutableMap dims = ImmutableMap.of("segment", sinkSegmentIdentifier); - // Note: query/segmentAndCache/time and query/segment/time are effectively the same here. They don't split apart + // Note: SEGMENT_AND_CACHE_TIME and SEGMENT_TIME are effectively the same here. They don't split apart // cache vs. non-cache due to the fact that Sinks may be partially cached and partially uncached. Making this // better would need to involve another accumulator like the cpuTimeAccumulator that we could share with the // sinkRunner. @@ -297,10 +295,10 @@ private QueryRunner withPerSinkMetrics( queryToolChest, sinkRunner, QueryMetric.SEGMENT_TIME, - dims + queryMetrics -> queryMetrics.segment(sinkSegmentIdentifier) ), QueryMetric.SEGMENT_AND_CACHE_TIME, - dims + queryMetrics -> queryMetrics.segment(sinkSegmentIdentifier) ).withWaitMeasuredFromNow(), queryToolChest, emitter, diff --git a/server/src/main/java/io/druid/server/coordination/ServerManager.java b/server/src/main/java/io/druid/server/coordination/ServerManager.java index 7fde82b2c54f..5d570210a1fd 100644 --- a/server/src/main/java/io/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/io/druid/server/coordination/ServerManager.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; import com.google.inject.Inject; @@ -43,6 +42,7 @@ import io.druid.query.MetricsEmittingQueryRunner; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryMetric; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -51,7 +51,6 @@ import io.druid.query.ReferenceCountingSegmentQueryRunner; import io.druid.query.ReportTimelineMissingSegmentQueryRunner; import io.druid.query.SegmentDescriptor; -import io.druid.query.QueryMetric; import io.druid.query.TableDataSource; import io.druid.query.spec.SpecificSegmentQueryRunner; import io.druid.query.spec.SpecificSegmentSpec; @@ -414,16 +413,17 @@ private QueryRunner buildAndDecorateQueryRunner( ) { SpecificSegmentSpec segmentSpec = new SpecificSegmentSpec(segmentDescriptor); + String segmentId = adapter.getIdentifier(); return CPUTimeMetricQueryRunner.safeBuild( new SpecificSegmentQueryRunner( new MetricsEmittingQueryRunner( emitter, toolChest, new BySegmentQueryRunner( - adapter.getIdentifier(), + segmentId, adapter.getDataInterval().getStart(), new CachingQueryRunner( - adapter.getIdentifier(), + segmentId, segmentDescriptor, objectMapper, cache, @@ -433,14 +433,14 @@ private QueryRunner buildAndDecorateQueryRunner( toolChest, new ReferenceCountingSegmentQueryRunner(factory, adapter, segmentDescriptor), QueryMetric.SEGMENT_TIME, - ImmutableMap.of("segment", adapter.getIdentifier()) + queryMetrics -> queryMetrics.segment(segmentId) ), cachingExec, cacheConfig ) ), QueryMetric.SEGMENT_AND_CACHE_TIME, - ImmutableMap.of("segment", adapter.getIdentifier()) + queryMetrics -> queryMetrics.segment(segmentId) ).withWaitMeasuredFromNow(), segmentSpec ), From fcf1d1b6a9a8c2dfb2b7b41877165ad418e8cee5 Mon Sep 17 00:00:00 2001 From: leventov Date: Wed, 22 Mar 2017 19:47:03 -0600 Subject: [PATCH 12/17] Dedicated TopNQueryMetricsFactory, GroupByQueryMetricsFactory and TimeseriesQueryMetricsFactory --- .../query/scan/ScanQueryQueryToolChest.java | 6 +-- .../query/scan/MultiSegmentScanQueryTest.java | 4 +- .../druid/query/scan/ScanQueryRunnerTest.java | 4 +- ...=> DefaultGenericQueryMetricsFactory.java} | 31 ++---------- .../java/io/druid/query/DruidMetrics.java | 2 +- ...y.java => GenericQueryMetricsFactory.java} | 20 ++------ .../java/io/druid/query/QueryMetrics.java | 37 +++++++++++++- .../java/io/druid/query/QueryToolChest.java | 13 +++-- .../DataSourceQueryQueryToolChest.java | 6 +-- .../DefaultGroupByQueryMetricsFactory.java | 49 +++++++++++++++++++ .../groupby/GroupByQueryMetricsFactory.java | 26 ++++++++++ .../groupby/GroupByQueryQueryToolChest.java | 10 ++-- .../SegmentMetadataQueryQueryToolChest.java | 10 ++-- .../search/SearchQueryQueryToolChest.java | 10 ++-- .../select/SelectQueryQueryToolChest.java | 10 ++-- .../TimeBoundaryQueryQueryToolChest.java | 10 ++-- .../DefaultTimeseriesQueryMetricsFactory.java | 49 +++++++++++++++++++ .../TimeseriesQueryMetricsFactory.java | 26 ++++++++++ .../TimeseriesQueryQueryToolChest.java | 10 ++-- .../topn/DefaultTopNQueryMetricsFactory.java | 48 ++++++++++++++++++ .../query/topn/TopNQueryMetricsFactory.java | 26 ++++++++++ .../query/topn/TopNQueryQueryToolChest.java | 10 ++-- .../DataSourceMetadataQueryTest.java | 6 +-- .../io/druid/guice/QueryToolChestModule.java | 16 ++++-- .../server/AsyncQueryForwardingServlet.java | 6 +-- .../io/druid/server/BrokerQueryResource.java | 4 +- .../java/io/druid/server/QueryResource.java | 6 +-- .../AsyncQueryForwardingServletTest.java | 4 +- .../io/druid/server/QueryResourceTest.java | 10 ++-- 29 files changed, 352 insertions(+), 117 deletions(-) rename processing/src/main/java/io/druid/query/{DefaultQueryMetricsFactory.java => DefaultGenericQueryMetricsFactory.java} (56%) rename processing/src/main/java/io/druid/query/{QueryMetricsFactory.java => GenericQueryMetricsFactory.java} (64%) create mode 100644 processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetricsFactory.java create mode 100644 processing/src/main/java/io/druid/query/groupby/GroupByQueryMetricsFactory.java create mode 100644 processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsFactory.java create mode 100644 processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetricsFactory.java create mode 100644 processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetricsFactory.java create mode 100644 processing/src/main/java/io/druid/query/topn/TopNQueryMetricsFactory.java diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java index 0c7e5cb2f730..905952487798 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java @@ -27,7 +27,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.query.Query; import io.druid.query.QueryMetrics; -import io.druid.query.QueryMetricsFactory; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.aggregation.MetricManipulationFn; @@ -40,10 +40,10 @@ public class ScanQueryQueryToolChest extends QueryToolChest factory = new ScanQueryRunnerFactory( diff --git a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java index 5f6674db720b..ceeae9e52516 100644 --- a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java +++ b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java @@ -27,7 +27,7 @@ import com.google.common.collect.Sets; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequences; -import io.druid.query.DefaultQueryMetricsFactory; +import io.druid.query.DefaultGenericQueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.TableDataSource; @@ -97,7 +97,7 @@ public class ScanQueryRunnerTest public static final String[] V_0112_0114 = ObjectArrays.concat(V_0112, V_0113, String.class); private static final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest( - DefaultQueryMetricsFactory.instance() + DefaultGenericQueryMetricsFactory.instance() ); @Parameterized.Parameters(name = "{0}") diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java similarity index 56% rename from processing/src/main/java/io/druid/query/DefaultQueryMetricsFactory.java rename to processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java index 3acaec0b5781..27acd1d15829 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java @@ -22,18 +22,12 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.groupby.DefaultGroupByQueryMetrics; -import io.druid.query.groupby.GroupByQueryMetrics; -import io.druid.query.timeseries.DefaultTimeseriesQueryMetrics; -import io.druid.query.timeseries.TimeseriesQueryMetrics; -import io.druid.query.topn.DefaultTopNQueryMetrics; -import io.druid.query.topn.TopNQueryMetrics; -public class DefaultQueryMetricsFactory implements QueryMetricsFactory +public class DefaultGenericQueryMetricsFactory implements GenericQueryMetricsFactory { - private static final DefaultQueryMetricsFactory INSTANCE = new DefaultQueryMetricsFactory(new DefaultObjectMapper()); + private static final DefaultGenericQueryMetricsFactory INSTANCE = new DefaultGenericQueryMetricsFactory(new DefaultObjectMapper()); - public static DefaultQueryMetricsFactory instance() + public static DefaultGenericQueryMetricsFactory instance() { return INSTANCE; } @@ -41,7 +35,7 @@ public static DefaultQueryMetricsFactory instance() private final ObjectMapper jsonMapper; @Inject - public DefaultQueryMetricsFactory(ObjectMapper jsonMapper) + public DefaultGenericQueryMetricsFactory(ObjectMapper jsonMapper) { this.jsonMapper = jsonMapper; } @@ -54,21 +48,4 @@ public QueryMetrics> makeMetrics(Query query) return queryMetrics; } - @Override - public TopNQueryMetrics makeTopNQueryMetrics() - { - return new DefaultTopNQueryMetrics(jsonMapper); - } - - @Override - public GroupByQueryMetrics makeGroupByQueryMetrics() - { - return new DefaultGroupByQueryMetrics(jsonMapper); - } - - @Override - public TimeseriesQueryMetrics makeTimeseriesQueryMetrics() - { - return new DefaultTimeseriesQueryMetrics(jsonMapper); - } } diff --git a/processing/src/main/java/io/druid/query/DruidMetrics.java b/processing/src/main/java/io/druid/query/DruidMetrics.java index fb1188d2ae6f..6a57eec7513e 100644 --- a/processing/src/main/java/io/druid/query/DruidMetrics.java +++ b/processing/src/main/java/io/druid/query/DruidMetrics.java @@ -55,7 +55,7 @@ public static int findNumComplexAggs(List aggs) } public static QueryMetrics makeRequestMetrics( - final QueryMetricsFactory queryMetricsFactory, + final GenericQueryMetricsFactory queryMetricsFactory, final QueryToolChest> toolChest, final Query query, final String remoteAddr diff --git a/processing/src/main/java/io/druid/query/QueryMetricsFactory.java b/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java similarity index 64% rename from processing/src/main/java/io/druid/query/QueryMetricsFactory.java rename to processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java index ecaa74eb441f..dd4297606da0 100644 --- a/processing/src/main/java/io/druid/query/QueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/GenericQueryMetricsFactory.java @@ -19,25 +19,15 @@ package io.druid.query; -import io.druid.query.groupby.GroupByQueryMetrics; -import io.druid.query.timeseries.TimeseriesQueryMetrics; -import io.druid.query.topn.TopNQueryMetrics; - /** - * This factory is used for DI of custom {@link QueryMetrics} implementations. + * This factory is used for DI of custom {@link QueryMetrics} implementations for all query types, which don't (yet) + * need to emit custom dimensions and/or metrics, i. e. they are good with the generic {@link QueryMetrics} interface. */ -public interface QueryMetricsFactory +public interface GenericQueryMetricsFactory { /** - * Creates a {@link QueryMetrics} for query, which doesn't have predefined QueryMetrics subclass (i. e. not - * a topN, groupBy or timeseries query). This method must call {@link QueryMetrics#query(Query)} with the given query - * on the created QueryMetrics object before returning. + * Creates a {@link QueryMetrics} for query, which doesn't have predefined QueryMetrics subclass. This method must + * call {@link QueryMetrics#query(Query)} with the given query on the created QueryMetrics object before returning. */ QueryMetrics> makeMetrics(Query query); - - TopNQueryMetrics makeTopNQueryMetrics(); - - GroupByQueryMetrics makeGroupByQueryMetrics(); - - TimeseriesQueryMetrics makeTimeseriesQueryMetrics(); } diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index 38db803c4756..cdfb0e88919b 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -49,7 +49,7 @@ * 4. Control over the dimension and metric names. * * Here, "control" is provided to the operator of a Druid cluster, who would exercise that control through a - * site-specific extension adding a QueryMetricsFactory impl. + * site-specific extension adding XxxQueryMetricsFactory impl(s). * * * Types of methods in this interface @@ -57,8 +57,10 @@ * 1. Methods, pulling some dimensions from the query object. These methods are used to populate the metric before the * query is run. These methods accept a single `QueryType query` parameter. {@link #query(Query)} calls all methods * of this type, hence pulling all available information from the query object as dimensions. + * * 2. Methods for setting dimensions, which become known in the process of the query execution or after the query is * completed. + * * 3. Methods to register metrics to be emitted later in bulk via {@link #emit(ServiceEmitter)}. These methods * return this QueryMetrics object back for chaining. Names of these methods start with "report" prefix. * @@ -90,6 +92,39 @@ * unit (i. e. nanoseconds for time metrics, bytes for metrics of data size, etc.), allowing the implementations of * this method to round the value up to more coarse-grained units, if they don't need the maximum precision. * + * + * Making subinterfaces of QueryMetrics for emitting custom dimensions and/or metrics for specific query types + * ----------------------------------------------------------------------------------------------------------- + * If a query type (e. g. {@link io.druid.query.search.search.SearchQuery} (it's runners) needs to emit custom + * dimensions and/or metrics which doesn't make sense for all other query types, the following steps should be executed: + * 1. Create `interface SearchQueryMetrics extends QueryMetrics` (here and below "Search" is the query type) with + * additional methods (see "Adding new methods" section above). + * + * 2. Create `class DefaultSearchQueryMetrics implements SearchQueryMetrics`. This class should implement extra methods + * from SearchQueryMetrics interfaces with empty bodies, AND DELEGATE ALL OTHER METHODS TO A QueryMetrics OBJECT, + * provided as a sole parameter in DefaultSearchQueryMetrics constructor. + * + * 3. Create `interface SearchQueryMetricsFactory` with a single method + * `SearchQueryMetrics makeMetrics(SearchQuery query);`. + * + * 4. Create `class DefaultSearchQueryMetricsFactory implements SearchQueryMetricsFactory`, which accepts {@link + * GenericQueryMetricsFactory} as injected constructor parameter, and implements makeMetrics() as + * `return new DefaultSearchQueryMetrics(genericQueryMetricsFactory.makeMetrics(query));` + * + * 5. Inject and use SearchQueryMetricsFactory instead of {@link GenericQueryMetricsFactory} in {@link + * io.druid.query.search.SearchQueryQueryToolChest}. + * + * 6. Specify `binder.bind(SearchQueryMetricsFactory.class).to(DefaultSearchQueryMetricsFactory.class)` in + * QueryToolChestModule (if the query type belongs to the core druid-processing, e. g. SearchQuery) or in some + * extension-specific Guice module otherwise, if the query type is defined in an extension, e. g. ScanQuery. + * + * This complex procedure is needed to ensure custom {@link GenericQueryMetricsFactory} specified by users still works + * for the query type when query type decides to create their custom QueryMetrics subclass. + * + * For compatibility reasons {@link io.druid.query.topn.TopNQueryMetrics}, {@link + * io.druid.query.groupby.GroupByQueryMetrics}, and {@link io.druid.query.timeseries.TimeseriesQueryMetrics} are special + * and shouldn't been taken as direct examples for following the plan specified above. + * * @param */ public interface QueryMetrics> diff --git a/processing/src/main/java/io/druid/query/QueryToolChest.java b/processing/src/main/java/io/druid/query/QueryToolChest.java index 9cc6276cb5d0..daabdfcf18e3 100644 --- a/processing/src/main/java/io/druid/query/QueryToolChest.java +++ b/processing/src/main/java/io/druid/query/QueryToolChest.java @@ -50,11 +50,16 @@ public abstract class QueryToolChestPublic extensions (belonging to the Druid source tree) should use injected {@link QueryMetricsFactory} object - * for creating the QueryMetrics object, returned from this method. + *

QueryToolChests for query types in core (druid-processing) and public extensions (belonging to the Druid source + * tree) should use delegate this method to {@link GenericQueryMetricsFactory#makeMetrics(Query)} on an injected + * instance of {@link GenericQueryMetricsFactory}, as long as they don't need to emit custom dimensions and/or + * metrics. * - *

This method should ensure that {@link QueryMetrics#query(Query)} is called with the given query passed on the - * created QueryMetrics object before returning. + *

If some custom dimensions and/or metrics should be emitted for a query type, a plan described in + * "Making subinterfaces of QueryMetrics" section in {@link QueryMetrics}'s class-level Javadocs should be followed. + * + *

One way or another, this method should ensure that {@link QueryMetrics#query(Query)} is called with the given + * query passed on the created QueryMetrics object before returning. * * @param query The query that is being processed * diff --git a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java index 3f87050066d2..1c5b2eb00ea7 100644 --- a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java @@ -30,9 +30,9 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.query.BySegmentSkippingQueryRunner; import io.druid.query.CacheStrategy; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryMetrics; -import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -51,10 +51,10 @@ public class DataSourceQueryQueryToolChest { }; - private final QueryMetricsFactory queryMetricsFactory; + private final GenericQueryMetricsFactory queryMetricsFactory; @Inject - public DataSourceQueryQueryToolChest(QueryMetricsFactory queryMetricsFactory) + public DataSourceQueryQueryToolChest(GenericQueryMetricsFactory queryMetricsFactory) { this.queryMetricsFactory = queryMetricsFactory; } diff --git a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetricsFactory.java new file mode 100644 index 000000000000..a50ea8ccf60d --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetricsFactory.java @@ -0,0 +1,49 @@ +/* + * 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.query.groupby; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import io.druid.jackson.DefaultObjectMapper; + +public class DefaultGroupByQueryMetricsFactory implements GroupByQueryMetricsFactory +{ + private static final GroupByQueryMetricsFactory INSTANCE = + new DefaultGroupByQueryMetricsFactory(new DefaultObjectMapper()); + + public static GroupByQueryMetricsFactory instance() + { + return INSTANCE; + } + + private final ObjectMapper jsonMapper; + + @Inject + public DefaultGroupByQueryMetricsFactory(ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + + @Override + public GroupByQueryMetrics makeMetrics() + { + return new DefaultGroupByQueryMetrics(jsonMapper); + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetricsFactory.java new file mode 100644 index 000000000000..da76ddc79e2a --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetricsFactory.java @@ -0,0 +1,26 @@ +/* + * 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.query.groupby; + +public interface GroupByQueryMetricsFactory +{ + + GroupByQueryMetrics makeMetrics(); +} diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 4fdbcc7014a5..d6f1ede0ffb2 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -40,11 +40,9 @@ import io.druid.query.BaseQuery; import io.druid.query.CacheStrategy; import io.druid.query.DataSource; -import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; import io.druid.query.QueryDataSource; -import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.SubqueryQueryRunner; @@ -85,21 +83,21 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeGroupByResults( @Override public GroupByQueryMetrics makeMetrics(GroupByQuery query) { - GroupByQueryMetrics queryMetrics = queryMetricsFactory.makeGroupByQueryMetrics(); + GroupByQueryMetrics queryMetrics = queryMetricsFactory.makeMetrics(); queryMetrics.query(query); return queryMetrics; } diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 33ae778f2735..94c0384386cd 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -39,10 +39,10 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.nary.BinaryFn; import io.druid.query.CacheStrategy; -import io.druid.query.DefaultQueryMetricsFactory; +import io.druid.query.DefaultGenericQueryMetricsFactory; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryMetrics; -import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.ResultMergeQueryRunner; @@ -79,15 +79,15 @@ public SegmentAnalysis apply(SegmentAnalysis analysis) }; private final SegmentMetadataQueryConfig config; - private final QueryMetricsFactory queryMetricsFactory; + private final GenericQueryMetricsFactory queryMetricsFactory; public SegmentMetadataQueryQueryToolChest(SegmentMetadataQueryConfig config) { - this(config, DefaultQueryMetricsFactory.instance()); + this(config, DefaultGenericQueryMetricsFactory.instance()); } @Inject - public SegmentMetadataQueryQueryToolChest(SegmentMetadataQueryConfig config, QueryMetricsFactory queryMetricsFactory) + public SegmentMetadataQueryQueryToolChest(SegmentMetadataQueryConfig config, GenericQueryMetricsFactory queryMetricsFactory) { this.config = config; this.queryMetricsFactory = queryMetricsFactory; diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index 112ee1f40890..a011ef429c8a 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -36,11 +36,11 @@ import io.druid.java.util.common.guava.nary.BinaryFn; import io.druid.query.BaseQuery; import io.druid.query.CacheStrategy; -import io.druid.query.DefaultQueryMetricsFactory; +import io.druid.query.DefaultGenericQueryMetricsFactory; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; import io.druid.query.QueryMetrics; -import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -74,21 +74,21 @@ public class SearchQueryQueryToolChest extends QueryToolChest configSupplier; - private final QueryMetricsFactory queryMetricsFactory; + private final GenericQueryMetricsFactory queryMetricsFactory; public SelectQueryQueryToolChest( ObjectMapper jsonMapper, @@ -89,7 +89,7 @@ public SelectQueryQueryToolChest( Supplier configSupplier ) { - this(jsonMapper, intervalChunkingQueryRunnerDecorator, configSupplier, new DefaultQueryMetricsFactory(jsonMapper)); + this(jsonMapper, intervalChunkingQueryRunnerDecorator, configSupplier, new DefaultGenericQueryMetricsFactory(jsonMapper)); } @Inject @@ -97,7 +97,7 @@ public SelectQueryQueryToolChest( ObjectMapper jsonMapper, IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator, Supplier configSupplier, - QueryMetricsFactory queryMetricsFactory + GenericQueryMetricsFactory queryMetricsFactory ) { this.jsonMapper = jsonMapper; diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 1845b0627653..0eb433331b47 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -30,10 +30,10 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.query.BySegmentSkippingQueryRunner; import io.druid.query.CacheStrategy; -import io.druid.query.DefaultQueryMetricsFactory; +import io.druid.query.DefaultGenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryMetrics; -import io.druid.query.QueryMetricsFactory; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -59,15 +59,15 @@ public class TimeBoundaryQueryQueryToolChest { }; - private final QueryMetricsFactory queryMetricsFactory; + private final GenericQueryMetricsFactory queryMetricsFactory; public TimeBoundaryQueryQueryToolChest() { - this(DefaultQueryMetricsFactory.instance()); + this(DefaultGenericQueryMetricsFactory.instance()); } @Inject - public TimeBoundaryQueryQueryToolChest(QueryMetricsFactory queryMetricsFactory) + public TimeBoundaryQueryQueryToolChest(GenericQueryMetricsFactory queryMetricsFactory) { this.queryMetricsFactory = queryMetricsFactory; } diff --git a/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsFactory.java new file mode 100644 index 000000000000..c7750f243fb5 --- /dev/null +++ b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsFactory.java @@ -0,0 +1,49 @@ +/* + * 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.query.timeseries; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import io.druid.jackson.DefaultObjectMapper; + +public class DefaultTimeseriesQueryMetricsFactory implements TimeseriesQueryMetricsFactory +{ + private static final TimeseriesQueryMetricsFactory INSTANCE = + new DefaultTimeseriesQueryMetricsFactory(new DefaultObjectMapper()); + + public static TimeseriesQueryMetricsFactory instance() + { + return INSTANCE; + } + + private final ObjectMapper jsonMapper; + + @Inject + public DefaultTimeseriesQueryMetricsFactory(ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + + @Override + public TimeseriesQueryMetrics makeMetrics() + { + return new DefaultTimeseriesQueryMetrics(jsonMapper); + } +} diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetricsFactory.java new file mode 100644 index 000000000000..d84f1542994b --- /dev/null +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetricsFactory.java @@ -0,0 +1,26 @@ +/* + * 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.query.timeseries; + +public interface TimeseriesQueryMetricsFactory +{ + + TimeseriesQueryMetrics makeMetrics(); +} diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index e3aa97cae118..0105704daf6a 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -29,10 +29,8 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.nary.BinaryFn; import io.druid.query.CacheStrategy; -import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; -import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -64,17 +62,17 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest, Result, @Override public TimeseriesQueryMetrics makeMetrics(TimeseriesQuery query) { - TimeseriesQueryMetrics queryMetrics = queryMetricsFactory.makeTimeseriesQueryMetrics(); + TimeseriesQueryMetrics queryMetrics = queryMetricsFactory.makeMetrics(); queryMetrics.query(query); return queryMetrics; } diff --git a/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetricsFactory.java new file mode 100644 index 000000000000..614103da5f4b --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetricsFactory.java @@ -0,0 +1,48 @@ +/* + * 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.query.topn; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import io.druid.jackson.DefaultObjectMapper; + +public class DefaultTopNQueryMetricsFactory implements TopNQueryMetricsFactory +{ + private static final TopNQueryMetricsFactory INSTANCE = new DefaultTopNQueryMetricsFactory(new DefaultObjectMapper()); + + public static TopNQueryMetricsFactory instance() + { + return INSTANCE; + } + + private final ObjectMapper jsonMapper; + + @Inject + public DefaultTopNQueryMetricsFactory(ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + + @Override + public TopNQueryMetrics makeMetrics() + { + return new DefaultTopNQueryMetrics(jsonMapper); + } +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/topn/TopNQueryMetricsFactory.java new file mode 100644 index 000000000000..e0ba452e61ec --- /dev/null +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryMetricsFactory.java @@ -0,0 +1,26 @@ +/* + * 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.query.topn; + +public interface TopNQueryMetricsFactory +{ + + TopNQueryMetrics makeMetrics(); +} diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index e5dc67e914fc..7d224fefa564 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -34,10 +34,8 @@ import io.druid.query.BaseQuery; import io.druid.query.BySegmentResultValue; import io.druid.query.CacheStrategy; -import io.druid.query.DefaultQueryMetricsFactory; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; -import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; @@ -71,21 +69,21 @@ public class TopNQueryQueryToolChest extends QueryToolChest, Result, Result segments = toolChest .filterSegments( diff --git a/server/src/main/java/io/druid/guice/QueryToolChestModule.java b/server/src/main/java/io/druid/guice/QueryToolChestModule.java index f1a8ea2049f5..0b7d84fe0035 100644 --- a/server/src/main/java/io/druid/guice/QueryToolChestModule.java +++ b/server/src/main/java/io/druid/guice/QueryToolChestModule.java @@ -23,16 +23,18 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.multibindings.MapBinder; -import io.druid.query.DefaultQueryMetricsFactory; +import io.druid.query.DefaultGenericQueryMetricsFactory; import io.druid.query.MapQueryToolChestWarehouse; import io.druid.query.Query; -import io.druid.query.QueryMetricsFactory; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; import io.druid.query.datasourcemetadata.DataSourceMetadataQuery; import io.druid.query.datasourcemetadata.DataSourceQueryQueryToolChest; +import io.druid.query.groupby.DefaultGroupByQueryMetricsFactory; import io.druid.query.groupby.GroupByQuery; import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.GroupByQueryMetricsFactory; import io.druid.query.groupby.GroupByQueryQueryToolChest; import io.druid.query.metadata.SegmentMetadataQueryConfig; import io.druid.query.metadata.SegmentMetadataQueryQueryToolChest; @@ -45,10 +47,14 @@ import io.druid.query.select.SelectQueryQueryToolChest; import io.druid.query.timeboundary.TimeBoundaryQuery; import io.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest; +import io.druid.query.timeseries.DefaultTimeseriesQueryMetricsFactory; import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesQueryMetricsFactory; import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import io.druid.query.topn.DefaultTopNQueryMetricsFactory; import io.druid.query.topn.TopNQuery; import io.druid.query.topn.TopNQueryConfig; +import io.druid.query.topn.TopNQueryMetricsFactory; import io.druid.query.topn.TopNQueryQueryToolChest; import java.util.Map; @@ -80,7 +86,11 @@ public void configure(Binder binder) } binder.bind(QueryToolChestWarehouse.class).to(MapQueryToolChestWarehouse.class); - binder.bind(QueryMetricsFactory.class).to(DefaultQueryMetricsFactory.class); + + binder.bind(GenericQueryMetricsFactory.class).to(DefaultGenericQueryMetricsFactory.class); + binder.bind(TopNQueryMetricsFactory.class).to(DefaultTopNQueryMetricsFactory.class); + binder.bind(GroupByQueryMetricsFactory.class).to(DefaultGroupByQueryMetricsFactory.class); + binder.bind(TimeseriesQueryMetricsFactory.class).to(DefaultTimeseriesQueryMetricsFactory.class); JsonConfigProvider.bind(binder, "druid.query.groupBy", GroupByQueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.search", SearchQueryConfig.class); diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 5fe67551d122..20a54c9f376e 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -32,9 +32,9 @@ import io.druid.guice.annotations.Smile; import io.druid.guice.http.DruidHttpClientConfig; import io.druid.query.DruidMetrics; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryMetrics; -import io.druid.query.QueryMetricsFactory; import io.druid.query.QueryToolChestWarehouse; import io.druid.server.log.RequestLogger; import io.druid.server.metrics.QueryCountStatsProvider; @@ -105,7 +105,7 @@ private static void handleException(HttpServletResponse response, ObjectMapper o private final DruidHttpClientConfig httpClientConfig; private final ServiceEmitter emitter; private final RequestLogger requestLogger; - private final QueryMetricsFactory queryMetricsFactory; + private final GenericQueryMetricsFactory queryMetricsFactory; private HttpClient broadcastClient; @@ -119,7 +119,7 @@ public AsyncQueryForwardingServlet( DruidHttpClientConfig httpClientConfig, ServiceEmitter emitter, RequestLogger requestLogger, - QueryMetricsFactory queryMetricsFactory + GenericQueryMetricsFactory queryMetricsFactory ) { this.warehouse = warehouse; diff --git a/server/src/main/java/io/druid/server/BrokerQueryResource.java b/server/src/main/java/io/druid/server/BrokerQueryResource.java index 2a38f17e8684..d94966a902aa 100644 --- a/server/src/main/java/io/druid/server/BrokerQueryResource.java +++ b/server/src/main/java/io/druid/server/BrokerQueryResource.java @@ -29,7 +29,7 @@ import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.query.Query; -import io.druid.query.QueryMetricsFactory; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChestWarehouse; import io.druid.server.http.security.StateResourceFilter; @@ -68,7 +68,7 @@ public BrokerQueryResource( RequestLogger requestLogger, QueryManager queryManager, AuthConfig authConfig, - QueryMetricsFactory queryMetricsFactory, + GenericQueryMetricsFactory queryMetricsFactory, TimelineServerView brokerServerView ) { diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 4d1f2b0a7f87..2bd7d192cbb3 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -38,11 +38,11 @@ import io.druid.java.util.common.guava.Yielder; import io.druid.java.util.common.guava.Yielders; import io.druid.query.DruidMetrics; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryContextKeys; import io.druid.query.QueryInterruptedException; import io.druid.query.QueryMetrics; -import io.druid.query.QueryMetricsFactory; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; @@ -102,7 +102,7 @@ public class QueryResource implements QueryCountStatsProvider protected final RequestLogger requestLogger; protected final QueryManager queryManager; protected final AuthConfig authConfig; - private final QueryMetricsFactory queryMetricsFactory; + private final GenericQueryMetricsFactory queryMetricsFactory; private final AtomicLong successfulQueryCount = new AtomicLong(); private final AtomicLong failedQueryCount = new AtomicLong(); private final AtomicLong interruptedQueryCount = new AtomicLong(); @@ -118,7 +118,7 @@ public QueryResource( RequestLogger requestLogger, QueryManager queryManager, AuthConfig authConfig, - QueryMetricsFactory queryMetricsFactory + GenericQueryMetricsFactory queryMetricsFactory ) { this.warehouse = warehouse; diff --git a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java index bee48a1c8271..0ac353aba182 100644 --- a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java +++ b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java @@ -41,7 +41,7 @@ import io.druid.guice.http.DruidHttpClientConfig; import io.druid.initialization.Initialization; import io.druid.java.util.common.lifecycle.Lifecycle; -import io.druid.query.DefaultQueryMetricsFactory; +import io.druid.query.DefaultGenericQueryMetricsFactory; import io.druid.query.MapQueryToolChestWarehouse; import io.druid.query.Query; import io.druid.query.QueryToolChest; @@ -234,7 +234,7 @@ public void log(RequestLogLine requestLogLine) throws IOException // noop } }, - new DefaultQueryMetricsFactory(jsonMapper) + new DefaultGenericQueryMetricsFactory(jsonMapper) ) { @Override diff --git a/server/src/test/java/io/druid/server/QueryResourceTest.java b/server/src/test/java/io/druid/server/QueryResourceTest.java index 8d345863b24d..f8ca466b97ac 100644 --- a/server/src/test/java/io/druid/server/QueryResourceTest.java +++ b/server/src/test/java/io/druid/server/QueryResourceTest.java @@ -29,7 +29,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; -import io.druid.query.DefaultQueryMetricsFactory; +import io.druid.query.DefaultGenericQueryMetricsFactory; import io.druid.query.MapQueryToolChestWarehouse; import io.druid.query.Query; import io.druid.query.QueryRunner; @@ -141,7 +141,7 @@ public void setup() new NoopRequestLogger(), queryManager, new AuthConfig(), - new DefaultQueryMetricsFactory(jsonMapper) + new DefaultGenericQueryMetricsFactory(jsonMapper) ); } @@ -216,7 +216,7 @@ public Access isAuthorized( new NoopRequestLogger(), queryManager, new AuthConfig(true), - new DefaultQueryMetricsFactory(jsonMapper) + new DefaultGenericQueryMetricsFactory(jsonMapper) ); Response response = queryResource.doPost( @@ -287,7 +287,7 @@ public Access isAuthorized( new NoopRequestLogger(), queryManager, new AuthConfig(true), - new DefaultQueryMetricsFactory(jsonMapper) + new DefaultGenericQueryMetricsFactory(jsonMapper) ); final String queryString = "{\"queryType\":\"timeBoundary\", \"dataSource\":\"allow\"," @@ -384,7 +384,7 @@ public Access isAuthorized( new NoopRequestLogger(), queryManager, new AuthConfig(true), - new DefaultQueryMetricsFactory(jsonMapper) + new DefaultGenericQueryMetricsFactory(jsonMapper) ); final String queryString = "{\"queryType\":\"timeBoundary\", \"dataSource\":\"allow\"," From ef32460cbebb5a3ac409190734541a0296c2ed8f Mon Sep 17 00:00:00 2001 From: leventov Date: Wed, 22 Mar 2017 19:55:25 -0600 Subject: [PATCH 13/17] Typo --- processing/src/main/java/io/druid/query/QueryMetrics.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index cdfb0e88919b..b78df5db9964 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -123,7 +123,7 @@ * * For compatibility reasons {@link io.druid.query.topn.TopNQueryMetrics}, {@link * io.druid.query.groupby.GroupByQueryMetrics}, and {@link io.druid.query.timeseries.TimeseriesQueryMetrics} are special - * and shouldn't been taken as direct examples for following the plan specified above. + * and shouldn't be taken as direct examples for following the plan specified above. * * @param */ From 9cf3f2232fa37be1f9163d8cf5940d12d66762cb Mon Sep 17 00:00:00 2001 From: leventov Date: Thu, 23 Mar 2017 12:59:55 -0600 Subject: [PATCH 14/17] More elaborate Javadoc of QueryMetrics --- .../main/java/io/druid/query/QueryMetrics.java | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index b78df5db9964..b6e0d8a9e978 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -74,6 +74,14 @@ * dimension or metric is useful and not very expensive to process and store then emit, skip (see above Goals, 1.) * otherwise. * + *

If implementors of custom QueryMetrics don't want to fix builds on every Druid release (e. g. if they want to add + * a single dimension to emitted events and don't want to alter other dimensions and emitted metrics), they could + * inherit their custom QueryMetrics from {@link DefaultQueryMetrics} or query-specific default implementation class, + * such as {@link io.druid.query.topn.DefaultTopNQueryMetrics}. Those classes are guaranteed to stay around and + * implement new methods, added to the QueryMetrics interface (or a query-specific subinterface). However, there is no + * 100% guarantee of compatibility, because methods could not only be added to QueryMetrics, existing methods could also + * be changed or removed. + * *

QueryMetrics is designed for use from a single thread, implementations shouldn't care about thread-safety. * * @@ -121,9 +129,11 @@ * This complex procedure is needed to ensure custom {@link GenericQueryMetricsFactory} specified by users still works * for the query type when query type decides to create their custom QueryMetrics subclass. * - * For compatibility reasons {@link io.druid.query.topn.TopNQueryMetrics}, {@link - * io.druid.query.groupby.GroupByQueryMetrics}, and {@link io.druid.query.timeseries.TimeseriesQueryMetrics} are special - * and shouldn't be taken as direct examples for following the plan specified above. + * {@link io.druid.query.topn.TopNQueryMetrics}, {@link io.druid.query.groupby.GroupByQueryMetrics}, and {@link + * io.druid.query.timeseries.TimeseriesQueryMetrics} are implemented differently, because they are introduced at the + * same time as the whole QueryMetrics abstraction and their default implementations have to actually emit more + * dimensions than the default generic QueryMetrics. So those subinterfaces shouldn't be taken as direct examples for + * following the plan specified above. * * @param */ From 022839bba979d0935dc7987e06470cf290878ef1 Mon Sep 17 00:00:00 2001 From: leventov Date: Thu, 23 Mar 2017 16:54:50 -0600 Subject: [PATCH 15/17] Formatting --- .../io/druid/query/DefaultGenericQueryMetricsFactory.java | 5 +++-- .../datasourcemetadata/DataSourceMetadataQueryTest.java | 3 ++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java index 27acd1d15829..6d1b5fc467ae 100644 --- a/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java @@ -25,9 +25,10 @@ public class DefaultGenericQueryMetricsFactory implements GenericQueryMetricsFactory { - private static final DefaultGenericQueryMetricsFactory INSTANCE = new DefaultGenericQueryMetricsFactory(new DefaultObjectMapper()); + private static final GenericQueryMetricsFactory INSTANCE = + new DefaultGenericQueryMetricsFactory(new DefaultObjectMapper()); - public static DefaultGenericQueryMetricsFactory instance() + public static GenericQueryMetricsFactory instance() { return INSTANCE; } diff --git a/processing/src/test/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java b/processing/src/test/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java index c3226304c6d1..e05caa4336d2 100644 --- a/processing/src/test/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java @@ -31,6 +31,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.query.DefaultGenericQueryMetricsFactory; import io.druid.query.Druids; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; @@ -149,7 +150,7 @@ public void testMaxIngestedEventTime() throws Exception @Test public void testFilterSegments() { - DefaultGenericQueryMetricsFactory queryMetricsFactory = DefaultGenericQueryMetricsFactory.instance(); + GenericQueryMetricsFactory queryMetricsFactory = DefaultGenericQueryMetricsFactory.instance(); DataSourceQueryQueryToolChest toolChest = new DataSourceQueryQueryToolChest(queryMetricsFactory); List segments = toolChest .filterSegments( From 00121868b5ff485d2db0004a967d5cdd346ed31b Mon Sep 17 00:00:00 2001 From: leventov Date: Thu, 23 Mar 2017 17:18:51 -0600 Subject: [PATCH 16/17] Replace QueryMetric enum with lambdas --- .../query/IntervalChunkingQueryRunner.java | 2 +- .../query/MetricsEmittingQueryRunner.java | 15 +++--- .../main/java/io/druid/query/QueryMetric.java | 51 ------------------- .../java/io/druid/query/QueryMetrics.java | 1 + .../appenderator/SinkQuerySegmentWalker.java | 8 +-- .../server/coordination/ServerManager.java | 6 +-- 6 files changed, 17 insertions(+), 66 deletions(-) delete mode 100644 processing/src/main/java/io/druid/query/QueryMetric.java diff --git a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java index 961f80c61344..a42ce69fde0e 100644 --- a/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java @@ -107,7 +107,7 @@ public Sequence apply(Interval singleInterval) emitter, toolChest, baseRunner, - QueryMetric.INTERVAL_CHUNK_TIME, + QueryMetrics::reportIntervalChunkTime, queryMetrics -> queryMetrics.chunkInterval(singleInterval) ).withWaitMeasuredFromNow() ), diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index eb95707ebc1c..096fb39d45fc 100644 --- a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.function.Consumer; +import java.util.function.ObjLongConsumer; /** */ @@ -37,7 +38,7 @@ public class MetricsEmittingQueryRunner implements QueryRunner private final QueryToolChest> queryToolChest; private final QueryRunner queryRunner; private final long creationTimeNs; - private final QueryMetric metric; + private final ObjLongConsumer>> reportMetric; private final Consumer>> applyCustomDimensions; private MetricsEmittingQueryRunner( @@ -45,7 +46,7 @@ private MetricsEmittingQueryRunner( QueryToolChest> queryToolChest, QueryRunner queryRunner, long creationTimeNs, - QueryMetric metric, + ObjLongConsumer>> reportMetric, Consumer>> applyCustomDimensions ) { @@ -53,7 +54,7 @@ private MetricsEmittingQueryRunner( this.queryToolChest = queryToolChest; this.queryRunner = queryRunner; this.creationTimeNs = creationTimeNs; - this.metric = metric; + this.reportMetric = reportMetric; this.applyCustomDimensions = applyCustomDimensions; } @@ -61,11 +62,11 @@ public MetricsEmittingQueryRunner( ServiceEmitter emitter, QueryToolChest> queryToolChest, QueryRunner queryRunner, - QueryMetric metric, + ObjLongConsumer>> reportMetric, Consumer>> applyCustomDimensions ) { - this(emitter, queryToolChest, queryRunner, -1, metric, applyCustomDimensions); + this(emitter, queryToolChest, queryRunner, -1, reportMetric, applyCustomDimensions); } public MetricsEmittingQueryRunner withWaitMeasuredFromNow() @@ -75,7 +76,7 @@ public MetricsEmittingQueryRunner withWaitMeasuredFromNow() queryToolChest, queryRunner, System.nanoTime(), - metric, + reportMetric, applyCustomDimensions ); } @@ -118,7 +119,7 @@ public void after(boolean isDone, Throwable thrown) queryMetrics.status("short"); } long timeTakenNs = System.nanoTime() - startTimeNs; - metric.report(queryMetrics, timeTakenNs); + reportMetric.accept(queryMetrics, timeTakenNs); if (creationTimeNs > 0) { queryMetrics.reportWaitTime(startTimeNs - creationTimeNs); diff --git a/processing/src/main/java/io/druid/query/QueryMetric.java b/processing/src/main/java/io/druid/query/QueryMetric.java deleted file mode 100644 index 45fa43e0a7e2..000000000000 --- a/processing/src/main/java/io/druid/query/QueryMetric.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.query; - -/** - * Used in {@link MetricsEmittingQueryRunner} as a binder to a particular metric registering method of - * {@link QueryMetrics}. - */ -public enum QueryMetric -{ - SEGMENT_TIME { - @Override - public void report(QueryMetrics metrics, long timeNs) - { - metrics.reportSegmentTime(timeNs); - } - }, - SEGMENT_AND_CACHE_TIME { - @Override - public void report(QueryMetrics metrics, long timeNs) - { - metrics.reportSegmentAndCacheTime(timeNs); - } - }, - INTERVAL_CHUNK_TIME { - @Override - public void report(QueryMetrics metrics, long timeNs) - { - metrics.reportIntervalChunkTime(timeNs); - } - }; - - public abstract void report(QueryMetrics metrics, long value); -} diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index b6e0d8a9e978..3afc3a122cb8 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -139,6 +139,7 @@ */ public interface QueryMetrics> { + /** * Pulls all information from the query object into dimensions of future metrics. */ diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 8fe983e7fa58..37ef7489831f 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -38,7 +38,7 @@ import io.druid.query.MetricsEmittingQueryRunner; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; -import io.druid.query.QueryMetric; +import io.druid.query.QueryMetrics; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -281,7 +281,7 @@ private QueryRunner withPerSinkMetrics( ) { - // Note: SEGMENT_AND_CACHE_TIME and SEGMENT_TIME are effectively the same here. They don't split apart + // Note: reportSegmentAndCacheTime and reportSegmentTime are effectively the same here. They don't split apart // cache vs. non-cache due to the fact that Sinks may be partially cached and partially uncached. Making this // better would need to involve another accumulator like the cpuTimeAccumulator that we could share with the // sinkRunner. @@ -294,10 +294,10 @@ private QueryRunner withPerSinkMetrics( emitter, queryToolChest, sinkRunner, - QueryMetric.SEGMENT_TIME, + QueryMetrics::reportSegmentTime, queryMetrics -> queryMetrics.segment(sinkSegmentIdentifier) ), - QueryMetric.SEGMENT_AND_CACHE_TIME, + QueryMetrics::reportSegmentAndCacheTime, queryMetrics -> queryMetrics.segment(sinkSegmentIdentifier) ).withWaitMeasuredFromNow(), queryToolChest, diff --git a/server/src/main/java/io/druid/server/coordination/ServerManager.java b/server/src/main/java/io/druid/server/coordination/ServerManager.java index 5d570210a1fd..65d94aab0d16 100644 --- a/server/src/main/java/io/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/io/druid/server/coordination/ServerManager.java @@ -42,7 +42,7 @@ import io.druid.query.MetricsEmittingQueryRunner; import io.druid.query.NoopQueryRunner; import io.druid.query.Query; -import io.druid.query.QueryMetric; +import io.druid.query.QueryMetrics; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; @@ -432,14 +432,14 @@ private QueryRunner buildAndDecorateQueryRunner( emitter, toolChest, new ReferenceCountingSegmentQueryRunner(factory, adapter, segmentDescriptor), - QueryMetric.SEGMENT_TIME, + QueryMetrics::reportSegmentTime, queryMetrics -> queryMetrics.segment(segmentId) ), cachingExec, cacheConfig ) ), - QueryMetric.SEGMENT_AND_CACHE_TIME, + QueryMetrics::reportSegmentAndCacheTime, queryMetrics -> queryMetrics.segment(segmentId) ).withWaitMeasuredFromNow(), segmentSpec From aebb764a0dc9f8edb101d8a618afe7431496b1dd Mon Sep 17 00:00:00 2001 From: leventov Date: Thu, 23 Mar 2017 17:30:20 -0600 Subject: [PATCH 17/17] Add comments and VisibleForTesting annotations --- .../io/druid/query/DefaultGenericQueryMetricsFactory.java | 7 +++++++ .../query/groupby/DefaultGroupByQueryMetricsFactory.java | 7 +++++++ .../io/druid/query/groupby/GroupByQueryQueryToolChest.java | 2 ++ .../query/metadata/SegmentMetadataQueryQueryToolChest.java | 1 + .../io/druid/query/search/SearchQueryQueryToolChest.java | 2 ++ .../timeboundary/TimeBoundaryQueryQueryToolChest.java | 2 ++ .../timeseries/DefaultTimeseriesQueryMetricsFactory.java | 6 ++++++ .../query/timeseries/TimeseriesQueryQueryToolChest.java | 2 ++ .../druid/query/topn/DefaultTopNQueryMetricsFactory.java | 6 ++++++ .../java/io/druid/query/topn/TopNQueryQueryToolChest.java | 2 ++ 10 files changed, 37 insertions(+) diff --git a/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java index 6d1b5fc467ae..b71da6e4dbb0 100644 --- a/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/DefaultGenericQueryMetricsFactory.java @@ -20,6 +20,7 @@ package io.druid.query; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; import io.druid.jackson.DefaultObjectMapper; @@ -28,6 +29,12 @@ public class DefaultGenericQueryMetricsFactory implements GenericQueryMetricsFac private static final GenericQueryMetricsFactory INSTANCE = new DefaultGenericQueryMetricsFactory(new DefaultObjectMapper()); + /** + * Should be used only in tests, directly or indirectly (e. g. in {@link + * io.druid.query.search.SearchQueryQueryToolChest#SearchQueryQueryToolChest( + * io.druid.query.search.search.SearchQueryConfig, IntervalChunkingQueryRunnerDecorator)}). + */ + @VisibleForTesting public static GenericQueryMetricsFactory instance() { return INSTANCE; diff --git a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetricsFactory.java index a50ea8ccf60d..f70c9c6c9872 100644 --- a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetricsFactory.java @@ -20,6 +20,7 @@ package io.druid.query.groupby; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; import io.druid.jackson.DefaultObjectMapper; @@ -28,6 +29,12 @@ public class DefaultGroupByQueryMetricsFactory implements GroupByQueryMetricsFac private static final GroupByQueryMetricsFactory INSTANCE = new DefaultGroupByQueryMetricsFactory(new DefaultObjectMapper()); + /** + * Should be used only in tests, directly or indirectly (via {@link + * GroupByQueryQueryToolChest#GroupByQueryQueryToolChest(io.druid.query.groupby.strategy.GroupByStrategySelector, + * io.druid.query.IntervalChunkingQueryRunnerDecorator)}). + */ + @VisibleForTesting public static GroupByQueryMetricsFactory instance() { return INSTANCE; diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index d6f1ede0ffb2..a5966fd5dd42 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -20,6 +20,7 @@ package io.druid.query.groupby; import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Predicate; @@ -85,6 +86,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest