From 7ccb88b216d1ba6ea6f606296798b1e277df2d02 Mon Sep 17 00:00:00 2001 From: dwivedi Date: Tue, 18 Jul 2017 19:57:20 -0700 Subject: [PATCH 1/8] granularity method in QueryMetrics. PR to emit granularity dimension for timeseries, search, groupBy, select and topN queries. --- .../io/druid/query/DefaultQueryMetrics.java | 20 +++++++++++++++++++ .../java/io/druid/query/QueryMetrics.java | 7 +++++++ .../groupby/DefaultGroupByQueryMetrics.java | 1 + .../search/SearchQueryQueryToolChest.java | 2 +- .../select/SelectQueryQueryToolChest.java | 2 +- .../DefaultTimeseriesQueryMetrics.java | 1 + .../query/topn/DefaultTopNQueryMetrics.java | 1 + .../druid/query/DefaultQueryMetricsTest.java | 4 +++- .../DefaultGroupByQueryMetricsTest.java | 3 ++- .../DefaultTimeseriesQueryMetricsTest.java | 3 ++- .../topn/DefaultTopNQueryMetricsTest.java | 3 ++- 11 files changed, 41 insertions(+), 6 deletions(-) diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index 8cc363a41c04..7881a6e0fb09 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -26,6 +26,7 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.collections.bitmap.BitmapFactory; +import io.druid.java.util.common.granularity.Granularity; import io.druid.query.filter.Filter; import org.joda.time.Interval; @@ -291,4 +292,23 @@ public void emit(ServiceEmitter emitter) } metrics.clear(); } + + // sets "query granularity". + @Override + public QueryMetrics granularity(Granularity granularity) + { + try { + setDimension( + "granularity", + jsonMapper.writeValueAsString(granularity == null + ? ImmutableMap.of() + : granularity + ) + ); + return this; + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } } diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index 7eab88c5af1a..3c06b349d50c 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -21,6 +21,7 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.collections.bitmap.BitmapFactory; +import io.druid.java.util.common.granularity.Granularity; import io.druid.query.filter.Filter; import org.joda.time.Interval; @@ -286,4 +287,10 @@ public interface QueryMetrics> * Emits all metrics, registered since the last {@code emit()} call on this QueryMetrics object. */ void emit(ServiceEmitter emitter); + + // Sets "query granularity". + default QueryMetrics granularity(Granularity granularity) + { + return this; + } } diff --git a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java index 808269e949a1..065c490c7977 100644 --- a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java @@ -38,6 +38,7 @@ public void query(GroupByQuery query) numDimensions(query); numMetrics(query); numComplexMetrics(query); + granularity(query.getGranularity()); } @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 6e9aa50956a9..d105a78664db 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -129,7 +129,7 @@ protected BinaryFn, Result, Result< @Override public QueryMetrics> makeMetrics(SearchQuery query) { - return queryMetricsFactory.makeMetrics(query); + return queryMetricsFactory.makeMetrics(query).granularity(query.getGranularity()); } @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 da15120e8df4..543c98c507d6 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -141,7 +141,7 @@ protected BinaryFn, Result, Result< @Override public QueryMetrics> makeMetrics(SelectQuery query) { - return queryMetricsFactory.makeMetrics(query); + return queryMetricsFactory.makeMetrics(query).granularity(query.getGranularity()); } @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 index d8a015bd2a39..7f31955ebdbb 100644 --- a/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java @@ -37,6 +37,7 @@ public void query(TimeseriesQuery query) super.query(query); numMetrics(query); numComplexMetrics(query); + granularity(query.getGranularity()); } @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 index 0ad35e3c0731..6032daf768c7 100644 --- a/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java @@ -41,6 +41,7 @@ public void query(TopNQuery query) dimension(query); numMetrics(query); numComplexMetrics(query); + granularity(query.getGranularity()); } @Override diff --git a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java index d6b1876952bc..3b640c942b3e 100644 --- a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java @@ -66,10 +66,11 @@ public void testDefaultQueryMetricsQuery() .filters(new SelectorDimFilter("tags", "t3", null)) .build(); queryMetrics.query(query); + queryMetrics.granularity(query.getGranularity()); queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(12, actualEvent.size()); + Assert.assertEquals(13, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); @@ -85,6 +86,7 @@ public void testDefaultQueryMetricsQuery() Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); Assert.assertEquals("query/time", actualEvent.get("metric")); Assert.assertEquals(0L, actualEvent.get("value")); + Assert.assertEquals("{\"type\":\"all\"}", actualEvent.get("granularity")); } @Test 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 951c97d09d44..f780b129b9e0 100644 --- a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java @@ -93,7 +93,7 @@ public void testDefaultGroupByQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(15, actualEvent.size()); + Assert.assertEquals(16, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); @@ -110,6 +110,7 @@ public void testDefaultGroupByQueryMetricsQuery() Assert.assertEquals("1", actualEvent.get("numDimensions")); Assert.assertEquals("2", actualEvent.get("numMetrics")); Assert.assertEquals("0", actualEvent.get("numComplexMetrics")); + Assert.assertEquals("\"MONTH\"", actualEvent.get("granularity")); // Metric Assert.assertEquals("query/time", actualEvent.get("metric")); 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 d2422a9ddfc8..a6f18b38b5a4 100644 --- a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java @@ -67,7 +67,7 @@ public void testDefaultTimeseriesQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(14, actualEvent.size()); + Assert.assertEquals(15, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); @@ -85,6 +85,7 @@ public void testDefaultTimeseriesQueryMetricsQuery() // Timeseries-specific dimensions Assert.assertEquals("2", actualEvent.get("numMetrics")); Assert.assertEquals("0", actualEvent.get("numComplexMetrics")); + Assert.assertEquals("\"DAY\"", actualEvent.get("granularity")); // Metric Assert.assertEquals("query/time", actualEvent.get("metric")); 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 e7df7d272e4d..a6b289f32bc0 100644 --- a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java @@ -71,7 +71,7 @@ public void testDefaultTopNQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(16, actualEvent.size()); + Assert.assertEquals(17, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); @@ -91,6 +91,7 @@ public void testDefaultTopNQueryMetricsQuery() Assert.assertEquals("tags", actualEvent.get("dimension")); Assert.assertEquals("1", actualEvent.get("numMetrics")); Assert.assertEquals("0", actualEvent.get("numComplexMetrics")); + Assert.assertEquals("{\"type\":\"all\"}", actualEvent.get("granularity")); // Metric Assert.assertEquals("query/time", actualEvent.get("metric")); From 1cc6b690a90fed6cf0cc2c42fa07908ca08731cf Mon Sep 17 00:00:00 2001 From: dwivedi Date: Fri, 21 Jul 2017 14:46:01 -0700 Subject: [PATCH 2/8] QueryMetricsFactory classes for search and select queries. --- .../io/druid/query/DefaultQueryMetrics.java | 20 ---- .../java/io/druid/query/QueryMetrics.java | 7 -- .../groupby/DefaultGroupByQueryMetrics.java | 8 +- .../query/groupby/GroupByQueryMetrics.java | 5 + .../search/SearchQueryQueryToolChest.java | 18 ++-- .../search/DefaultSearchQueryMetrics.java | 44 ++++++++ .../DefaultSearchQueryMetricsFactory.java | 52 +++++++++ .../search/search/SearchQueryMetrics.java | 31 ++++++ .../search/SearchQueryMetricsFactory.java | 36 +++++++ .../select/DefaultSelectQueryMetrics.java | 44 ++++++++ .../DefaultSelectQueryMetricsFactory.java | 52 +++++++++ .../query/select/SelectQueryMetrics.java | 30 ++++++ .../select/SelectQueryMetricsFactory.java | 36 +++++++ .../select/SelectQueryQueryToolChest.java | 15 ++- .../DefaultTimeseriesQueryMetrics.java | 8 +- .../timeseries/TimeseriesQueryMetrics.java | 5 + .../query/topn/DefaultTopNQueryMetrics.java | 8 +- .../io/druid/query/topn/TopNQueryMetrics.java | 5 + .../druid/query/DefaultQueryMetricsTest.java | 5 +- .../DefaultGroupByQueryMetricsTest.java | 2 +- .../search/DefaultSearchQueryMetricsTest.java | 100 ++++++++++++++++++ .../select/DefaultSelectQueryMetricsTest.java | 92 ++++++++++++++++ .../DefaultTimeseriesQueryMetricsTest.java | 2 +- .../topn/DefaultTopNQueryMetricsTest.java | 2 +- .../io/druid/guice/QueryToolChestModule.java | 28 +++++ 25 files changed, 602 insertions(+), 53 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java create mode 100644 processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetricsFactory.java create mode 100644 processing/src/main/java/io/druid/query/search/search/SearchQueryMetrics.java create mode 100644 processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java create mode 100644 processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java create mode 100644 processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetricsFactory.java create mode 100644 processing/src/main/java/io/druid/query/select/SelectQueryMetrics.java create mode 100644 processing/src/main/java/io/druid/query/select/SelectQueryMetricsFactory.java create mode 100644 processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java create mode 100644 processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java diff --git a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java index 7881a6e0fb09..8cc363a41c04 100644 --- a/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/DefaultQueryMetrics.java @@ -26,7 +26,6 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.collections.bitmap.BitmapFactory; -import io.druid.java.util.common.granularity.Granularity; import io.druid.query.filter.Filter; import org.joda.time.Interval; @@ -292,23 +291,4 @@ public void emit(ServiceEmitter emitter) } metrics.clear(); } - - // sets "query granularity". - @Override - public QueryMetrics granularity(Granularity granularity) - { - try { - setDimension( - "granularity", - jsonMapper.writeValueAsString(granularity == null - ? ImmutableMap.of() - : granularity - ) - ); - return this; - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - } } diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index 3c06b349d50c..7eab88c5af1a 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -21,7 +21,6 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.collections.bitmap.BitmapFactory; -import io.druid.java.util.common.granularity.Granularity; import io.druid.query.filter.Filter; import org.joda.time.Interval; @@ -287,10 +286,4 @@ public interface QueryMetrics> * Emits all metrics, registered since the last {@code emit()} call on this QueryMetrics object. */ void emit(ServiceEmitter emitter); - - // Sets "query granularity". - default QueryMetrics granularity(Granularity granularity) - { - return this; - } } diff --git a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java index 065c490c7977..d723f9fb4f28 100644 --- a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java @@ -38,7 +38,7 @@ public void query(GroupByQuery query) numDimensions(query); numMetrics(query); numComplexMetrics(query); - granularity(query.getGranularity()); + granularity(query); } @Override @@ -59,4 +59,10 @@ public void numComplexMetrics(GroupByQuery query) int numComplexAggs = DruidMetrics.findNumComplexAggs(query.getAggregatorSpecs()); setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); } + + @Override + public void granularity(GroupByQuery query) + { + setDimension("granularity", query.getGranularity().toString()); + } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetrics.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetrics.java index 2d9ef261d193..985c9445b998 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryMetrics.java @@ -42,4 +42,9 @@ public interface GroupByQueryMetrics extends QueryMetrics * method. */ void numComplexMetrics(GroupByQuery query); + + /** + * Sets the granularity of {@link GroupByQuery#getGranularity()} of the given query as dimension. + */ + void granularity(GroupByQuery query); } 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 d105a78664db..49d537ce11c2 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -37,12 +37,9 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.nary.BinaryFn; import io.druid.query.CacheStrategy; -import io.druid.query.DefaultGenericQueryMetricsFactory; -import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; import io.druid.query.QueryContexts; -import io.druid.query.QueryMetrics; import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; @@ -52,9 +49,12 @@ import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.DimFilter; +import io.druid.query.search.search.DefaultSearchQueryMetricsFactory; import io.druid.query.search.search.SearchHit; import io.druid.query.search.search.SearchQuery; import io.druid.query.search.search.SearchQueryConfig; +import io.druid.query.search.search.SearchQueryMetrics; +import io.druid.query.search.search.SearchQueryMetricsFactory; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -76,7 +76,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest, Result, Result< } @Override - public QueryMetrics> makeMetrics(SearchQuery query) + public SearchQueryMetrics makeMetrics(SearchQuery query) { - return queryMetricsFactory.makeMetrics(query).granularity(query.getGranularity()); + SearchQueryMetrics metrics = queryMetricsFactory.makeMetrics(); + metrics.query(query); + return metrics; } @Override diff --git a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java new file mode 100644 index 000000000000..f441c80cec46 --- /dev/null +++ b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java @@ -0,0 +1,44 @@ +/* + * 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.search.search; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.query.DefaultQueryMetrics; + +public class DefaultSearchQueryMetrics extends DefaultQueryMetrics implements SearchQueryMetrics +{ + public DefaultSearchQueryMetrics(ObjectMapper jsonMapper) + { + super(jsonMapper); + } + + @Override + public void query (SearchQuery query) + { + super.query(query); + granularity(query); + } + + @Override + public void granularity(SearchQuery query) + { + setDimension("granularity", query.getGranularity().toString()); + } +} diff --git a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetricsFactory.java new file mode 100644 index 000000000000..d8f6cc0318f2 --- /dev/null +++ b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetricsFactory.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.search.search; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.inject.Inject; +import io.druid.guice.annotations.Json; +import io.druid.jackson.DefaultObjectMapper; + +public class DefaultSearchQueryMetricsFactory implements SearchQueryMetricsFactory +{ + private final ObjectMapper jsonMapper; + private static final SearchQueryMetricsFactory INSTANCE = + new DefaultSearchQueryMetricsFactory(new DefaultObjectMapper()); + + //Used only for testng. + @VisibleForTesting + public static SearchQueryMetricsFactory instance() + { + return INSTANCE; + } + + @Inject + public DefaultSearchQueryMetricsFactory(@Json ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + + @Override + public SearchQueryMetrics makeMetrics() + { + return new DefaultSearchQueryMetrics(jsonMapper); + } +} diff --git a/processing/src/main/java/io/druid/query/search/search/SearchQueryMetrics.java b/processing/src/main/java/io/druid/query/search/search/SearchQueryMetrics.java new file mode 100644 index 000000000000..60ffa6dbf51d --- /dev/null +++ b/processing/src/main/java/io/druid/query/search/search/SearchQueryMetrics.java @@ -0,0 +1,31 @@ +/* + * 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.search.search; + +import io.druid.query.QueryMetrics; + +public interface SearchQueryMetrics extends QueryMetrics +{ + /** + * Sets the granularity of {@link SearchQuery#getGranularity()} of the given query as dimension. + */ + void granularity(SearchQuery query); + +} diff --git a/processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java new file mode 100644 index 000000000000..dc9eaba3ce50 --- /dev/null +++ b/processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java @@ -0,0 +1,36 @@ +/* + * 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.search.search; + +/** + * Implementations could be injected using + * + * PolyBind + * .optionBinder(binder, Key.get(SearchQueryMetricsFactory.class)) + * .addBinding("myCustomSearchQueryMetricsFactory") + * .to(MyCustomSearchQueryMetricsFactory.class); + * + * And then setting property: + * druid.query.search.queryMetricsFactory=myCustomSearchQueryMetricsFactory + */ +public interface SearchQueryMetricsFactory +{ + SearchQueryMetrics makeMetrics(); +} diff --git a/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java new file mode 100644 index 000000000000..3dd022be9861 --- /dev/null +++ b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java @@ -0,0 +1,44 @@ +/* + * 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.select; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.query.DefaultQueryMetrics; + +public class DefaultSelectQueryMetrics extends DefaultQueryMetrics implements SelectQueryMetrics +{ + public DefaultSelectQueryMetrics(ObjectMapper jsonMapper) + { + super(jsonMapper); + } + + @Override + public void query (SelectQuery query) + { + super.query(query); + granularity(query); + } + + @Override + public void granularity(SelectQuery query) + { + setDimension("granularity", query.getGranularity().toString()); + } +} diff --git a/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetricsFactory.java new file mode 100644 index 000000000000..7a1d438c1606 --- /dev/null +++ b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetricsFactory.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.select; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.inject.Inject; +import io.druid.guice.annotations.Json; +import io.druid.jackson.DefaultObjectMapper; + +public class DefaultSelectQueryMetricsFactory implements SelectQueryMetricsFactory +{ + private final ObjectMapper jsonMapper; + private static final SelectQueryMetricsFactory INSTANCE = + new DefaultSelectQueryMetricsFactory(new DefaultObjectMapper()); + + //Used only for testng. + @VisibleForTesting + public static SelectQueryMetricsFactory instance() + { + return INSTANCE; + } + + @Inject + public DefaultSelectQueryMetricsFactory(@Json ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + + @Override + public SelectQueryMetrics makeMetrics() + { + return new DefaultSelectQueryMetrics(jsonMapper); + } +} diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryMetrics.java b/processing/src/main/java/io/druid/query/select/SelectQueryMetrics.java new file mode 100644 index 000000000000..f4757979c6a9 --- /dev/null +++ b/processing/src/main/java/io/druid/query/select/SelectQueryMetrics.java @@ -0,0 +1,30 @@ +/* + * 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.select; + +import io.druid.query.QueryMetrics; + +public interface SelectQueryMetrics extends QueryMetrics +{ + /** + * Sets the granularity of {@link SelectQuery#getGranularity()} of the given query as dimension. + */ + void granularity(SelectQuery query); +} diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/select/SelectQueryMetricsFactory.java new file mode 100644 index 000000000000..54f801c485b1 --- /dev/null +++ b/processing/src/main/java/io/druid/query/select/SelectQueryMetricsFactory.java @@ -0,0 +1,36 @@ +/* + * 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.select; + +/** + * Implementations could be injected using + * + * PolyBind + * .optionBinder(binder, Key.get(SelectQueryMetricsFactory.class)) + * .addBinding("myCustomSelectQueryMetricsFactory") + * .to(MyCustomSelectQueryMetricsFactory.class); + * + * And then setting property: + * druid.query.select.queryMetricsFactory=myCustomSelectQueryMetricsFactory + */ +public interface SelectQueryMetricsFactory +{ + SelectQueryMetrics makeMetrics(); +} 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 543c98c507d6..e997bb67427f 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -38,11 +38,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.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.QueryPlus; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; @@ -83,7 +80,7 @@ public class SelectQueryQueryToolChest extends QueryToolChest configSupplier; - private final GenericQueryMetricsFactory queryMetricsFactory; + private final SelectQueryMetricsFactory queryMetricsFactory; public SelectQueryQueryToolChest( ObjectMapper jsonMapper, @@ -91,7 +88,7 @@ public SelectQueryQueryToolChest( Supplier configSupplier ) { - this(jsonMapper, intervalChunkingQueryRunnerDecorator, configSupplier, new DefaultGenericQueryMetricsFactory(jsonMapper)); + this(jsonMapper, intervalChunkingQueryRunnerDecorator, configSupplier, new DefaultSelectQueryMetricsFactory(jsonMapper)); } @Inject @@ -99,7 +96,7 @@ public SelectQueryQueryToolChest( ObjectMapper jsonMapper, IntervalChunkingQueryRunnerDecorator intervalChunkingQueryRunnerDecorator, Supplier configSupplier, - GenericQueryMetricsFactory queryMetricsFactory + SelectQueryMetricsFactory queryMetricsFactory ) { this.jsonMapper = jsonMapper; @@ -139,9 +136,11 @@ protected BinaryFn, Result, Result< } @Override - public QueryMetrics> makeMetrics(SelectQuery query) + public SelectQueryMetrics makeMetrics(SelectQuery query) { - return queryMetricsFactory.makeMetrics(query).granularity(query.getGranularity()); + SelectQueryMetrics queryMetrics = queryMetricsFactory.makeMetrics(); + queryMetrics.query(query); + return queryMetrics; } @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 index 7f31955ebdbb..038378ffb8e3 100644 --- a/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java @@ -37,7 +37,7 @@ public void query(TimeseriesQuery query) super.query(query); numMetrics(query); numComplexMetrics(query); - granularity(query.getGranularity()); + granularity(query); } @Override @@ -52,4 +52,10 @@ public void numComplexMetrics(TimeseriesQuery query) int numComplexAggs = DruidMetrics.findNumComplexAggs(query.getAggregatorSpecs()); setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); } + + @Override + public void granularity(TimeseriesQuery query) + { + setDimension("granularity", query.getGranularity().toString()); + } } diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetrics.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetrics.java index db7553525dc6..8b2ce6771b28 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryMetrics.java @@ -37,4 +37,9 @@ public interface TimeseriesQueryMetrics extends QueryMetrics * method. */ void numComplexMetrics(TimeseriesQuery query); + + /** + * Sets the granularity of {@link TimeseriesQuery#getGranularity()} of the given query as dimension. + */ + void granularity(TimeseriesQuery query); } diff --git a/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java index 6032daf768c7..fd899af33ba9 100644 --- a/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java @@ -41,7 +41,7 @@ public void query(TopNQuery query) dimension(query); numMetrics(query); numComplexMetrics(query); - granularity(query.getGranularity()); + granularity(query); } @Override @@ -69,6 +69,12 @@ public void numComplexMetrics(TopNQuery query) setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); } + @Override + public void granularity(TopNQuery query) + { + setDimension("granularity", query.getGranularity().toString()); + } + @Override public void dimensionCardinality(int cardinality) { diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryMetrics.java b/processing/src/main/java/io/druid/query/topn/TopNQueryMetrics.java index 6a70b8dcf44f..25eb0fbd2252 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryMetrics.java @@ -50,6 +50,11 @@ public interface TopNQueryMetrics extends QueryMetrics */ void numComplexMetrics(TopNQuery query); + /** + * Sets the granularity of {@link TopNQuery#getGranularity()} of the given query as dimension. + */ + void granularity(TopNQuery query); + void dimensionCardinality(int cardinality); void algorithm(TopNAlgorithm algorithm); diff --git a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java index 3b640c942b3e..aee5c6e8fb4c 100644 --- a/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/DefaultQueryMetricsTest.java @@ -66,11 +66,9 @@ public void testDefaultQueryMetricsQuery() .filters(new SelectorDimFilter("tags", "t3", null)) .build(); queryMetrics.query(query); - queryMetrics.granularity(query.getGranularity()); - queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(13, actualEvent.size()); + Assert.assertEquals(12, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); @@ -86,7 +84,6 @@ public void testDefaultQueryMetricsQuery() Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); Assert.assertEquals("query/time", actualEvent.get("metric")); Assert.assertEquals(0L, actualEvent.get("value")); - Assert.assertEquals("{\"type\":\"all\"}", actualEvent.get("granularity")); } @Test 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 f780b129b9e0..d8aac04acd06 100644 --- a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java @@ -110,7 +110,7 @@ public void testDefaultGroupByQueryMetricsQuery() Assert.assertEquals("1", actualEvent.get("numDimensions")); Assert.assertEquals("2", actualEvent.get("numMetrics")); Assert.assertEquals("0", actualEvent.get("numComplexMetrics")); - Assert.assertEquals("\"MONTH\"", actualEvent.get("granularity")); + Assert.assertEquals("{type=period, period=P1M, timeZone=UTC, origin=null}", actualEvent.get("granularity")); // Metric Assert.assertEquals("query/time", actualEvent.get("metric")); diff --git a/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java b/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java new file mode 100644 index 000000000000..538783de414f --- /dev/null +++ b/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java @@ -0,0 +1,100 @@ +/* + * 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.search; + +import com.google.common.collect.ImmutableSet; +import com.metamx.emitter.service.ServiceEmitter; +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 io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.ListFilteredDimensionSpec; +import io.druid.query.search.search.DefaultSearchQueryMetrics; +import io.druid.query.search.search.SearchQuery; +import io.druid.segment.TestHelper; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class DefaultSearchQueryMetricsTest +{ + + /** + * Tests that passed a query {@link DefaultSearchQueryMetrics} produces events with a certain set of dimensions. + */ + @Test + public void testDefaultSearchQueryMetricsQuery() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultSearchQueryMetrics queryMetrics = new DefaultSearchQueryMetrics(TestHelper.getJsonMapper()); + SearchQuery query = Druids + .newSearchQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .dimensions(new ListFilteredDimensionSpec( + new DefaultDimensionSpec("tags", "tags"), + ImmutableSet.of("t3"), + null + )) + .build(); + queryMetrics.query(query); + + queryMetrics.reportQueryTime(0).emit(serviceEmitter); + Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals(13, 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("{type=period, period=P1D, timeZone=UTC, origin=null}", actualEvent.get("granularity")); + + // Metric + Assert.assertEquals("query/time", actualEvent.get("metric")); + Assert.assertEquals(0L, actualEvent.get("value")); + } + + @Test + public void testDefaultSearchQueryMetricsMetricNamesAndUnits() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultSearchQueryMetrics queryMetrics = new DefaultSearchQueryMetrics(TestHelper.getJsonMapper()); + DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); + } +} diff --git a/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java b/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java new file mode 100644 index 000000000000..688ed10817fe --- /dev/null +++ b/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java @@ -0,0 +1,92 @@ +/* + * 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.select; + +import com.metamx.emitter.service.ServiceEmitter; +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 io.druid.segment.TestHelper; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class DefaultSelectQueryMetricsTest +{ + + /** + * Tests that passed a query {@link DefaultSelectQueryMetrics} produces events with a certain set of dimensions. + */ + @Test + public void testDefaultSelectQueryMetricsQuery() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultSelectQueryMetrics queryMetrics = new DefaultSelectQueryMetrics(TestHelper.getJsonMapper()); + SelectQuery query = Druids + .newSelectQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .descending(true) + .pagingSpec(PagingSpec.newSpec(1)) + .build(); + queryMetrics.query(query); + + queryMetrics.reportQueryTime(0).emit(serviceEmitter); + Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); + Assert.assertEquals(13, 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("{type=period, period=P1D, timeZone=UTC, origin=null}", actualEvent.get("granularity")); + + // Metric + Assert.assertEquals("query/time", actualEvent.get("metric")); + Assert.assertEquals(0L, actualEvent.get("value")); + } + + @Test + public void testDefaultSelectQueryMetricsMetricNamesAndUnits() + { + CachingEmitter cachingEmitter = new CachingEmitter(); + ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); + DefaultSelectQueryMetrics queryMetrics = new DefaultSelectQueryMetrics(TestHelper.getJsonMapper()); + 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 index a6f18b38b5a4..bd5e52468b0a 100644 --- a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java @@ -85,7 +85,7 @@ public void testDefaultTimeseriesQueryMetricsQuery() // Timeseries-specific dimensions Assert.assertEquals("2", actualEvent.get("numMetrics")); Assert.assertEquals("0", actualEvent.get("numComplexMetrics")); - Assert.assertEquals("\"DAY\"", actualEvent.get("granularity")); + Assert.assertEquals("{type=period, period=P1D, timeZone=UTC, origin=null}", actualEvent.get("granularity")); // Metric Assert.assertEquals("query/time", actualEvent.get("metric")); 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 a6b289f32bc0..548c955c3a41 100644 --- a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java @@ -91,7 +91,7 @@ public void testDefaultTopNQueryMetricsQuery() Assert.assertEquals("tags", actualEvent.get("dimension")); Assert.assertEquals("1", actualEvent.get("numMetrics")); Assert.assertEquals("0", actualEvent.get("numComplexMetrics")); - Assert.assertEquals("{\"type\":\"all\"}", actualEvent.get("granularity")); + Assert.assertEquals("AllGranularity", actualEvent.get("granularity")); // Metric Assert.assertEquals("query/time", actualEvent.get("metric")); diff --git a/server/src/main/java/io/druid/guice/QueryToolChestModule.java b/server/src/main/java/io/druid/guice/QueryToolChestModule.java index 3e4da18bb4d6..2640bbec754a 100644 --- a/server/src/main/java/io/druid/guice/QueryToolChestModule.java +++ b/server/src/main/java/io/druid/guice/QueryToolChestModule.java @@ -41,10 +41,14 @@ import io.druid.query.metadata.SegmentMetadataQueryQueryToolChest; import io.druid.query.metadata.metadata.SegmentMetadataQuery; import io.druid.query.search.SearchQueryQueryToolChest; +import io.druid.query.search.search.DefaultSearchQueryMetricsFactory; import io.druid.query.search.search.SearchQuery; import io.druid.query.search.search.SearchQueryConfig; +import io.druid.query.search.search.SearchQueryMetricsFactory; +import io.druid.query.select.DefaultSelectQueryMetricsFactory; import io.druid.query.select.SelectQuery; import io.druid.query.select.SelectQueryConfig; +import io.druid.query.select.SelectQueryMetricsFactory; import io.druid.query.select.SelectQueryQueryToolChest; import io.druid.query.timeboundary.TimeBoundaryQuery; import io.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest; @@ -68,6 +72,8 @@ public class QueryToolChestModule implements Module public static final String GROUPBY_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.groupBy.queryMetricsFactory"; public static final String TIMESERIES_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.timeseries.queryMetricsFactory"; public static final String TOPN_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.topN.queryMetricsFactory"; + public static final String SELECT_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.select.queryMetricsFactory"; + public static final String SEARCH_QUERY_METRICS_FACTORY_PROPERTY = "druid.query.search.queryMetricsFactory"; public final Map, Class> mappings = ImmutableMap., Class>builder() @@ -142,5 +148,27 @@ public void configure(Binder binder) .optionBinder(binder, Key.get(TopNQueryMetricsFactory.class)) .addBinding("default") .to(DefaultTopNQueryMetricsFactory.class); + + PolyBind.createChoice( + binder, + SELECT_QUERY_METRICS_FACTORY_PROPERTY, + Key.get(SelectQueryMetricsFactory.class), + Key.get(DefaultSelectQueryMetricsFactory.class) + ); + PolyBind + .optionBinder(binder, Key.get(SelectQueryMetricsFactory.class)) + .addBinding("default") + .to(DefaultSelectQueryMetricsFactory.class); + + PolyBind.createChoice( + binder, + SEARCH_QUERY_METRICS_FACTORY_PROPERTY, + Key.get(SearchQueryMetricsFactory.class), + Key.get(DefaultSearchQueryMetricsFactory.class) + ); + PolyBind + .optionBinder(binder, Key.get(SearchQueryMetricsFactory.class)) + .addBinding("default") + .to(DefaultSearchQueryMetricsFactory.class); } } From daf5763a929c6b48c007daf837419c525175f8e5 Mon Sep 17 00:00:00 2001 From: dwivedi Date: Tue, 8 Aug 2017 12:13:19 -0700 Subject: [PATCH 3/8] Empty implementation for Granularity() method. --- .../io/druid/query/groupby/DefaultGroupByQueryMetrics.java | 1 - .../druid/query/search/search/DefaultSearchQueryMetrics.java | 1 - .../io/druid/query/select/DefaultSelectQueryMetrics.java | 1 - .../query/timeseries/DefaultTimeseriesQueryMetrics.java | 1 - .../java/io/druid/query/topn/DefaultTopNQueryMetrics.java | 1 - .../druid/query/groupby/DefaultGroupByQueryMetricsTest.java | 3 +-- .../io/druid/query/search/DefaultSearchQueryMetricsTest.java | 5 +---- .../io/druid/query/select/DefaultSelectQueryMetricsTest.java | 5 +---- .../query/timeseries/DefaultTimeseriesQueryMetricsTest.java | 3 +-- .../io/druid/query/topn/DefaultTopNQueryMetricsTest.java | 3 +-- 10 files changed, 5 insertions(+), 19 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java index d723f9fb4f28..09af924ca55c 100644 --- a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java @@ -63,6 +63,5 @@ public void numComplexMetrics(GroupByQuery query) @Override public void granularity(GroupByQuery query) { - setDimension("granularity", query.getGranularity().toString()); } } diff --git a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java index f441c80cec46..0c388e43f525 100644 --- a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java @@ -39,6 +39,5 @@ public void query (SearchQuery query) @Override public void granularity(SearchQuery query) { - setDimension("granularity", query.getGranularity().toString()); } } diff --git a/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java index 3dd022be9861..2cbdd71a0a00 100644 --- a/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java @@ -39,6 +39,5 @@ public void query (SelectQuery query) @Override public void granularity(SelectQuery query) { - setDimension("granularity", query.getGranularity().toString()); } } diff --git a/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java index 038378ffb8e3..5d02ee09216e 100644 --- a/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java @@ -56,6 +56,5 @@ public void numComplexMetrics(TimeseriesQuery query) @Override public void granularity(TimeseriesQuery query) { - setDimension("granularity", query.getGranularity().toString()); } } diff --git a/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java index fd899af33ba9..91ffd8360d85 100644 --- a/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java @@ -72,7 +72,6 @@ public void numComplexMetrics(TopNQuery query) @Override public void granularity(TopNQuery query) { - setDimension("granularity", query.getGranularity().toString()); } @Override 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 d8aac04acd06..951c97d09d44 100644 --- a/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/groupby/DefaultGroupByQueryMetricsTest.java @@ -93,7 +93,7 @@ public void testDefaultGroupByQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(16, actualEvent.size()); + Assert.assertEquals(15, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); @@ -110,7 +110,6 @@ public void testDefaultGroupByQueryMetricsQuery() Assert.assertEquals("1", actualEvent.get("numDimensions")); Assert.assertEquals("2", actualEvent.get("numMetrics")); Assert.assertEquals("0", actualEvent.get("numComplexMetrics")); - Assert.assertEquals("{type=period, period=P1M, timeZone=UTC, origin=null}", actualEvent.get("granularity")); // Metric Assert.assertEquals("query/time", actualEvent.get("metric")); diff --git a/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java b/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java index 538783de414f..3982d2402f4f 100644 --- a/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java @@ -66,7 +66,7 @@ public void testDefaultSearchQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(13, actualEvent.size()); + Assert.assertEquals(12, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); @@ -81,9 +81,6 @@ public void testDefaultSearchQueryMetricsQuery() Assert.assertEquals(expectedIntervals.get(0).toDuration().toString(), actualEvent.get("duration")); Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); - // Timeseries-specific dimensions - Assert.assertEquals("{type=period, period=P1D, timeZone=UTC, origin=null}", actualEvent.get("granularity")); - // Metric Assert.assertEquals("query/time", actualEvent.get("metric")); Assert.assertEquals(0L, actualEvent.get("value")); diff --git a/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java b/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java index 688ed10817fe..bd2ec7438c6f 100644 --- a/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java @@ -58,7 +58,7 @@ public void testDefaultSelectQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(13, actualEvent.size()); + Assert.assertEquals(12, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); @@ -73,9 +73,6 @@ public void testDefaultSelectQueryMetricsQuery() Assert.assertEquals(expectedIntervals.get(0).toDuration().toString(), actualEvent.get("duration")); Assert.assertEquals("", actualEvent.get(DruidMetrics.ID)); - // Timeseries-specific dimensions - Assert.assertEquals("{type=period, period=P1D, timeZone=UTC, origin=null}", actualEvent.get("granularity")); - // Metric Assert.assertEquals("query/time", actualEvent.get("metric")); Assert.assertEquals(0L, actualEvent.get("value")); 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 bd5e52468b0a..d2422a9ddfc8 100644 --- a/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetricsTest.java @@ -67,7 +67,7 @@ public void testDefaultTimeseriesQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(15, actualEvent.size()); + Assert.assertEquals(14, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); @@ -85,7 +85,6 @@ public void testDefaultTimeseriesQueryMetricsQuery() // Timeseries-specific dimensions Assert.assertEquals("2", actualEvent.get("numMetrics")); Assert.assertEquals("0", actualEvent.get("numComplexMetrics")); - Assert.assertEquals("{type=period, period=P1D, timeZone=UTC, origin=null}", actualEvent.get("granularity")); // Metric Assert.assertEquals("query/time", actualEvent.get("metric")); 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 548c955c3a41..e7df7d272e4d 100644 --- a/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/topn/DefaultTopNQueryMetricsTest.java @@ -71,7 +71,7 @@ public void testDefaultTopNQueryMetricsQuery() queryMetrics.reportQueryTime(0).emit(serviceEmitter); Map actualEvent = cachingEmitter.getLastEmittedEvent().toMap(); - Assert.assertEquals(17, actualEvent.size()); + Assert.assertEquals(16, actualEvent.size()); Assert.assertTrue(actualEvent.containsKey("feed")); Assert.assertTrue(actualEvent.containsKey("timestamp")); Assert.assertEquals("", actualEvent.get("host")); @@ -91,7 +91,6 @@ public void testDefaultTopNQueryMetricsQuery() Assert.assertEquals("tags", actualEvent.get("dimension")); Assert.assertEquals("1", actualEvent.get("numMetrics")); Assert.assertEquals("0", actualEvent.get("numComplexMetrics")); - Assert.assertEquals("AllGranularity", actualEvent.get("granularity")); // Metric Assert.assertEquals("query/time", actualEvent.get("metric")); From f4d46ad0bc7f1023c85d6c0b874f955a2fe9305a Mon Sep 17 00:00:00 2001 From: dwivedi Date: Tue, 15 Aug 2017 12:22:19 -0700 Subject: [PATCH 4/8] Review comment changes. --- .../java/io/druid/query/QueryMetrics.java | 38 ++-- .../groupby/DefaultGroupByQueryMetrics.java | 1 + .../search/SearchQueryQueryToolChest.java | 2 +- .../search/DefaultSearchQueryMetrics.java | 209 ++++++++++++++++- .../DefaultSearchQueryMetricsFactory.java | 26 +-- .../search/SearchQueryMetricsFactory.java | 12 +- .../select/DefaultSelectQueryMetrics.java | 210 +++++++++++++++++- .../DefaultSelectQueryMetricsFactory.java | 27 ++- .../select/SelectQueryMetricsFactory.java | 2 +- .../select/SelectQueryQueryToolChest.java | 4 +- .../DefaultTimeseriesQueryMetrics.java | 1 + .../query/topn/DefaultTopNQueryMetrics.java | 1 + .../search/DefaultSearchQueryMetricsTest.java | 16 +- .../select/DefaultSelectQueryMetricsTest.java | 15 +- 14 files changed, 489 insertions(+), 75 deletions(-) diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index 7eab88c5af1a..b6dff99b2b8a 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -110,29 +110,31 @@ * * 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). + * If a query type (e. g. {@link io.druid.query.metadata.metadata.SegmentMetadataQuery} (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: * - * 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. + * 1. Create `interface SegmentMetadataQueryMetrics extends QueryMetrics` (here and below "SegmentMetadata" is the + * query type) with additional methods (see "Adding new methods" section above). * - * 3. Create `interface SearchQueryMetricsFactory` with a single method - * `SearchQueryMetrics makeMetrics(SearchQuery query);`. + * 2. Create `class DefaultSegmentMetadataQueryMetrics implements SegmentMetadataQueryMetrics`. This class should + * implement extra methods from SegmentMetadataQueryMetrics interfaces with empty bodies, AND DELEGATE ALL OTHER + * METHODS TO A QueryMetrics OBJECT, provided as a sole parameter in DefaultSegmentMetadataQueryMetrics constructor. * - * 4. Create `class DefaultSearchQueryMetricsFactory implements SearchQueryMetricsFactory`, which accepts {@link - * GenericQueryMetricsFactory} as injected constructor parameter, and implements makeMetrics() as - * `return new DefaultSearchQueryMetrics(genericQueryMetricsFactory.makeMetrics(query));` + * 3. Create `interface SegmentMetadataQueryMetricsFactory` with a single method + * `SegmentMetadataQueryMetrics makeMetrics(SegmentMetadataQuery query);`. * - * 5. Inject and use SearchQueryMetricsFactory instead of {@link GenericQueryMetricsFactory} in {@link - * io.druid.query.search.SearchQueryQueryToolChest}. + * 4. Create `class DefaultSegmentMetadataQueryMetricsFactory implements SegmentMetadataQueryMetricsFactory`, + * which accepts {@link GenericQueryMetricsFactory} as injected constructor parameter, and implements makeMetrics() as + * `return new DefaultSegmentMetadataQueryMetrics(genericQueryMetricsFactory.makeMetrics(query));` * - * 6. Establish injection of SearchQueryMetricsFactory using config and provider method in QueryToolChestModule - * (see how it is done in QueryToolChestModule for existing query types with custom metrics, e. g. {@link - * io.druid.query.topn.TopNQueryMetricsFactory}), if the query type belongs to the core druid-processing, e. g. - * SearchQuery. If the query type defined in an extension, you can specify + * 5. Inject and use SegmentMetadataQueryMetricsFactory instead of {@link GenericQueryMetricsFactory} in + * {@link io.druid.query.metadata.SegmentMetadataQueryQueryToolChest}. + * + * 6. Establish injection of SegmentMetadataQueryMetricsFactory using config and provider method in + * QueryToolChestModule (see how it is done in {@link io.druid.guice.QueryToolChestModule} for existing query types + * with custom metrics, e. g. {@link io.druid.query.topn.TopNQueryMetricsFactory}), if the query type belongs to the + * core druid-processing, e. g. SegmentMetadataQuery. If the query type defined in an extension, you can specify * `binder.bind(ScanQueryMetricsFactory.class).to(DefaultScanQueryMetricsFactory.class)` in the extension's * Guice module, if the query type is defined in an extension, e. g. ScanQuery. Or establish similar configuration, * as for the core query types. diff --git a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java index 09af924ca55c..5fee46adbb63 100644 --- a/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/groupby/DefaultGroupByQueryMetrics.java @@ -63,5 +63,6 @@ public void numComplexMetrics(GroupByQuery query) @Override public void granularity(GroupByQuery query) { + //Don't emit by default } } 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 49d537ce11c2..642914897fe2 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -129,7 +129,7 @@ protected BinaryFn, Result, Result< @Override public SearchQueryMetrics makeMetrics(SearchQuery query) { - SearchQueryMetrics metrics = queryMetricsFactory.makeMetrics(); + SearchQueryMetrics metrics = queryMetricsFactory.makeMetrics(query); metrics.query(query); return metrics; } diff --git a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java index 0c388e43f525..f98d0adcacec 100644 --- a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java @@ -19,25 +19,220 @@ package io.druid.query.search.search; -import com.fasterxml.jackson.databind.ObjectMapper; -import io.druid.query.DefaultQueryMetrics; +import com.metamx.emitter.service.ServiceEmitter; +import io.druid.collections.bitmap.BitmapFactory; +import io.druid.query.BitmapResultFactory; +import io.druid.query.Query; +import io.druid.query.QueryMetrics; +import io.druid.query.filter.Filter; +import org.joda.time.Interval; -public class DefaultSearchQueryMetrics extends DefaultQueryMetrics implements SearchQueryMetrics +import java.util.List; + +public class DefaultSearchQueryMetrics implements SearchQueryMetrics { - public DefaultSearchQueryMetrics(ObjectMapper jsonMapper) + private QueryMetrics> queryMetrics; + + public DefaultSearchQueryMetrics(QueryMetrics> queryMetrics) { - super(jsonMapper); + this.queryMetrics = queryMetrics; } @Override - public void query (SearchQuery query) + public void query(SearchQuery query) { - super.query(query); + dataSource(query); + queryType(query); + interval(query); + hasFilters(query); + duration(query); + queryId(query); granularity(query); } + @Override + public void dataSource(SearchQuery query) + { + queryMetrics.dataSource(query); + } + + @Override + public void queryType(SearchQuery query) + { + queryMetrics.queryType(query); + } + + @Override + public void interval(SearchQuery query) + { + queryMetrics.interval(query); + } + + @Override + public void hasFilters(SearchQuery query) + { + queryMetrics.hasFilters(query); + } + + @Override + public void duration(SearchQuery query) + { + queryMetrics.duration(query); + } + + @Override + public void queryId(SearchQuery query) + { + queryMetrics.queryId(query); + } + + @Override + public void context(SearchQuery query) + { + queryMetrics.context(query); + } + + @Override + public void server(String host) + { + queryMetrics.server(host); + } + + @Override + public void remoteAddress(String remoteAddress) + { + queryMetrics.remoteAddress(remoteAddress); + } + + @Override + public void status(String status) + { + queryMetrics.status(status); + } + + @Override + public void success(boolean success) + { + queryMetrics.success(success); + } + + @Override + public void segment(String segmentIdentifier) + { + queryMetrics.segment(segmentIdentifier); + } + + @Override + public void chunkInterval(Interval interval) + { + queryMetrics.chunkInterval(interval); + } + + @Override + public void preFilters(List preFilters) + { + queryMetrics.preFilters(preFilters); + } + + @Override + public void postFilters(List postFilters) + { + queryMetrics.postFilters(postFilters); + } + + @Override + public BitmapResultFactory makeBitmapResultFactory(BitmapFactory factory) + { + return queryMetrics.makeBitmapResultFactory(factory); + } + + @Override + public QueryMetrics reportQueryTime(long timeNs) + { + return queryMetrics.reportQueryTime(timeNs); + } + + @Override + public QueryMetrics reportQueryBytes(long byteCount) + { + return queryMetrics.reportQueryBytes(byteCount); + } + + @Override + public QueryMetrics reportWaitTime(long timeNs) + { + return queryMetrics.reportWaitTime(timeNs); + } + + @Override + public QueryMetrics reportSegmentTime(long timeNs) + { + return queryMetrics.reportSegmentTime(timeNs); + } + + @Override + public QueryMetrics reportSegmentAndCacheTime(long timeNs) + { + return queryMetrics.reportSegmentAndCacheTime(timeNs); + } + + @Override + public QueryMetrics reportIntervalChunkTime(long timeNs) + { + return queryMetrics.reportIntervalChunkTime(timeNs); + } + + @Override + public QueryMetrics reportCpuTime(long timeNs) + { + return queryMetrics.reportCpuTime(timeNs); + } + + @Override + public QueryMetrics reportNodeTimeToFirstByte(long timeNs) + { + return queryMetrics.reportNodeTimeToFirstByte(timeNs); + } + + @Override + public QueryMetrics reportNodeTime(long timeNs) + { + return queryMetrics.reportNodeTime(timeNs); + } + + @Override + public QueryMetrics reportNodeBytes(long byteCount) + { + return queryMetrics.reportNodeBytes(byteCount); + } + + @Override + public QueryMetrics reportBitmapConstructionTime(long timeNs) + { + return queryMetrics.reportBitmapConstructionTime(timeNs); + } + + @Override + public QueryMetrics reportSegmentRows(long numRows) + { + return queryMetrics.reportSegmentRows(numRows); + } + + @Override + public QueryMetrics reportPreFilteredRows(long numRows) + { + return queryMetrics.reportPreFilteredRows(numRows); + } + + @Override + public void emit(ServiceEmitter emitter) + { + queryMetrics.emit(emitter); + } + @Override public void granularity(SearchQuery query) { + // Don't emit by default } } diff --git a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetricsFactory.java index d8f6cc0318f2..93b40da78257 100644 --- a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetricsFactory.java @@ -19,34 +19,32 @@ package io.druid.query.search.search; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; -import io.druid.guice.annotations.Json; -import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.DefaultGenericQueryMetricsFactory; +import io.druid.query.GenericQueryMetricsFactory; public class DefaultSearchQueryMetricsFactory implements SearchQueryMetricsFactory { - private final ObjectMapper jsonMapper; private static final SearchQueryMetricsFactory INSTANCE = - new DefaultSearchQueryMetricsFactory(new DefaultObjectMapper()); + new DefaultSearchQueryMetricsFactory(DefaultGenericQueryMetricsFactory.instance()); + private final GenericQueryMetricsFactory genericQueryMetricsFactory; - //Used only for testng. - @VisibleForTesting - public static SearchQueryMetricsFactory instance() + @Inject + public DefaultSearchQueryMetricsFactory(GenericQueryMetricsFactory genericQueryMetricsFactory) { - return INSTANCE; + this.genericQueryMetricsFactory = genericQueryMetricsFactory; } - @Inject - public DefaultSearchQueryMetricsFactory(@Json ObjectMapper jsonMapper) + @VisibleForTesting + public static SearchQueryMetricsFactory instance() { - this.jsonMapper = jsonMapper; + return INSTANCE; } @Override - public SearchQueryMetrics makeMetrics() + public SearchQueryMetrics makeMetrics(SearchQuery query) { - return new DefaultSearchQueryMetrics(jsonMapper); + return new DefaultSearchQueryMetrics(genericQueryMetricsFactory.makeMetrics(query)); } } diff --git a/processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java index dc9eaba3ce50..1743ca228b5a 100644 --- a/processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java @@ -21,16 +21,16 @@ /** * Implementations could be injected using - * + *

* PolyBind - * .optionBinder(binder, Key.get(SearchQueryMetricsFactory.class)) - * .addBinding("myCustomSearchQueryMetricsFactory") - * .to(MyCustomSearchQueryMetricsFactory.class); - * + * .optionBinder(binder, Key.get(SearchQueryMetricsFactory.class)) + * .addBinding("myCustomSearchQueryMetricsFactory") + * .to(MyCustomSearchQueryMetricsFactory.class); + *

* And then setting property: * druid.query.search.queryMetricsFactory=myCustomSearchQueryMetricsFactory */ public interface SearchQueryMetricsFactory { - SearchQueryMetrics makeMetrics(); + SearchQueryMetrics makeMetrics(SearchQuery query); } diff --git a/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java index 2cbdd71a0a00..641783935433 100644 --- a/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java @@ -19,25 +19,221 @@ package io.druid.query.select; -import com.fasterxml.jackson.databind.ObjectMapper; -import io.druid.query.DefaultQueryMetrics; +import com.metamx.emitter.service.ServiceEmitter; +import io.druid.collections.bitmap.BitmapFactory; +import io.druid.query.BitmapResultFactory; +import io.druid.query.Query; +import io.druid.query.QueryMetrics; +import io.druid.query.filter.Filter; +import org.joda.time.Interval; -public class DefaultSelectQueryMetrics extends DefaultQueryMetrics implements SelectQueryMetrics +import java.util.List; + +public class DefaultSelectQueryMetrics implements SelectQueryMetrics { - public DefaultSelectQueryMetrics(ObjectMapper jsonMapper) + private QueryMetrics> queryMetrics; + + public DefaultSelectQueryMetrics(QueryMetrics> queryMetrics) { - super(jsonMapper); + this.queryMetrics = queryMetrics; } @Override - public void query (SelectQuery query) + public void query(SelectQuery query) { - super.query(query); + dataSource(query); + queryType(query); + interval(query); + hasFilters(query); + duration(query); + queryId(query); granularity(query); } + @Override + public void dataSource(SelectQuery query) + { + queryMetrics.dataSource(query); + } + + @Override + public void queryType(SelectQuery query) + { + queryMetrics.queryType(query); + } + + @Override + public void interval(SelectQuery query) + { + queryMetrics.interval(query); + } + + @Override + public void hasFilters(SelectQuery query) + { + queryMetrics.hasFilters(query); + } + + @Override + public void duration(SelectQuery query) + { + queryMetrics.duration(query); + } + + @Override + public void queryId(SelectQuery query) + { + queryMetrics.queryId(query); + } + + @Override + public void context(SelectQuery query) + { + queryMetrics.context(query); + } + + @Override + public void server(String host) + { + queryMetrics.server(host); + } + + @Override + public void remoteAddress(String remoteAddress) + { + queryMetrics.remoteAddress(remoteAddress); + } + + @Override + public void status(String status) + { + queryMetrics.status(status); + } + + @Override + public void success(boolean success) + { + queryMetrics.success(success); + } + + @Override + public void segment(String segmentIdentifier) + { + queryMetrics.segment(segmentIdentifier); + } + + @Override + public void chunkInterval(Interval interval) + { + queryMetrics.chunkInterval(interval); + } + + @Override + public void preFilters(List preFilters) + { + queryMetrics.preFilters(preFilters); + } + + @Override + public void postFilters(List postFilters) + { + queryMetrics.postFilters(postFilters); + } + + @Override + public BitmapResultFactory makeBitmapResultFactory(BitmapFactory factory) + { + return queryMetrics.makeBitmapResultFactory(factory); + } + + @Override + public QueryMetrics reportQueryTime(long timeNs) + { + return queryMetrics.reportQueryTime(timeNs); + } + + @Override + public QueryMetrics reportQueryBytes(long byteCount) + { + return queryMetrics.reportQueryBytes(byteCount); + } + + @Override + public QueryMetrics reportWaitTime(long timeNs) + { + return queryMetrics.reportWaitTime(timeNs); + } + + @Override + public QueryMetrics reportSegmentTime(long timeNs) + { + return queryMetrics.reportSegmentTime(timeNs); + } + + @Override + public QueryMetrics reportSegmentAndCacheTime(long timeNs) + { + return queryMetrics.reportSegmentAndCacheTime(timeNs); + } + + @Override + public QueryMetrics reportIntervalChunkTime(long timeNs) + { + return queryMetrics.reportIntervalChunkTime(timeNs); + } + + @Override + public QueryMetrics reportCpuTime(long timeNs) + { + return queryMetrics.reportCpuTime(timeNs); + } + + @Override + public QueryMetrics reportNodeTimeToFirstByte(long timeNs) + { + return queryMetrics.reportNodeTimeToFirstByte(timeNs); + } + + @Override + public QueryMetrics reportNodeTime(long timeNs) + { + return queryMetrics.reportNodeTime(timeNs); + } + + @Override + public QueryMetrics reportNodeBytes(long byteCount) + { + return queryMetrics.reportNodeBytes(byteCount); + } + + @Override + public QueryMetrics reportBitmapConstructionTime(long timeNs) + { + return queryMetrics.reportBitmapConstructionTime(timeNs); + } + + @Override + public QueryMetrics reportSegmentRows(long numRows) + { + return queryMetrics.reportSegmentRows(numRows); + } + + @Override + public QueryMetrics reportPreFilteredRows(long numRows) + { + return queryMetrics.reportPreFilteredRows(numRows); + } + + @Override + public void emit(ServiceEmitter emitter) + { + queryMetrics.emit(emitter); + } + @Override public void granularity(SelectQuery query) { + // Don't emit by default } + } diff --git a/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetricsFactory.java index 7a1d438c1606..5c90dae4777a 100644 --- a/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetricsFactory.java @@ -19,34 +19,33 @@ package io.druid.query.select; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; -import io.druid.guice.annotations.Json; -import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.DefaultGenericQueryMetricsFactory; +import io.druid.query.GenericQueryMetricsFactory; public class DefaultSelectQueryMetricsFactory implements SelectQueryMetricsFactory { - private final ObjectMapper jsonMapper; private static final SelectQueryMetricsFactory INSTANCE = - new DefaultSelectQueryMetricsFactory(new DefaultObjectMapper()); + new DefaultSelectQueryMetricsFactory(DefaultGenericQueryMetricsFactory.instance()); - //Used only for testng. - @VisibleForTesting - public static SelectQueryMetricsFactory instance() + private final GenericQueryMetricsFactory genericQueryMetricsFactory; + + @Inject + public DefaultSelectQueryMetricsFactory(GenericQueryMetricsFactory genericQueryMetricsFactory) { - return INSTANCE; + this.genericQueryMetricsFactory = genericQueryMetricsFactory; } - @Inject - public DefaultSelectQueryMetricsFactory(@Json ObjectMapper jsonMapper) + @VisibleForTesting + public static SelectQueryMetricsFactory instance() { - this.jsonMapper = jsonMapper; + return INSTANCE; } @Override - public SelectQueryMetrics makeMetrics() + public SelectQueryMetrics makeMetrics(SelectQuery query) { - return new DefaultSelectQueryMetrics(jsonMapper); + return new DefaultSelectQueryMetrics(genericQueryMetricsFactory.makeMetrics(query)); } } diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/select/SelectQueryMetricsFactory.java index 54f801c485b1..cae363364668 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryMetricsFactory.java @@ -32,5 +32,5 @@ */ public interface SelectQueryMetricsFactory { - SelectQueryMetrics makeMetrics(); + SelectQueryMetrics makeMetrics(SelectQuery selectQuery); } 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 e997bb67427f..08a75e2eb3ed 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -88,7 +88,7 @@ public SelectQueryQueryToolChest( Supplier configSupplier ) { - this(jsonMapper, intervalChunkingQueryRunnerDecorator, configSupplier, new DefaultSelectQueryMetricsFactory(jsonMapper)); + this(jsonMapper, intervalChunkingQueryRunnerDecorator, configSupplier, DefaultSelectQueryMetricsFactory.instance()); } @Inject @@ -138,7 +138,7 @@ protected BinaryFn, Result, Result< @Override public SelectQueryMetrics makeMetrics(SelectQuery query) { - SelectQueryMetrics queryMetrics = queryMetricsFactory.makeMetrics(); + SelectQueryMetrics queryMetrics = queryMetricsFactory.makeMetrics(query); queryMetrics.query(query); return queryMetrics; } diff --git a/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java index 5d02ee09216e..b7f0f9a4507c 100644 --- a/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/timeseries/DefaultTimeseriesQueryMetrics.java @@ -56,5 +56,6 @@ public void numComplexMetrics(TimeseriesQuery query) @Override public void granularity(TimeseriesQuery query) { + // Don't emit by default } } diff --git a/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java index 91ffd8360d85..91a81b32c913 100644 --- a/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/topn/DefaultTopNQueryMetrics.java @@ -72,6 +72,7 @@ public void numComplexMetrics(TopNQuery query) @Override public void granularity(TopNQuery query) { + // Don't emit by default } @Override diff --git a/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java b/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java index 3982d2402f4f..b749e4ad74aa 100644 --- a/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java @@ -29,8 +29,9 @@ import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.ListFilteredDimensionSpec; import io.druid.query.search.search.DefaultSearchQueryMetrics; +import io.druid.query.search.search.DefaultSearchQueryMetricsFactory; import io.druid.query.search.search.SearchQuery; -import io.druid.segment.TestHelper; +import io.druid.query.search.search.SearchQueryMetrics; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -50,7 +51,6 @@ public void testDefaultSearchQueryMetricsQuery() { CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultSearchQueryMetrics queryMetrics = new DefaultSearchQueryMetrics(TestHelper.getJsonMapper()); SearchQuery query = Druids .newSearchQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) @@ -62,6 +62,9 @@ public void testDefaultSearchQueryMetricsQuery() null )) .build(); + + SearchQueryMetrics queryMetrics = DefaultSearchQueryMetricsFactory.instance().makeMetrics(query); + queryMetrics.query(query); queryMetrics.reportQueryTime(0).emit(serviceEmitter); @@ -89,9 +92,16 @@ public void testDefaultSearchQueryMetricsQuery() @Test public void testDefaultSearchQueryMetricsMetricNamesAndUnits() { + SearchQuery query = Druids + .newSearchQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .build(); + CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultSearchQueryMetrics queryMetrics = new DefaultSearchQueryMetrics(TestHelper.getJsonMapper()); + SearchQueryMetrics queryMetrics = DefaultSearchQueryMetricsFactory.instance().makeMetrics(query); DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); } } diff --git a/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java b/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java index bd2ec7438c6f..942a027d4a12 100644 --- a/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java @@ -45,7 +45,6 @@ public void testDefaultSelectQueryMetricsQuery() { CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultSelectQueryMetrics queryMetrics = new DefaultSelectQueryMetrics(TestHelper.getJsonMapper()); SelectQuery query = Druids .newSelectQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) @@ -54,6 +53,9 @@ public void testDefaultSelectQueryMetricsQuery() .descending(true) .pagingSpec(PagingSpec.newSpec(1)) .build(); + + SelectQueryMetrics queryMetrics = DefaultSelectQueryMetricsFactory.instance().makeMetrics(query); + queryMetrics.query(query); queryMetrics.reportQueryTime(0).emit(serviceEmitter); @@ -83,7 +85,16 @@ public void testDefaultSelectQueryMetricsMetricNamesAndUnits() { CachingEmitter cachingEmitter = new CachingEmitter(); ServiceEmitter serviceEmitter = new ServiceEmitter("", "", cachingEmitter); - DefaultSelectQueryMetrics queryMetrics = new DefaultSelectQueryMetrics(TestHelper.getJsonMapper()); + SelectQuery query = Druids + .newSelectQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.dayGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .descending(true) + .pagingSpec(PagingSpec.newSpec(1)) + .build(); + + SelectQueryMetrics queryMetrics = DefaultSelectQueryMetricsFactory.instance().makeMetrics(query); DefaultQueryMetricsTest.testQueryMetricsDefaultMetricNamesAndUnits(cachingEmitter, serviceEmitter, queryMetrics); } } From dd4bf9655f3ceadae113cb538edc138bbeed3c7d Mon Sep 17 00:00:00 2001 From: Akash Dwivedi Date: Tue, 15 Aug 2017 13:46:45 -0700 Subject: [PATCH 5/8] Remove unused import. --- .../io/druid/query/select/DefaultSelectQueryMetricsTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java b/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java index 942a027d4a12..6aeefe2b2957 100644 --- a/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/select/DefaultSelectQueryMetricsTest.java @@ -25,7 +25,6 @@ import io.druid.query.DruidMetrics; import io.druid.query.Druids; import io.druid.query.QueryRunnerTestHelper; -import io.druid.segment.TestHelper; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; From 84c1493cb2cec2d0bb736deea7ae53b1f7e188a2 Mon Sep 17 00:00:00 2001 From: Akash Dwivedi Date: Tue, 5 Sep 2017 15:24:14 -0700 Subject: [PATCH 6/8] empty query() method. --- .../java/io/druid/query/QueryMetrics.java | 12 ++- .../search/DefaultSearchQueryMetrics.java | 92 ++++++++++--------- .../search/SearchQueryMetricsFactory.java | 12 +-- .../select/DefaultSelectQueryMetrics.java | 92 +++++++++---------- 4 files changed, 108 insertions(+), 100 deletions(-) diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index b6dff99b2b8a..f2d02a6f82bd 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -121,6 +121,9 @@ * implement extra methods from SegmentMetadataQueryMetrics interfaces with empty bodies, AND DELEGATE ALL OTHER * METHODS TO A QueryMetrics OBJECT, provided as a sole parameter in DefaultSegmentMetadataQueryMetrics constructor. * + * NOTE: query(),dataSource(), queryType(), interval(), hasFilters(), duration() and queryId() methods or any + * "pre-query-execution-time" methods should either have a empty body or throw exception. + * * 3. Create `interface SegmentMetadataQueryMetricsFactory` with a single method * `SegmentMetadataQueryMetrics makeMetrics(SegmentMetadataQuery query);`. * @@ -133,9 +136,9 @@ * * 6. Establish injection of SegmentMetadataQueryMetricsFactory using config and provider method in * QueryToolChestModule (see how it is done in {@link io.druid.guice.QueryToolChestModule} for existing query types - * with custom metrics, e. g. {@link io.druid.query.topn.TopNQueryMetricsFactory}), if the query type belongs to the - * core druid-processing, e. g. SegmentMetadataQuery. If the query type defined in an extension, you can specify - * `binder.bind(ScanQueryMetricsFactory.class).to(DefaultScanQueryMetricsFactory.class)` in the extension's + * with custom metrics, e. g. {@link io.druid.query.search.search.SearchQueryMetricsFactory}), if the query type + * belongs to the core druid-processing, e. g. SegmentMetadataQuery. If the query type defined in an extension, you + * can specify `binder.bind(ScanQueryMetricsFactory.class).to(DefaultScanQueryMetricsFactory.class)` in the extension's * Guice module, if the query type is defined in an extension, e. g. ScanQuery. Or establish similar configuration, * as for the core query types. * @@ -148,6 +151,9 @@ * dimensions than the default generic QueryMetrics. So those subinterfaces shouldn't be taken as direct examples for * following the plan specified above. * + * Refer {@link io.druid.query.search.search.SearchQueryMetricsFactory} + * and {@link io.druid.query.select.SelectQueryMetricsFactory} as an implementation example of this procedure. + * * @param */ public interface QueryMetrics> diff --git a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java index f98d0adcacec..8b8b404bd3ab 100644 --- a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java @@ -21,6 +21,7 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.collections.bitmap.BitmapFactory; +import io.druid.java.util.common.ISE; import io.druid.query.BitmapResultFactory; import io.druid.query.Query; import io.druid.query.QueryMetrics; @@ -29,210 +30,211 @@ import java.util.List; +/* + * This class is implemented with delegation to another QueryMetrics for compatibility, see "Making subinterfaces of + * QueryMetrics for emitting custom dimensions and/or metrics for specific query types" section in {@link QueryMetrics} + * javadoc. + */ public class DefaultSearchQueryMetrics implements SearchQueryMetrics { - private QueryMetrics> queryMetrics; + private QueryMetrics> delegateQueryMetrics; + + // queryMetrics.query(query) must already be called on the provided queryMetrics. public DefaultSearchQueryMetrics(QueryMetrics> queryMetrics) { - this.queryMetrics = queryMetrics; + this.delegateQueryMetrics = queryMetrics; } @Override public void query(SearchQuery query) { - dataSource(query); - queryType(query); - interval(query); - hasFilters(query); - duration(query); - queryId(query); - granularity(query); + //delegateQueryMetrics.query(query) must already be called on the provided queryMetrics. } @Override public void dataSource(SearchQuery query) { - queryMetrics.dataSource(query); + throw new ISE("Unsupported method in default query metrics implementation."); } @Override public void queryType(SearchQuery query) { - queryMetrics.queryType(query); + throw new ISE("Unsupported method in default query metrics implementation."); } @Override public void interval(SearchQuery query) { - queryMetrics.interval(query); + throw new ISE("Unsupported method in default query metrics implementation."); } @Override public void hasFilters(SearchQuery query) { - queryMetrics.hasFilters(query); + throw new ISE("Unsupported method in default query metrics implementation."); } @Override public void duration(SearchQuery query) { - queryMetrics.duration(query); + throw new ISE("Unsupported method in default query metrics implementation."); } @Override public void queryId(SearchQuery query) { - queryMetrics.queryId(query); + throw new ISE("Unsupported method in default query metrics implementation."); + } + + @Override + public void granularity(SearchQuery query) + { + // Don't emit by default } @Override public void context(SearchQuery query) { - queryMetrics.context(query); + delegateQueryMetrics.context(query); } @Override public void server(String host) { - queryMetrics.server(host); + delegateQueryMetrics.server(host); } @Override public void remoteAddress(String remoteAddress) { - queryMetrics.remoteAddress(remoteAddress); + delegateQueryMetrics.remoteAddress(remoteAddress); } @Override public void status(String status) { - queryMetrics.status(status); + delegateQueryMetrics.status(status); } @Override public void success(boolean success) { - queryMetrics.success(success); + delegateQueryMetrics.success(success); } @Override public void segment(String segmentIdentifier) { - queryMetrics.segment(segmentIdentifier); + delegateQueryMetrics.segment(segmentIdentifier); } @Override public void chunkInterval(Interval interval) { - queryMetrics.chunkInterval(interval); + delegateQueryMetrics.chunkInterval(interval); } @Override public void preFilters(List preFilters) { - queryMetrics.preFilters(preFilters); + delegateQueryMetrics.preFilters(preFilters); } @Override public void postFilters(List postFilters) { - queryMetrics.postFilters(postFilters); + delegateQueryMetrics.postFilters(postFilters); } @Override public BitmapResultFactory makeBitmapResultFactory(BitmapFactory factory) { - return queryMetrics.makeBitmapResultFactory(factory); + return delegateQueryMetrics.makeBitmapResultFactory(factory); } @Override public QueryMetrics reportQueryTime(long timeNs) { - return queryMetrics.reportQueryTime(timeNs); + return delegateQueryMetrics.reportQueryTime(timeNs); } @Override public QueryMetrics reportQueryBytes(long byteCount) { - return queryMetrics.reportQueryBytes(byteCount); + return delegateQueryMetrics.reportQueryBytes(byteCount); } @Override public QueryMetrics reportWaitTime(long timeNs) { - return queryMetrics.reportWaitTime(timeNs); + return delegateQueryMetrics.reportWaitTime(timeNs); } @Override public QueryMetrics reportSegmentTime(long timeNs) { - return queryMetrics.reportSegmentTime(timeNs); + return delegateQueryMetrics.reportSegmentTime(timeNs); } @Override public QueryMetrics reportSegmentAndCacheTime(long timeNs) { - return queryMetrics.reportSegmentAndCacheTime(timeNs); + return delegateQueryMetrics.reportSegmentAndCacheTime(timeNs); } @Override public QueryMetrics reportIntervalChunkTime(long timeNs) { - return queryMetrics.reportIntervalChunkTime(timeNs); + return delegateQueryMetrics.reportIntervalChunkTime(timeNs); } @Override public QueryMetrics reportCpuTime(long timeNs) { - return queryMetrics.reportCpuTime(timeNs); + return delegateQueryMetrics.reportCpuTime(timeNs); } @Override public QueryMetrics reportNodeTimeToFirstByte(long timeNs) { - return queryMetrics.reportNodeTimeToFirstByte(timeNs); + return delegateQueryMetrics.reportNodeTimeToFirstByte(timeNs); } @Override public QueryMetrics reportNodeTime(long timeNs) { - return queryMetrics.reportNodeTime(timeNs); + return delegateQueryMetrics.reportNodeTime(timeNs); } @Override public QueryMetrics reportNodeBytes(long byteCount) { - return queryMetrics.reportNodeBytes(byteCount); + return delegateQueryMetrics.reportNodeBytes(byteCount); } @Override public QueryMetrics reportBitmapConstructionTime(long timeNs) { - return queryMetrics.reportBitmapConstructionTime(timeNs); + return delegateQueryMetrics.reportBitmapConstructionTime(timeNs); } @Override public QueryMetrics reportSegmentRows(long numRows) { - return queryMetrics.reportSegmentRows(numRows); + return delegateQueryMetrics.reportSegmentRows(numRows); } @Override public QueryMetrics reportPreFilteredRows(long numRows) { - return queryMetrics.reportPreFilteredRows(numRows); + return delegateQueryMetrics.reportPreFilteredRows(numRows); } @Override public void emit(ServiceEmitter emitter) { - queryMetrics.emit(emitter); - } - - @Override - public void granularity(SearchQuery query) - { - // Don't emit by default + delegateQueryMetrics.emit(emitter); } } diff --git a/processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java index 1743ca228b5a..ed857573489a 100644 --- a/processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java @@ -21,16 +21,16 @@ /** * Implementations could be injected using - *

+ * * PolyBind - * .optionBinder(binder, Key.get(SearchQueryMetricsFactory.class)) - * .addBinding("myCustomSearchQueryMetricsFactory") - * .to(MyCustomSearchQueryMetricsFactory.class); - *

+ * .optionBinder(binder, Key.get(SearchQueryMetricsFactory.class)) + * .addBinding("myCustomSearchQueryMetricsFactory") + * .to(MyCustomSearchQueryMetricsFactory.class); + * * And then setting property: * druid.query.search.queryMetricsFactory=myCustomSearchQueryMetricsFactory */ public interface SearchQueryMetricsFactory { - SearchQueryMetrics makeMetrics(SearchQuery query); + SearchQueryMetrics makeMetrics(SearchQuery searchQuery); } diff --git a/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java index 641783935433..7a2a9dbec246 100644 --- a/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java @@ -21,6 +21,7 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.collections.bitmap.BitmapFactory; +import io.druid.java.util.common.ISE; import io.druid.query.BitmapResultFactory; import io.druid.query.Query; import io.druid.query.QueryMetrics; @@ -29,211 +30,210 @@ import java.util.List; +/* + * This class is implemented with delegation to another QueryMetrics for compatibility, see "Making subinterfaces of + * QueryMetrics for emitting custom dimensions and/or metrics for specific query types" section in {@link QueryMetrics} + * javadoc. + */ public class DefaultSelectQueryMetrics implements SelectQueryMetrics { - private QueryMetrics> queryMetrics; + private QueryMetrics> delegateQueryMetrics; + // queryMetrics.query(query) must already be called on the provided queryMetrics. public DefaultSelectQueryMetrics(QueryMetrics> queryMetrics) { - this.queryMetrics = queryMetrics; + this.delegateQueryMetrics = queryMetrics; } @Override public void query(SelectQuery query) { - dataSource(query); - queryType(query); - interval(query); - hasFilters(query); - duration(query); - queryId(query); - granularity(query); + // delegateQueryMetrics.query(query) must already be called on the provided queryMetrics. } @Override public void dataSource(SelectQuery query) { - queryMetrics.dataSource(query); + throw new ISE("Unsupported method in default query metrics implementation."); } @Override public void queryType(SelectQuery query) { - queryMetrics.queryType(query); + throw new ISE("Unsupported method in default query metrics implementation."); } @Override public void interval(SelectQuery query) { - queryMetrics.interval(query); + throw new ISE("Unsupported method in default query metrics implementation."); } @Override public void hasFilters(SelectQuery query) { - queryMetrics.hasFilters(query); + throw new ISE("Unsupported method in default query metrics implementation."); } @Override public void duration(SelectQuery query) { - queryMetrics.duration(query); + throw new ISE("Unsupported method in default query metrics implementation."); } @Override public void queryId(SelectQuery query) { - queryMetrics.queryId(query); + throw new ISE("Unsupported method in default query metrics implementation."); + } + + @Override + public void granularity(SelectQuery query) + { + // Don't emit by default } @Override public void context(SelectQuery query) { - queryMetrics.context(query); + delegateQueryMetrics.context(query); } @Override public void server(String host) { - queryMetrics.server(host); + delegateQueryMetrics.server(host); } @Override public void remoteAddress(String remoteAddress) { - queryMetrics.remoteAddress(remoteAddress); + delegateQueryMetrics.remoteAddress(remoteAddress); } @Override public void status(String status) { - queryMetrics.status(status); + delegateQueryMetrics.status(status); } @Override public void success(boolean success) { - queryMetrics.success(success); + delegateQueryMetrics.success(success); } @Override public void segment(String segmentIdentifier) { - queryMetrics.segment(segmentIdentifier); + delegateQueryMetrics.segment(segmentIdentifier); } @Override public void chunkInterval(Interval interval) { - queryMetrics.chunkInterval(interval); + delegateQueryMetrics.chunkInterval(interval); } @Override public void preFilters(List preFilters) { - queryMetrics.preFilters(preFilters); + delegateQueryMetrics.preFilters(preFilters); } @Override public void postFilters(List postFilters) { - queryMetrics.postFilters(postFilters); + delegateQueryMetrics.postFilters(postFilters); } @Override public BitmapResultFactory makeBitmapResultFactory(BitmapFactory factory) { - return queryMetrics.makeBitmapResultFactory(factory); + return delegateQueryMetrics.makeBitmapResultFactory(factory); } @Override public QueryMetrics reportQueryTime(long timeNs) { - return queryMetrics.reportQueryTime(timeNs); + return delegateQueryMetrics.reportQueryTime(timeNs); } @Override public QueryMetrics reportQueryBytes(long byteCount) { - return queryMetrics.reportQueryBytes(byteCount); + return delegateQueryMetrics.reportQueryBytes(byteCount); } @Override public QueryMetrics reportWaitTime(long timeNs) { - return queryMetrics.reportWaitTime(timeNs); + return delegateQueryMetrics.reportWaitTime(timeNs); } @Override public QueryMetrics reportSegmentTime(long timeNs) { - return queryMetrics.reportSegmentTime(timeNs); + return delegateQueryMetrics.reportSegmentTime(timeNs); } @Override public QueryMetrics reportSegmentAndCacheTime(long timeNs) { - return queryMetrics.reportSegmentAndCacheTime(timeNs); + return delegateQueryMetrics.reportSegmentAndCacheTime(timeNs); } @Override public QueryMetrics reportIntervalChunkTime(long timeNs) { - return queryMetrics.reportIntervalChunkTime(timeNs); + return delegateQueryMetrics.reportIntervalChunkTime(timeNs); } @Override public QueryMetrics reportCpuTime(long timeNs) { - return queryMetrics.reportCpuTime(timeNs); + return delegateQueryMetrics.reportCpuTime(timeNs); } @Override public QueryMetrics reportNodeTimeToFirstByte(long timeNs) { - return queryMetrics.reportNodeTimeToFirstByte(timeNs); + return delegateQueryMetrics.reportNodeTimeToFirstByte(timeNs); } @Override public QueryMetrics reportNodeTime(long timeNs) { - return queryMetrics.reportNodeTime(timeNs); + return delegateQueryMetrics.reportNodeTime(timeNs); } @Override public QueryMetrics reportNodeBytes(long byteCount) { - return queryMetrics.reportNodeBytes(byteCount); + return delegateQueryMetrics.reportNodeBytes(byteCount); } @Override public QueryMetrics reportBitmapConstructionTime(long timeNs) { - return queryMetrics.reportBitmapConstructionTime(timeNs); + return delegateQueryMetrics.reportBitmapConstructionTime(timeNs); } @Override public QueryMetrics reportSegmentRows(long numRows) { - return queryMetrics.reportSegmentRows(numRows); + return delegateQueryMetrics.reportSegmentRows(numRows); } @Override public QueryMetrics reportPreFilteredRows(long numRows) { - return queryMetrics.reportPreFilteredRows(numRows); + return delegateQueryMetrics.reportPreFilteredRows(numRows); } @Override public void emit(ServiceEmitter emitter) { - queryMetrics.emit(emitter); + delegateQueryMetrics.emit(emitter); } - - @Override - public void granularity(SelectQuery query) - { - // Don't emit by default - } - } From 65775e0fb1d750f44e173751bedf9ad355a3302e Mon Sep 17 00:00:00 2001 From: dwivedi Date: Tue, 5 Sep 2017 22:14:41 -0700 Subject: [PATCH 7/8] checkstyle fix. --- processing/src/main/java/io/druid/query/QueryMetrics.java | 2 +- .../io/druid/query/search/search/DefaultSearchQueryMetrics.java | 2 +- .../java/io/druid/query/select/DefaultSelectQueryMetrics.java | 2 +- 3 files changed, 3 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 f2d02a6f82bd..04781132134a 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -121,7 +121,7 @@ * implement extra methods from SegmentMetadataQueryMetrics interfaces with empty bodies, AND DELEGATE ALL OTHER * METHODS TO A QueryMetrics OBJECT, provided as a sole parameter in DefaultSegmentMetadataQueryMetrics constructor. * - * NOTE: query(),dataSource(), queryType(), interval(), hasFilters(), duration() and queryId() methods or any + * NOTE: query(), dataSource(), queryType(), interval(), hasFilters(), duration() and queryId() methods or any * "pre-query-execution-time" methods should either have a empty body or throw exception. * * 3. Create `interface SegmentMetadataQueryMetricsFactory` with a single method diff --git a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java index 8b8b404bd3ab..a01d2ab4c00c 100644 --- a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java @@ -30,7 +30,7 @@ import java.util.List; -/* +/** * This class is implemented with delegation to another QueryMetrics for compatibility, see "Making subinterfaces of * QueryMetrics for emitting custom dimensions and/or metrics for specific query types" section in {@link QueryMetrics} * javadoc. diff --git a/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java index 7a2a9dbec246..3fbae8a99038 100644 --- a/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/select/DefaultSelectQueryMetrics.java @@ -30,7 +30,7 @@ import java.util.List; -/* +/** * This class is implemented with delegation to another QueryMetrics for compatibility, see "Making subinterfaces of * QueryMetrics for emitting custom dimensions and/or metrics for specific query types" section in {@link QueryMetrics} * javadoc. From f82d58b57106102f02c03e9408434083d98e8a69 Mon Sep 17 00:00:00 2001 From: dwivedi Date: Thu, 14 Sep 2017 17:40:14 -0700 Subject: [PATCH 8/8] Import fix. --- processing/src/main/java/io/druid/query/QueryMetrics.java | 5 +++-- .../search/{search => }/DefaultSearchQueryMetrics.java | 2 +- .../{search => }/DefaultSearchQueryMetricsFactory.java | 2 +- .../druid/query/search/{search => }/SearchQueryMetrics.java | 2 +- .../search/{search => }/SearchQueryMetricsFactory.java | 2 +- .../io/druid/query/search/SearchQueryQueryToolChest.java | 6 ------ .../druid/query/search/DefaultSearchQueryMetricsTest.java | 4 ---- 7 files changed, 7 insertions(+), 16 deletions(-) rename processing/src/main/java/io/druid/query/search/{search => }/DefaultSearchQueryMetrics.java (99%) rename processing/src/main/java/io/druid/query/search/{search => }/DefaultSearchQueryMetricsFactory.java (97%) rename processing/src/main/java/io/druid/query/search/{search => }/SearchQueryMetrics.java (96%) rename processing/src/main/java/io/druid/query/search/{search => }/SearchQueryMetricsFactory.java (97%) diff --git a/processing/src/main/java/io/druid/query/QueryMetrics.java b/processing/src/main/java/io/druid/query/QueryMetrics.java index 04781132134a..6e1198b1f05e 100644 --- a/processing/src/main/java/io/druid/query/QueryMetrics.java +++ b/processing/src/main/java/io/druid/query/QueryMetrics.java @@ -22,6 +22,7 @@ import com.metamx.emitter.service.ServiceEmitter; import io.druid.collections.bitmap.BitmapFactory; import io.druid.query.filter.Filter; +import io.druid.query.search.SearchQueryMetricsFactory; import org.joda.time.Interval; import java.util.List; @@ -136,7 +137,7 @@ * * 6. Establish injection of SegmentMetadataQueryMetricsFactory using config and provider method in * QueryToolChestModule (see how it is done in {@link io.druid.guice.QueryToolChestModule} for existing query types - * with custom metrics, e. g. {@link io.druid.query.search.search.SearchQueryMetricsFactory}), if the query type + * with custom metrics, e. g. {@link SearchQueryMetricsFactory}), if the query type * belongs to the core druid-processing, e. g. SegmentMetadataQuery. If the query type defined in an extension, you * can specify `binder.bind(ScanQueryMetricsFactory.class).to(DefaultScanQueryMetricsFactory.class)` in the extension's * Guice module, if the query type is defined in an extension, e. g. ScanQuery. Or establish similar configuration, @@ -151,7 +152,7 @@ * dimensions than the default generic QueryMetrics. So those subinterfaces shouldn't be taken as direct examples for * following the plan specified above. * - * Refer {@link io.druid.query.search.search.SearchQueryMetricsFactory} + * Refer {@link SearchQueryMetricsFactory} * and {@link io.druid.query.select.SelectQueryMetricsFactory} as an implementation example of this procedure. * * @param diff --git a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java b/processing/src/main/java/io/druid/query/search/DefaultSearchQueryMetrics.java similarity index 99% rename from processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java rename to processing/src/main/java/io/druid/query/search/DefaultSearchQueryMetrics.java index a01d2ab4c00c..43fc752548c0 100644 --- a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/search/DefaultSearchQueryMetrics.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.query.search.search; +package io.druid.query.search; import com.metamx.emitter.service.ServiceEmitter; import io.druid.collections.bitmap.BitmapFactory; diff --git a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/search/DefaultSearchQueryMetricsFactory.java similarity index 97% rename from processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetricsFactory.java rename to processing/src/main/java/io/druid/query/search/DefaultSearchQueryMetricsFactory.java index 93b40da78257..47485ed90b89 100644 --- a/processing/src/main/java/io/druid/query/search/search/DefaultSearchQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/search/DefaultSearchQueryMetricsFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.query.search.search; +package io.druid.query.search; import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; diff --git a/processing/src/main/java/io/druid/query/search/search/SearchQueryMetrics.java b/processing/src/main/java/io/druid/query/search/SearchQueryMetrics.java similarity index 96% rename from processing/src/main/java/io/druid/query/search/search/SearchQueryMetrics.java rename to processing/src/main/java/io/druid/query/search/SearchQueryMetrics.java index 60ffa6dbf51d..c9ccb9d42f2a 100644 --- a/processing/src/main/java/io/druid/query/search/search/SearchQueryMetrics.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryMetrics.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.query.search.search; +package io.druid.query.search; import io.druid.query.QueryMetrics; diff --git a/processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java b/processing/src/main/java/io/druid/query/search/SearchQueryMetricsFactory.java similarity index 97% rename from processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java rename to processing/src/main/java/io/druid/query/search/SearchQueryMetricsFactory.java index ed857573489a..842e289288e2 100644 --- a/processing/src/main/java/io/druid/query/search/search/SearchQueryMetricsFactory.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryMetricsFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.query.search.search; +package io.druid.query.search; /** * Implementations could be injected using 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 3da852a4fbc5..74148d66aea4 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -49,12 +49,6 @@ import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.DimFilter; -import io.druid.query.search.DefaultSearchQueryMetricsFactory; -import io.druid.query.search.SearchHit; -import io.druid.query.search.SearchQuery; -import io.druid.query.search.SearchQueryConfig; -import io.druid.query.search.SearchQueryMetrics; -import io.druid.query.search.SearchQueryMetricsFactory; import javax.annotation.Nullable; import java.nio.ByteBuffer; diff --git a/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java b/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java index b749e4ad74aa..dbb2db70320e 100644 --- a/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java +++ b/processing/src/test/java/io/druid/query/search/DefaultSearchQueryMetricsTest.java @@ -28,10 +28,6 @@ import io.druid.query.QueryRunnerTestHelper; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.ListFilteredDimensionSpec; -import io.druid.query.search.search.DefaultSearchQueryMetrics; -import io.druid.query.search.search.DefaultSearchQueryMetricsFactory; -import io.druid.query.search.search.SearchQuery; -import io.druid.query.search.search.SearchQueryMetrics; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test;