From 3a6452c6d4149e068528317cce9030a397b5ff4d Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 10 Feb 2016 16:06:06 -0800 Subject: [PATCH 01/12] Make QuotableWhiteSpaceSplitter able to take json * Fixes #2435 --- .../content/configuration/indexing-service.md | 2 +- .../indexing/overlord/ForkingTaskRunner.java | 26 ++++++++++++++--- .../overlord/ForkingTaskRunnerTest.java | 29 +++++++++++++++---- 3 files changed, 47 insertions(+), 10 deletions(-) diff --git a/docs/content/configuration/indexing-service.md b/docs/content/configuration/indexing-service.md index a8c48cfd9e6..0fce6272122 100644 --- a/docs/content/configuration/indexing-service.md +++ b/docs/content/configuration/indexing-service.md @@ -254,7 +254,7 @@ Middle managers pass their configurations down to their child peons. The middle |`druid.indexer.runner.compressZnodes`|Indicates whether or not the middle managers should compress Znodes.|true| |`druid.indexer.runner.classpath`|Java classpath for the peon.|System.getProperty("java.class.path")| |`druid.indexer.runner.javaCommand`|Command required to execute java.|java| -|`druid.indexer.runner.javaOpts`|-X Java options to run the peon in its own JVM.|""| +|`druid.indexer.runner.javaOpts`|-X Java options to run the peon in its own JVM. Can be either a string or a json string list. Quotable parameters or parameters with spaces are encouraged to use json string lists|""| |`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper.|524288| |`druid.indexer.runner.startPort`|The port that peons begin running on.|8100| |`druid.indexer.runner.separateIngestionEndpoint`|Use separate server and consequently separate jetty thread pool for ingesting events|false| diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java index 63a1180b741..d64397447e4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java @@ -21,6 +21,9 @@ package io.druid.indexing.overlord; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.CharMatcher; import com.google.common.base.Joiner; @@ -45,6 +48,7 @@ import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.logger.Logger; import com.metamx.emitter.EmittingLogger; import io.druid.concurrent.Execs; import io.druid.guice.annotations.Self; @@ -75,7 +79,6 @@ import java.util.Map; import java.util.Properties; import java.util.UUID; import java.util.concurrent.Callable; -import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; /** @@ -244,7 +247,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer command.add("-cp"); command.add(taskClasspath); - Iterables.addAll(command, new QuotableWhiteSpaceSplitter(config.getJavaOpts())); + Iterables.addAll(command, new QuotableWhiteSpaceSplitter(config.getJavaOpts(), jsonMapper)); // Override task specific javaOpts Object taskJavaOpts = task.getContextValue( @@ -253,7 +256,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer if (taskJavaOpts != null) { Iterables.addAll( command, - new QuotableWhiteSpaceSplitter((String) taskJavaOpts) + new QuotableWhiteSpaceSplitter((String) taskJavaOpts, jsonMapper) ); } @@ -673,16 +676,31 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer */ class QuotableWhiteSpaceSplitter implements Iterable { + private static final Logger LOG = new Logger(QuotableWhiteSpaceSplitter.class); private final String string; + private final ObjectMapper mapper; - public QuotableWhiteSpaceSplitter(String string) + public QuotableWhiteSpaceSplitter(String string, ObjectMapper jsonMapper) { this.string = Preconditions.checkNotNull(string); + this.mapper = jsonMapper; } @Override public Iterator iterator() { + try (JsonParser parser = mapper.getFactory().createParser(string)) { + final JsonToken token = parser.nextToken(); + if (JsonToken.START_ARRAY.equals(token)) { + return mapper.>readValue(string, new TypeReference>() + { + }).iterator(); + } + } + catch (IOException e) { + LOG.debug(e, "Could not parse %s", string); + } + LOG.debug("Not json, hoping it is a good string : %s", string); return Splitter.on( new CharMatcher() { diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/ForkingTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/ForkingTaskRunnerTest.java index 6a1d3c86502..a000f81df0b 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/ForkingTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/ForkingTaskRunnerTest.java @@ -19,14 +19,22 @@ package io.druid.indexing.overlord; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; +import io.druid.jackson.DefaultObjectMapper; import org.junit.Assert; import org.junit.Test; +import java.util.Arrays; +import java.util.List; + public class ForkingTaskRunnerTest { + private static final ObjectMapper mapper = new DefaultObjectMapper(); // This tests the test to make sure the test fails when it should. @Test(expected = AssertionError.class) public void testPatternMatcherFailureForJavaOptions() @@ -43,7 +51,7 @@ public class ForkingTaskRunnerTest @Test public void testPatternMatcherLeavesUnbalancedQuoteJavaOptions() { - Assert.assertEquals("\"", Iterators.get(new QuotableWhiteSpaceSplitter("\"").iterator(), 0)); + Assert.assertEquals("\"", Iterators.get(new QuotableWhiteSpaceSplitter("\"", mapper).iterator(), 0)); } @Test @@ -88,7 +96,7 @@ public class ForkingTaskRunnerTest @Test public void testEmpty() { - Assert.assertTrue(ImmutableList.copyOf(new QuotableWhiteSpaceSplitter("")).isEmpty()); + Assert.assertTrue(ImmutableList.copyOf(new QuotableWhiteSpaceSplitter("", mapper)).isEmpty()); } @Test @@ -97,7 +105,8 @@ public class ForkingTaskRunnerTest Assert.assertEquals( ImmutableList.of("start", "stop"), ImmutableList.copyOf( new QuotableWhiteSpaceSplitter( - "start\t\t\t\t \n\f\r\n \f\f \n\r\f\n\r\t stop" + "start\t\t\t\t \n\f\r\n \f\f \n\r\f\n\r\t stop", + mapper ) ) ); @@ -108,16 +117,26 @@ public class ForkingTaskRunnerTest { Assert.assertTrue( ImmutableList.copyOf( - new QuotableWhiteSpaceSplitter(" \t \t\t\t\t \n\n \f\f \n\f\r\t") + new QuotableWhiteSpaceSplitter(" \t \t\t\t\t \n\n \f\f \n\f\r\t", mapper) ).isEmpty() ); } private static void checkValues(String[] strings) { + + try { + Assert.assertEquals( + ImmutableList.copyOf(strings), + ImmutableList.copyOf(new QuotableWhiteSpaceSplitter(mapper.writeValueAsString(Arrays.asList(strings)), mapper)) + ); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } Assert.assertEquals( ImmutableList.copyOf(strings), - ImmutableList.copyOf(new QuotableWhiteSpaceSplitter(Joiner.on(" ").join(strings))) + ImmutableList.copyOf(new QuotableWhiteSpaceSplitter(Joiner.on(" ").join(strings), mapper)) ); } } From 05a144e39aeecfaefe0bc2d5e8ba6852c1f63731 Mon Sep 17 00:00:00 2001 From: Keuntae Park Date: Mon, 11 Jan 2016 18:03:42 +0900 Subject: [PATCH 02/12] fix crash with filtered aggregator at ingestion time - only for selector filter because extraction filter is not supported as cardinality is not fixed at ingestion time --- .../incremental/OffheapIncrementalIndex.java | 14 +++++++--- .../incremental/OnheapIncrementalIndex.java | 3 +++ .../incremental/IncrementalIndexTest.java | 27 ++++++++++++++++--- 3 files changed, 37 insertions(+), 7 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java index 355cf19417b..f5d4d8248cd 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -151,8 +151,6 @@ public class OffheapIncrementalIndex extends IncrementalIndex selectors = Maps.newHashMap(); aggOffsetInBuffer = new int[metrics.length]; - BufferAggregator[] aggregators = new BufferAggregator[metrics.length]; - for (int i = 0; i < metrics.length; i++) { AggregatorFactory agg = metrics[i]; @@ -167,7 +165,6 @@ public class OffheapIncrementalIndex extends IncrementalIndex new OnheapIncrementalIndex.ObjectCachingColumnSelectorFactory(columnSelectorFactory) ); - aggregators[i] = agg.factorizeBuffered(columnSelectorFactory); if (i == 0) { aggOffsetInBuffer[i] = 0; } else { @@ -177,7 +174,7 @@ public class OffheapIncrementalIndex extends IncrementalIndex aggsTotalSize = aggOffsetInBuffer[metrics.length - 1] + metrics[metrics.length - 1].getMaxIntermediateSize(); - return aggregators; + return new BufferAggregator[metrics.length]; } @Override @@ -203,6 +200,15 @@ public class OffheapIncrementalIndex extends IncrementalIndex bufferOffset = indexAndOffset[1]; aggBuffer = aggBuffers.get(bufferIndex).get(); } else { + rowContainer.set(row); + for (int i = 0; i < metrics.length; i++) { + final AggregatorFactory agg = metrics[i]; + getAggs()[i] = agg.factorizeBuffered( + makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics) + ); + } + rowContainer.set(null); + bufferIndex = aggBuffers.size() - 1; ByteBuffer lastBuffer = aggBuffers.isEmpty() ? null : aggBuffers.get(aggBuffers.size() - 1).get(); int[] lastAggregatorsIndexAndOffset = indexAndOffsets.isEmpty() diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index 9676ee67b32..dcf6c0d2946 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -155,12 +155,15 @@ public class OnheapIncrementalIndex extends IncrementalIndex } else { aggs = new Aggregator[metrics.length]; + rowContainer.set(row); for (int i = 0; i < metrics.length; i++) { final AggregatorFactory agg = metrics[i]; aggs[i] = agg.factorize( selectors.get(agg.getName()) ); } + rowContainer.set(null); + final Integer rowIndex = indexIncrement.getAndIncrement(); concurrentSet(rowIndex, aggs); diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java index 86777202051..0fd115b45c6 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java @@ -29,6 +29,8 @@ import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.CloserRule; +import io.druid.query.aggregation.FilteredAggregatorFactory; +import io.druid.query.filter.SelectorDimFilter; import org.joda.time.DateTime; import org.junit.Rule; import org.junit.Test; @@ -73,7 +75,15 @@ public class IncrementalIndexTest public IncrementalIndex createIndex() { return new OnheapIncrementalIndex( - 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000 + 0, + QueryGranularity.MINUTE, + new AggregatorFactory[]{ + new FilteredAggregatorFactory( + new CountAggregatorFactory("cnt"), + new SelectorDimFilter("billy", "A") + ) + }, + 1000 ); } } @@ -88,7 +98,12 @@ public class IncrementalIndexTest return new OffheapIncrementalIndex( 0L, QueryGranularity.NONE, - new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, + new AggregatorFactory[]{ + new FilteredAggregatorFactory( + new CountAggregatorFactory("cnt"), + new SelectorDimFilter("billy", "A") + ) + }, 1000000, new StupidPool( new Supplier() @@ -104,7 +119,6 @@ public class IncrementalIndexTest } } } - } ); } @@ -153,6 +167,13 @@ public class IncrementalIndexTest ImmutableMap.of("billy", "A", "joe", "B") ) ); + index.add( + new MapBasedInputRow( + new DateTime().minus(1).getMillis(), + Lists.newArrayList("billy", "joe"), + ImmutableMap.of("billy", "C", "joe", "B") + ) + ); index.add( new MapBasedInputRow( new DateTime().minus(1).getMillis(), From 40ade32a1f807d0fd111de3eb437d46b7d22090a Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 10 Feb 2016 21:16:04 -0800 Subject: [PATCH 03/12] Fix dependencies. * Don't put druid****selfcontained.jar at the end of the hadoop isolated classpath * Add `provided` to prevent repeated dependency inclusion in the extension directories --- extensions/hdfs-storage/pom.xml | 7 +++++++ extensions/mysql-metadata-storage/pom.xml | 4 ++++ extensions/postgresql-metadata-storage/pom.xml | 4 ++++ extensions/s3-extensions/pom.xml | 8 ++++++++ .../java/io/druid/indexing/common/task/HadoopTask.java | 2 +- services/src/main/java/io/druid/cli/PullDependencies.java | 4 ++-- 6 files changed, 26 insertions(+), 3 deletions(-) diff --git a/extensions/hdfs-storage/pom.xml b/extensions/hdfs-storage/pom.xml index 4ead6f81ab3..ad5481daa91 100644 --- a/extensions/hdfs-storage/pom.xml +++ b/extensions/hdfs-storage/pom.xml @@ -35,25 +35,30 @@ io.druid druid-api + provided io.druid druid-common ${project.parent.version} + provided net.java.dev.jets3t jets3t + provided org.apache.httpcomponents httpclient + provided org.apache.httpcomponents httpcore + provided org.apache.hadoop @@ -63,10 +68,12 @@ com.metamx emitter + provided commons-io commons-io + provided diff --git a/extensions/mysql-metadata-storage/pom.xml b/extensions/mysql-metadata-storage/pom.xml index 5bdd663de0d..71540fe0edb 100644 --- a/extensions/mysql-metadata-storage/pom.xml +++ b/extensions/mysql-metadata-storage/pom.xml @@ -36,16 +36,19 @@ io.druid druid-api + provided io.druid druid-common ${project.parent.version} + provided io.druid druid-server ${project.parent.version} + provided mysql @@ -55,6 +58,7 @@ org.jdbi jdbi + provided diff --git a/extensions/postgresql-metadata-storage/pom.xml b/extensions/postgresql-metadata-storage/pom.xml index 7291b526a43..7e2a367531d 100644 --- a/extensions/postgresql-metadata-storage/pom.xml +++ b/extensions/postgresql-metadata-storage/pom.xml @@ -36,16 +36,19 @@ io.druid druid-api + provided io.druid druid-common ${project.parent.version} + provided io.druid druid-server ${project.parent.version} + provided org.postgresql @@ -55,6 +58,7 @@ org.jdbi jdbi + provided diff --git a/extensions/s3-extensions/pom.xml b/extensions/s3-extensions/pom.xml index c89c3f1bd98..695fd7f46b9 100644 --- a/extensions/s3-extensions/pom.xml +++ b/extensions/s3-extensions/pom.xml @@ -36,39 +36,47 @@ io.druid druid-api + provided io.druid druid-aws-common ${project.parent.version} + provided net.java.dev.jets3t jets3t + provided org.apache.logging.log4j log4j-1.2-api + provided org.apache.httpcomponents httpclient + provided org.apache.httpcomponents httpcore + provided com.metamx emitter + provided commons-io commons-io + provided diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopTask.java index bdb44ef2334..9b86555a52f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopTask.java @@ -87,7 +87,7 @@ public abstract class HadoopTask extends AbstractTask return false; } final String fName = Paths.get(input.toURI()).getFileName().toString(); - return fName.startsWith("druid") && fName.endsWith(".jar"); + return fName.startsWith("druid") && fName.endsWith(".jar") && !fName.endsWith("selfcontained.jar"); } catch (URISyntaxException e) { throw Throwables.propagate(e); diff --git a/services/src/main/java/io/druid/cli/PullDependencies.java b/services/src/main/java/io/druid/cli/PullDependencies.java index 55c05589f10..2e94702c327 100644 --- a/services/src/main/java/io/druid/cli/PullDependencies.java +++ b/services/src/main/java/io/druid/cli/PullDependencies.java @@ -121,8 +121,8 @@ public class PullDependencies implements Runnable "com.fasterxml.jackson.core", "com.fasterxml.jackson.dataformat", "com.fasterxml.jackson.datatype", - "io.netty", - "org.roaringbitmap" + "org.roaringbitmap", + "net.java.dev.jets3t" ); private TeslaAether aether; From 189376a6f9ce9835798ce6937e963f817800a4ec Mon Sep 17 00:00:00 2001 From: Will Lauer Date: Fri, 5 Feb 2016 15:29:16 -0600 Subject: [PATCH 04/12] Adding optional error bounds to sketch aggs and post-aggs By setting a new optional parameter, `errorBoundsStdDev`, to the number of standard deviations to use when computing error bounds, the return type for both the SketchMergeAggregator and the SketchEstimate PostAggregator can be changed from a simple double (estimate) to a JSON object containing the estimate, expected high bound, expected low bound, and standard devations used when computing bounds (same value as passed in). --- .../theta/SketchEstimatePostAggregator.java | 33 ++++- .../theta/SketchEstimateWithErrorBounds.java | 113 ++++++++++++++++++ .../theta/SketchMergeAggregatorFactory.java | 38 +++++- .../OldSketchBuildAggregatorFactory.java | 2 +- .../OldSketchEstimatePostAggregator.java | 2 +- .../OldSketchMergeAggregatorFactory.java | 2 +- .../theta/SketchAggregationTest.java | 49 ++++++-- .../SketchEstimateWithErrorBoundsTest.java | 43 +++++++ .../sketch_test_data_group_by_query.json | 16 +++ 9 files changed, 271 insertions(+), 27 deletions(-) create mode 100644 extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBounds.java create mode 100644 extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBoundsTest.java diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java index 50baa3cc4fe..00cc7883257 100644 --- a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java @@ -35,15 +35,18 @@ public class SketchEstimatePostAggregator implements PostAggregator private final String name; private final PostAggregator field; + private final Integer errorBoundsStdDev; @JsonCreator public SketchEstimatePostAggregator( @JsonProperty("name") String name, - @JsonProperty("field") PostAggregator field + @JsonProperty("field") PostAggregator field, + @JsonProperty("errorBoundsStdDev") Integer errorBoundsStdDev ) { this.name = Preconditions.checkNotNull(name, "name is null"); this.field = Preconditions.checkNotNull(field, "field is null"); + this.errorBoundsStdDev = errorBoundsStdDev; } @Override @@ -64,7 +67,16 @@ public class SketchEstimatePostAggregator implements PostAggregator public Object compute(Map combinedAggregators) { Sketch sketch = (Sketch) field.compute(combinedAggregators); - return sketch.getEstimate(); + if (errorBoundsStdDev != null) { + SketchEstimateWithErrorBounds result = new SketchEstimateWithErrorBounds( + sketch.getEstimate(), + sketch.getUpperBound(errorBoundsStdDev), + sketch.getLowerBound(errorBoundsStdDev), + errorBoundsStdDev); + return result; + } else { + return sketch.getEstimate(); + } } @Override @@ -80,13 +92,20 @@ public class SketchEstimatePostAggregator implements PostAggregator return field; } + @JsonProperty + public Integer getErrorBoundsStdDev() + { + return errorBoundsStdDev; + } + @Override public String toString() { return "SketchEstimatePostAggregator{" + - "name='" + name + '\'' + - ", field=" + field + - "}"; + "name='" + name + '\'' + + ", field=" + field + + ", errorBoundsStdDev=" + errorBoundsStdDev + + "}"; } @Override @@ -104,6 +123,9 @@ public class SketchEstimatePostAggregator implements PostAggregator if (!name.equals(that.name)) { return false; } + if (errorBoundsStdDev != that.errorBoundsStdDev) { + return false; + } return field.equals(that.field); } @@ -113,6 +135,7 @@ public class SketchEstimatePostAggregator implements PostAggregator { int result = name.hashCode(); result = 31 * result + field.hashCode(); + result = 31 * result + (errorBoundsStdDev != null ? errorBoundsStdDev.hashCode() : 0); return result; } } diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBounds.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBounds.java new file mode 100644 index 00000000000..137c70b4578 --- /dev/null +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBounds.java @@ -0,0 +1,113 @@ +/** + * 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.aggregation.datasketches.theta; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Container class used to return estimates in conjunction with + * estimated error bounds. + */ +public class SketchEstimateWithErrorBounds +{ + final private double estimate; + final private double highBound; + final private double lowBound; + final private int numStdDev; + + @JsonCreator + public SketchEstimateWithErrorBounds( + @JsonProperty("estimate") double estimate, + @JsonProperty("highBound") double highBound, + @JsonProperty("lowBound") double lowBound, + @JsonProperty("numStdDev") int numStdDev + ) + { + this.estimate = estimate; + this.highBound = highBound; + this.lowBound = lowBound; + this.numStdDev = numStdDev; + } + + @JsonProperty + public double getEstimate() + { + return estimate; + } + + @JsonProperty + public double getHighBound() + { + return highBound; + } + + @JsonProperty + public double getLowBound() + { + return lowBound; + } + + @JsonProperty + public int getNumStdDev() + { + return numStdDev; + } + + @Override + public String toString() + { + return "SketchEstimateWithErrorBounds{" + + "estimate=" + Double.toString(estimate) + + ", highBound=" + Double.toString(highBound) + + ", lowBound="+ Double.toString(lowBound) + + ", numStdDev=" + Integer.toString(numStdDev) + + "}"; + } + + @Override + public int hashCode() + { + int result = Double.valueOf(estimate).hashCode(); + result = 31 * result + Double.valueOf(highBound).hashCode(); + result = 31 * result + Double.valueOf(lowBound).hashCode(); + result = 31 * result + Integer.valueOf(numStdDev).hashCode(); + + return result; + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } else if (obj == null || getClass() != obj.getClass()) { + return false; + } + + SketchEstimateWithErrorBounds that = (SketchEstimateWithErrorBounds) obj; + if (estimate != that.estimate || + highBound != that.highBound || + lowBound != that.lowBound || + numStdDev != numStdDev) { + return false; + } + return true; + } +} diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java index 69e869f121f..68383dd629f 100644 --- a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java @@ -35,6 +35,7 @@ public class SketchMergeAggregatorFactory extends SketchAggregatorFactory private final boolean shouldFinalize; private final boolean isInputThetaSketch; + private final Integer errorBoundsStdDev; @JsonCreator public SketchMergeAggregatorFactory( @@ -42,12 +43,14 @@ public class SketchMergeAggregatorFactory extends SketchAggregatorFactory @JsonProperty("fieldName") String fieldName, @JsonProperty("size") Integer size, @JsonProperty("shouldFinalize") Boolean shouldFinalize, - @JsonProperty("isInputThetaSketch") Boolean isInputThetaSketch + @JsonProperty("isInputThetaSketch") Boolean isInputThetaSketch, + @JsonProperty("errorBoundsStdDev") Integer errorBoundsStdDev ) { super(name, fieldName, size, CACHE_TYPE_ID); this.shouldFinalize = (shouldFinalize == null) ? true : shouldFinalize.booleanValue(); this.isInputThetaSketch = (isInputThetaSketch == null) ? false : isInputThetaSketch.booleanValue(); + this.errorBoundsStdDev = errorBoundsStdDev; } @Override @@ -59,7 +62,8 @@ public class SketchMergeAggregatorFactory extends SketchAggregatorFactory fieldName, size, shouldFinalize, - isInputThetaSketch + isInputThetaSketch, + errorBoundsStdDev ) ); } @@ -67,7 +71,7 @@ public class SketchMergeAggregatorFactory extends SketchAggregatorFactory @Override public AggregatorFactory getCombiningFactory() { - return new SketchMergeAggregatorFactory(name, name, size, shouldFinalize, false); + return new SketchMergeAggregatorFactory(name, name, size, shouldFinalize, false, errorBoundsStdDev); } @Override @@ -81,7 +85,8 @@ public class SketchMergeAggregatorFactory extends SketchAggregatorFactory name, Math.max(size, castedOther.size), shouldFinalize, - true + true, + errorBoundsStdDev ); } else { throw new AggregatorFactoryNotMergeableException(this, other); @@ -99,6 +104,12 @@ public class SketchMergeAggregatorFactory extends SketchAggregatorFactory { return isInputThetaSketch; } + + @JsonProperty + public Integer getErrorBoundsStdDev() + { + return errorBoundsStdDev; + } /** * Finalize the computation on sketch object and returns estimate from underlying @@ -112,7 +123,17 @@ public class SketchMergeAggregatorFactory extends SketchAggregatorFactory public Object finalizeComputation(Object object) { if (shouldFinalize) { - return ((Sketch) object).getEstimate(); + Sketch sketch = (Sketch) object; + if (errorBoundsStdDev != null) { + SketchEstimateWithErrorBounds result = new SketchEstimateWithErrorBounds( + sketch.getEstimate(), + sketch.getUpperBound(errorBoundsStdDev), + sketch.getLowerBound(errorBoundsStdDev), + errorBoundsStdDev); + return result; + } else { + return sketch.getEstimate(); + } } else { return object; } @@ -146,6 +167,9 @@ public class SketchMergeAggregatorFactory extends SketchAggregatorFactory if (shouldFinalize != that.shouldFinalize) { return false; } + if (errorBoundsStdDev != that.errorBoundsStdDev) { + return false; + } return isInputThetaSketch == that.isInputThetaSketch; } @@ -156,6 +180,7 @@ public class SketchMergeAggregatorFactory extends SketchAggregatorFactory int result = super.hashCode(); result = 31 * result + (shouldFinalize ? 1 : 0); result = 31 * result + (isInputThetaSketch ? 1 : 0); + result = 31 * result + (errorBoundsStdDev != null ? errorBoundsStdDev.hashCode() : 0); return result; } @@ -166,8 +191,9 @@ public class SketchMergeAggregatorFactory extends SketchAggregatorFactory + "fieldName=" + fieldName + ", name=" + name + ", size=" + size - + ",shouldFinalize=" + shouldFinalize + + ", shouldFinalize=" + shouldFinalize + ", isInputThetaSketch=" + isInputThetaSketch + + ", errorBoundsStdDev=" + errorBoundsStdDev + "}"; } } diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchBuildAggregatorFactory.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchBuildAggregatorFactory.java index b242aebed8d..e916a7bb41c 100644 --- a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchBuildAggregatorFactory.java +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchBuildAggregatorFactory.java @@ -34,6 +34,6 @@ public class OldSketchBuildAggregatorFactory extends SketchMergeAggregatorFactor @JsonProperty("size") Integer size ) { - super(name, fieldName, size, true, false); + super(name, fieldName, size, true, false, null); } } diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchEstimatePostAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchEstimatePostAggregator.java index 5d0cb31b1e9..f372a31a319 100644 --- a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchEstimatePostAggregator.java +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchEstimatePostAggregator.java @@ -34,6 +34,6 @@ public class OldSketchEstimatePostAggregator extends SketchEstimatePostAggregato @JsonProperty("field") PostAggregator field ) { - super(name, field); + super(name, field, null); } } diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchMergeAggregatorFactory.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchMergeAggregatorFactory.java index 9c660180c42..5741736f382 100644 --- a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchMergeAggregatorFactory.java +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldSketchMergeAggregatorFactory.java @@ -35,6 +35,6 @@ public class OldSketchMergeAggregatorFactory extends SketchMergeAggregatorFactor @JsonProperty("shouldFinalize") Boolean shouldFinalize ) { - super(name, fieldName, size, shouldFinalize, true); + super(name, fieldName, size, shouldFinalize, true, null); } } diff --git a/extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java b/extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java index 9fcb46e63ca..97b994e84e0 100644 --- a/extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java +++ b/extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java @@ -28,6 +28,7 @@ import com.metamx.common.guava.Sequences; import com.yahoo.sketches.theta.Sketch; import com.yahoo.sketches.theta.Sketches; import io.druid.data.input.MapBasedRow; +import io.druid.data.input.Row; import io.druid.granularity.QueryGranularity; import io.druid.query.Result; import io.druid.query.aggregation.AggregationTestHelper; @@ -65,7 +66,7 @@ public class SketchAggregationTest @Test public void testSimpleDataIngestAndGpByQuery() throws Exception { - Sequence seq = helper.createIndexAndRunQueryOnSegment( + Sequence seq = helper.createIndexAndRunQueryOnSegment( new File(this.getClass().getClassLoader().getResource("simple_test_data.tsv").getFile()), readFileFromClasspathAsString("simple_test_data_record_parser.json"), readFileFromClasspathAsString("simple_test_data_aggregators.json"), @@ -75,7 +76,7 @@ public class SketchAggregationTest readFileFromClasspathAsString("simple_test_data_group_by_query.json") ); - List results = Sequences.toList(seq, Lists.newArrayList()); + List results = Sequences.toList(seq, Lists.newArrayList()); Assert.assertEquals(1, results.size()); Assert.assertEquals( new MapBasedRow( @@ -123,7 +124,7 @@ public class SketchAggregationTest @Test public void testSketchDataIngestAndGpByQuery() throws Exception { - Sequence seq = helper.createIndexAndRunQueryOnSegment( + Sequence seq = helper.createIndexAndRunQueryOnSegment( new File(SketchAggregationTest.class.getClassLoader().getResource("sketch_test_data.tsv").getFile()), readFileFromClasspathAsString("sketch_test_data_record_parser.json"), readFileFromClasspathAsString("sketch_test_data_aggregators.json"), @@ -133,7 +134,7 @@ public class SketchAggregationTest readFileFromClasspathAsString("sketch_test_data_group_by_query.json") ); - List results = Sequences.toList(seq, Lists.newArrayList()); + List results = Sequences.toList(seq, Lists.newArrayList()); Assert.assertEquals(1, results.size()); Assert.assertEquals( new MapBasedRow( @@ -141,7 +142,11 @@ public class SketchAggregationTest ImmutableMap .builder() .put("sids_sketch_count", 50.0) + .put("sids_sketch_count_with_err", + new SketchEstimateWithErrorBounds(50.0, 50.0, 50.0, 2)) .put("sketchEstimatePostAgg", 50.0) + .put("sketchEstimatePostAggWithErrorBounds", + new SketchEstimateWithErrorBounds(50.0, 50.0, 50.0, 2)) .put("sketchUnionPostAggEstimate", 50.0) .put("sketchIntersectionPostAggEstimate", 50.0) .put("sketchAnotBPostAggEstimate", 0.0) @@ -155,7 +160,7 @@ public class SketchAggregationTest @Test public void testThetaCardinalityOnSimpleColumn() throws Exception { - Sequence seq = helper.createIndexAndRunQueryOnSegment( + Sequence seq = helper.createIndexAndRunQueryOnSegment( new File(SketchAggregationTest.class.getClassLoader().getResource("simple_test_data.tsv").getFile()), readFileFromClasspathAsString("simple_test_data_record_parser2.json"), "[" @@ -170,7 +175,7 @@ public class SketchAggregationTest readFileFromClasspathAsString("simple_test_data_group_by_query.json") ); - List results = Sequences.toList(seq, Lists.newArrayList()); + List results = Sequences.toList(seq, Lists.newArrayList()); Assert.assertEquals(1, results.size()); Assert.assertEquals( new MapBasedRow( @@ -192,9 +197,10 @@ public class SketchAggregationTest @Test public void testSketchMergeAggregatorFactorySerde() throws Exception { - assertAggregatorFactorySerde(new SketchMergeAggregatorFactory("name", "fieldName", 16, null, null)); - assertAggregatorFactorySerde(new SketchMergeAggregatorFactory("name", "fieldName", 16, false, true)); - assertAggregatorFactorySerde(new SketchMergeAggregatorFactory("name", "fieldName", 16, true, false)); + assertAggregatorFactorySerde(new SketchMergeAggregatorFactory("name", "fieldName", 16, null, null, null)); + assertAggregatorFactorySerde(new SketchMergeAggregatorFactory("name", "fieldName", 16, false, true, null)); + assertAggregatorFactorySerde(new SketchMergeAggregatorFactory("name", "fieldName", 16, true, false, null)); + assertAggregatorFactorySerde(new SketchMergeAggregatorFactory("name", "fieldName", 16, true, false, 2)); } @Test @@ -202,14 +208,22 @@ public class SketchAggregationTest { Sketch sketch = Sketches.updateSketchBuilder().build(128); - SketchMergeAggregatorFactory agg = new SketchMergeAggregatorFactory("name", "fieldName", 16, null, null); + SketchMergeAggregatorFactory agg = new SketchMergeAggregatorFactory("name", "fieldName", 16, null, null, null); Assert.assertEquals(0.0, ((Double) agg.finalizeComputation(sketch)).doubleValue(), 0.0001); - agg = new SketchMergeAggregatorFactory("name", "fieldName", 16, true, null); + agg = new SketchMergeAggregatorFactory("name", "fieldName", 16, true, null, null); Assert.assertEquals(0.0, ((Double) agg.finalizeComputation(sketch)).doubleValue(), 0.0001); - agg = new SketchMergeAggregatorFactory("name", "fieldName", 16, false, null); + agg = new SketchMergeAggregatorFactory("name", "fieldName", 16, false, null, null); Assert.assertEquals(sketch, agg.finalizeComputation(sketch)); + + agg = new SketchMergeAggregatorFactory("name", "fieldName", 16, true, null, 2); + SketchEstimateWithErrorBounds est = (SketchEstimateWithErrorBounds) agg.finalizeComputation(sketch); + Assert.assertEquals(0.0, est.getEstimate(), 0.0001); + Assert.assertEquals(0.0, est.getHighBound(), 0.0001); + Assert.assertEquals(0.0, est.getLowBound(), 0.0001); + Assert.assertEquals(2, est.getNumStdDev()); + } private void assertAggregatorFactorySerde(AggregatorFactory agg) throws Exception @@ -229,7 +243,16 @@ public class SketchAggregationTest assertPostAggregatorSerde( new SketchEstimatePostAggregator( "name", - new FieldAccessPostAggregator("name", "fieldName") + new FieldAccessPostAggregator("name", "fieldName"), + null + ) + ); + + assertPostAggregatorSerde( + new SketchEstimatePostAggregator( + "name", + new FieldAccessPostAggregator("name", "fieldName"), + 2 ) ); } diff --git a/extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBoundsTest.java b/extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBoundsTest.java new file mode 100644 index 00000000000..4b3ccb9821a --- /dev/null +++ b/extensions/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchEstimateWithErrorBoundsTest.java @@ -0,0 +1,43 @@ +/** + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.druid.query.aggregation.datasketches.theta; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.jackson.DefaultObjectMapper; + +public class SketchEstimateWithErrorBoundsTest +{ + + @Test + public void testSerde() throws IOException + { + ObjectMapper mapper = new DefaultObjectMapper(); + + SketchEstimateWithErrorBounds est = new SketchEstimateWithErrorBounds(100.0,101.5,98.5,2); + + Assert.assertEquals(est, mapper.readValue( + mapper.writeValueAsString(est), SketchEstimateWithErrorBounds.class)); + } + +} diff --git a/extensions/datasketches/src/test/resources/sketch_test_data_group_by_query.json b/extensions/datasketches/src/test/resources/sketch_test_data_group_by_query.json index d4d908a2f28..61008a767a1 100644 --- a/extensions/datasketches/src/test/resources/sketch_test_data_group_by_query.json +++ b/extensions/datasketches/src/test/resources/sketch_test_data_group_by_query.json @@ -10,6 +10,13 @@ "fieldName": "sids_sketch", "size": 16384 }, + { + "type": "thetaSketch", + "name": "sids_sketch_count_with_err", + "fieldName": "sids_sketch", + "size": 16384, + "errorBoundsStdDev": 2 + }, { "type": "thetaSketch", "name": "non_existing_col_validation", @@ -26,6 +33,15 @@ "fieldName": "sids_sketch_count" } }, + { + "type": "thetaSketchEstimate", + "name": "sketchEstimatePostAggWithErrorBounds", + "errorBoundsStdDev": 2, + "field": { + "type": "fieldAccess", + "fieldName": "sids_sketch_count" + } + }, { "type": "thetaSketchEstimate", "name": "sketchIntersectionPostAggEstimate", From 4e119b7a24db6fd420c57489d8a472a278586968 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Thu, 21 Jan 2016 13:54:37 -0600 Subject: [PATCH 05/12] Adding lookup ref manager and lookup dimension spec impl --- docs/content/querying/dimensionspecs.md | 40 ++- .../query/extraction/NamespacedExtractor.java | 1 + pom.xml | 6 + processing/pom.xml | 5 + .../druid/query/dimension/DimensionSpec.java | 3 +- .../query/dimension/LookupDimensionSpec.java | 235 ++++++++++++++++++ .../dimension/RegexFilteredDimensionSpec.java | 4 - .../query/extraction/LookupExtractionFn.java | 2 +- .../query/extraction/LookupExtractor.java | 7 +- .../extraction/LookupExtractorFactory.java | 50 ++++ .../extraction/LookupReferencesManager.java | 183 ++++++++++++++ .../query/extraction/MapLookupExtractor.java | 14 +- .../dimension/LookupDimensionSpecTest.java | 231 +++++++++++++++++ .../query/extraction/LookupExtractorTest.java | 12 +- .../LookupReferencesManagerTest.java | 172 +++++++++++++ .../extraction/MapLookupExtractorTest.java | 20 +- .../LookupExtractionFnExpectationsTest.java | 8 +- .../extraction/LookupExtractionFnTest.java | 14 +- .../query/groupby/GroupByQueryRunnerTest.java | 30 +-- .../query/search/SearchQueryRunnerTest.java | 2 +- .../druid/query/topn/TopNQueryRunnerTest.java | 27 +- .../io/druid/query/topn/TopNQueryTest.java | 2 +- .../src/main/java/io/druid/cli/CliBroker.java | 3 + .../main/java/io/druid/cli/CliHistorical.java | 3 +- .../src/main/java/io/druid/cli/CliPeon.java | 3 +- .../src/main/java/io/druid/cli/CliRouter.java | 2 + .../java/io/druid/guice/RealtimeModule.java | 2 + 27 files changed, 1022 insertions(+), 59 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java create mode 100644 processing/src/main/java/io/druid/query/extraction/LookupExtractorFactory.java create mode 100644 processing/src/main/java/io/druid/query/extraction/LookupReferencesManager.java create mode 100644 processing/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java create mode 100644 processing/src/test/java/io/druid/query/extraction/LookupReferencesManagerTest.java diff --git a/docs/content/querying/dimensionspecs.md b/docs/content/querying/dimensionspecs.md index 5a65a50856a..1587326ae36 100644 --- a/docs/content/querying/dimensionspecs.md +++ b/docs/content/querying/dimensionspecs.md @@ -252,7 +252,7 @@ It is illegal to set `retainMissingValue = true` and also specify a `replaceMiss A property of `injective` specifies if optimizations can be used which assume there is no combining of multiple names into one. For example: If ABC123 is the only key that maps to SomeCompany, that can be optimized since it is a unique lookup. But if both ABC123 and DEF456 BOTH map to SomeCompany, then that is NOT a unique lookup. Setting this value to true and setting `retainMissingValue` to FALSE (the default) may cause undesired behavior. -A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = false`). +A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = true`). The optimization layer will run on the broker and it will rewrite the extraction filter as clause of selector filters. For instance the following filter @@ -393,3 +393,41 @@ or without setting "locale" (in this case, the current value of the default loca "type" : "lower" } ``` + +### Lookup DimensionSpecs + +Lookup DimensionSpecs can be used to define directly a lookup implementation as dimension spec. +Generally speaking there is two different kind of lookups implementations. +The first kind is passed at the query time like `map` implementation. + +```json +{ + "type":"lookup", + "dimension":"dimensionName", + "outputName":"dimensionOutputName", + "replaceMissingValuesWith":"missing_value", + "retainMissingValue":false, + "lookup":{"type": "map", "map":{"key":"value"}, "isOneToOne":false} +} +``` + +A property of `retainMissingValue` and `replaceMissingValueWith` can be specified at query time to hint how to handle missing values. Setting `replaceMissingValueWith` to `""` has the same effect as setting it to `null` or omitting the property. +Setting `retainMissingValue` to true will use the dimension's original value if it is not found in the lookup. +The default values are `replaceMissingValueWith = null` and `retainMissingValue = false` which causes missing values to be treated as missing. + +It is illegal to set `retainMissingValue = true` and also specify a `replaceMissingValueWith`. + +A property of `injective` specifies if optimizations can be used which assume there is no combining of multiple names into one. For example: If ABC123 is the only key that maps to SomeCompany, that can be optimized since it is a unique lookup. But if both ABC123 and DEF456 BOTH map to SomeCompany, then that is NOT a unique lookup. Setting this value to true and setting `retainMissingValue` to FALSE (the default) may cause undesired behavior. + +A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = true`). + +The second kind where it is not possible to pass at query time due to their size, will be based on an external lookup table or resource that is already registered via configuration file or/and coordinator. + +```json +{ + "type":"lookup" + "dimension":"dimensionName" + "outputName":"dimensionOutputName" + "name":"lookupName" +} +``` diff --git a/extensions/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java b/extensions/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java index 659adb3f8f6..276c80f13a9 100644 --- a/extensions/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java +++ b/extensions/namespace-lookup/src/main/java/io/druid/query/extraction/NamespacedExtractor.java @@ -87,4 +87,5 @@ public class NamespacedExtractor extends LookupExtractor { return reverseExtractionFunction.apply(value); } + } diff --git a/pom.xml b/pom.xml index c5c84cb2fe5..cacd1aad2a5 100644 --- a/pom.xml +++ b/pom.xml @@ -585,6 +585,12 @@ 1.3 test + + pl.pragmatists + JUnitParams + 1.0.4 + test + diff --git a/processing/pom.xml b/processing/pom.xml index d28dce7e5ac..026cfe9a1bf 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -101,6 +101,11 @@ caliper test + + pl.pragmatists + JUnitParams + test + diff --git a/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java index 807da4d2c9b..21b9201d8c1 100644 --- a/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java @@ -31,7 +31,8 @@ import io.druid.segment.DimensionSelector; @JsonSubTypes.Type(name = "default", value = DefaultDimensionSpec.class), @JsonSubTypes.Type(name = "extraction", value = ExtractionDimensionSpec.class), @JsonSubTypes.Type(name = "regexFiltered", value = RegexFilteredDimensionSpec.class), - @JsonSubTypes.Type(name = "listFiltered", value = ListFilteredDimensionSpec.class) + @JsonSubTypes.Type(name = "listFiltered", value = ListFilteredDimensionSpec.class), + @JsonSubTypes.Type(name = "lookup", value = LookupDimensionSpec.class) }) public interface DimensionSpec { diff --git a/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java new file mode 100644 index 00000000000..31b9118c9a3 --- /dev/null +++ b/processing/src/main/java/io/druid/query/dimension/LookupDimensionSpec.java @@ -0,0 +1,235 @@ +/* + * 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.dimension; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.metamx.common.StringUtils; +import io.druid.query.extraction.ExtractionFn; +import io.druid.query.extraction.LookupExtractionFn; +import io.druid.query.extraction.LookupExtractor; +import io.druid.query.extraction.LookupReferencesManager; +import io.druid.query.filter.DimFilterCacheHelper; +import io.druid.segment.DimensionSelector; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; + +public class LookupDimensionSpec implements DimensionSpec +{ + private static final byte CACHE_TYPE_ID = 0x4; + + @JsonProperty + private final String dimension; + + @JsonProperty + private final String outputName; + + @JsonProperty + private final LookupExtractor lookup; + + @JsonProperty + private final boolean retainMissingValue; + + @JsonProperty + private final String replaceMissingValueWith; + + @JsonProperty + private final String name; + + @JsonProperty + private final boolean optimize; + + private final LookupReferencesManager lookupReferencesManager; + + @JsonCreator + public LookupDimensionSpec( + @JsonProperty("dimension") String dimension, + @JsonProperty("outputName") String outputName, + @JsonProperty("lookup") LookupExtractor lookup, + @JsonProperty("retainMissingValue") boolean retainMissingValue, + @JsonProperty("replaceMissingValueWith") String replaceMissingValueWith, + @JsonProperty("name") String name, + @JacksonInject LookupReferencesManager lookupReferencesManager, + @JsonProperty("optimize") Boolean optimize + ) + { + this.retainMissingValue = retainMissingValue; + this.optimize = optimize == null ? true : optimize; + this.replaceMissingValueWith = Strings.emptyToNull(replaceMissingValueWith); + this.dimension = Preconditions.checkNotNull(dimension, "dimension can not be Null"); + this.outputName = Preconditions.checkNotNull(outputName, "outputName can not be Null"); + this.lookupReferencesManager = lookupReferencesManager; + this.name = name; + this.lookup = lookup; + Preconditions.checkArgument( + Strings.isNullOrEmpty(name) ^ (lookup == null), + "name [%s] and lookup [%s] are mutually exclusive please provide either a name or a lookup", name, lookup + ); + + if (!Strings.isNullOrEmpty(name)) { + Preconditions.checkNotNull( + this.lookupReferencesManager, + "The system is not configured to allow for lookups, please read about configuring a lookup manager in the docs" + ); + } + } + + @Override + @JsonProperty + public String getDimension() + { + return dimension; + } + + @Override + @JsonProperty + public String getOutputName() + { + return outputName; + } + + @JsonProperty + @Nullable + public LookupExtractor getLookup() + { + return lookup; + } + + @JsonProperty + @Nullable + public String getName() + { + return name; + } + + @Override + public ExtractionFn getExtractionFn() + { + final LookupExtractor lookupExtractor = Strings.isNullOrEmpty(name) + ? this.lookup + : Preconditions.checkNotNull( + this.lookupReferencesManager.get(name).get(), + "can not find lookup with name [%s]", + name + ); + return new LookupExtractionFn( + lookupExtractor, + retainMissingValue, + replaceMissingValueWith, + lookupExtractor.isOneToOne(), + optimize + ); + + } + + @Override + public DimensionSelector decorate(DimensionSelector selector) + { + return selector; + } + + @Override + public byte[] getCacheKey() + { + byte[] dimensionBytes = StringUtils.toUtf8(dimension); + byte[] dimExtractionFnBytes = Strings.isNullOrEmpty(name) + ? getLookup().getCacheKey() + : StringUtils.toUtf8(name); + byte[] outputNameBytes = StringUtils.toUtf8(outputName); + byte[] replaceWithBytes = StringUtils.toUtf8(Strings.nullToEmpty(replaceMissingValueWith)); + + + return ByteBuffer.allocate(6 + + dimensionBytes.length + + outputNameBytes.length + + dimExtractionFnBytes.length + + replaceWithBytes.length) + .put(CACHE_TYPE_ID) + .put(dimensionBytes) + .put(DimFilterCacheHelper.STRING_SEPARATOR) + .put(outputNameBytes) + .put(DimFilterCacheHelper.STRING_SEPARATOR) + .put(dimExtractionFnBytes) + .put(DimFilterCacheHelper.STRING_SEPARATOR) + .put(replaceWithBytes) + .put(DimFilterCacheHelper.STRING_SEPARATOR) + .put(retainMissingValue == true ? (byte) 1 : (byte) 0) + .array(); + } + + @Override + public boolean preservesOrdering() + { + return getExtractionFn().preservesOrdering(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (!(o instanceof LookupDimensionSpec)) { + return false; + } + + LookupDimensionSpec that = (LookupDimensionSpec) o; + + if (retainMissingValue != that.retainMissingValue) { + return false; + } + if (optimize != that.optimize) { + return false; + } + if (!getDimension().equals(that.getDimension())) { + return false; + } + if (!getOutputName().equals(that.getOutputName())) { + return false; + } + if (getLookup() != null ? !getLookup().equals(that.getLookup()) : that.getLookup() != null) { + return false; + } + if (replaceMissingValueWith != null + ? !replaceMissingValueWith.equals(that.replaceMissingValueWith) + : that.replaceMissingValueWith != null) { + return false; + } + return getName() != null ? getName().equals(that.getName()) : that.getName() == null; + + } + + @Override + public int hashCode() + { + int result = getDimension().hashCode(); + result = 31 * result + getOutputName().hashCode(); + result = 31 * result + (getLookup() != null ? getLookup().hashCode() : 0); + result = 31 * result + (retainMissingValue ? 1 : 0); + result = 31 * result + (replaceMissingValueWith != null ? replaceMissingValueWith.hashCode() : 0); + result = 31 * result + (getName() != null ? getName().hashCode() : 0); + result = 31 * result + (optimize ? 1 : 0); + return result; + } +} diff --git a/processing/src/main/java/io/druid/query/dimension/RegexFilteredDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/RegexFilteredDimensionSpec.java index c092b41d4a1..db3548b76f6 100644 --- a/processing/src/main/java/io/druid/query/dimension/RegexFilteredDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/RegexFilteredDimensionSpec.java @@ -65,10 +65,6 @@ public class RegexFilteredDimensionSpec extends BaseFilteredDimensionSpec return selector; } - if (selector == null) { - return selector; - } - int count = 0; final Map forwardMapping = new HashMap<>(); diff --git a/processing/src/main/java/io/druid/query/extraction/LookupExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/LookupExtractionFn.java index d12b545116e..d099e9e7b9a 100644 --- a/processing/src/main/java/io/druid/query/extraction/LookupExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/LookupExtractionFn.java @@ -60,7 +60,7 @@ public class LookupExtractionFn extends FunctionalExtraction injective ); this.lookup = lookup; - this.optimize = optimize == null ? false : optimize; + this.optimize = optimize == null ? true : optimize; } diff --git a/processing/src/main/java/io/druid/query/extraction/LookupExtractor.java b/processing/src/main/java/io/druid/query/extraction/LookupExtractor.java index 2de937df05e..2aeb31d4046 100644 --- a/processing/src/main/java/io/druid/query/extraction/LookupExtractor.java +++ b/processing/src/main/java/io/druid/query/extraction/LookupExtractor.java @@ -106,6 +106,11 @@ public abstract class LookupExtractor * @return A byte array that can be used to uniquely identify if results of a prior lookup can use the cached values */ - @Nullable public abstract byte[] getCacheKey(); + + // make this abstract again once @drcrallen fix the metmax lookup implementation. + public boolean isOneToOne() + { + return false; + } } diff --git a/processing/src/main/java/io/druid/query/extraction/LookupExtractorFactory.java b/processing/src/main/java/io/druid/query/extraction/LookupExtractorFactory.java new file mode 100644 index 00000000000..c7a9969efda --- /dev/null +++ b/processing/src/main/java/io/druid/query/extraction/LookupExtractorFactory.java @@ -0,0 +1,50 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.extraction; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.base.Supplier; + +/** + * Users of Lookup Extraction need to implement a {@link LookupExtractorFactory} supplier of type {@link LookupExtractor}. + * Such factory will manage the state and life cycle of an given lookup. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +public interface LookupExtractorFactory extends Supplier +{ + /** + *

+ * This method will be called to start the LookupExtractor upon registered + * Calling start multiple times should not lead to any failure and suppose to return true in both cases. + *

+ * + * @return true if start successfully started the {@link LookupExtractor} + */ + public boolean start(); + + /** + *

+ * This method will be called to stop the LookupExtractor upon deletion. + * Calling this method multiple times should not lead to any failure. + *

+ * @return true if successfully closed the {@link LookupExtractor} + */ + public boolean close(); +} diff --git a/processing/src/main/java/io/druid/query/extraction/LookupReferencesManager.java b/processing/src/main/java/io/druid/query/extraction/LookupReferencesManager.java new file mode 100644 index 00000000000..14eaac9bfda --- /dev/null +++ b/processing/src/main/java/io/druid/query/extraction/LookupReferencesManager.java @@ -0,0 +1,183 @@ +/* + * 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.extraction; + + +import com.google.common.collect.Maps; +import com.metamx.common.ISE; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.logger.Logger; +import io.druid.guice.ManageLifecycle; + +import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * This class provide a basic {@link LookupExtractorFactory} references manager. + * It allows basic operations fetching, listing, adding and deleting of {@link LookupExtractor} objects + * It is be used by queries to fetch the lookup reference. + * It is used by Lookup configuration manager to add/remove or list lookups configuration via HTTP or other protocols. + */ + +@ManageLifecycle +public class LookupReferencesManager +{ + private static final Logger LOGGER = new Logger(LookupReferencesManager.class); + private final ConcurrentMap lookupMap = new ConcurrentHashMap(); + private final Object lock = new Object(); + private final AtomicBoolean started = new AtomicBoolean(false); + + @LifecycleStart + public void start() + { + synchronized (lock) { + if (!started.getAndSet(true)) { + LOGGER.info("Started lookup factory references manager"); + } + } + } + + @LifecycleStop + public void stop() + { + synchronized (lock) { + if (started.getAndSet(false)) { + LOGGER.info("Stopped lookup factory references manager"); + for (String lookupName : lookupMap.keySet()) { + remove(lookupName); + } + } + } + } + + /** + * @param lookupName name of the lookupExtractorFactory object + * @param lookupExtractorFactory {@link LookupExtractorFactory} implementation reference. + * + * @return true if the lookup is added otherwise false. + * + * @throws IllegalStateException If the manager is closed or if start of lookup returns false. + */ + public boolean put(String lookupName, final LookupExtractorFactory lookupExtractorFactory) + { + synchronized (lock) { + assertStarted(); + if (lookupMap.containsKey(lookupName)) { + LOGGER.warn("lookup [%s] is not add, another lookup with the same name already exist", lookupName); + return false; + } + if (!lookupExtractorFactory.start()) { + throw new ISE("start method returned false for lookup [%s]", lookupName); + } + return (null == lookupMap.putIfAbsent(lookupName, lookupExtractorFactory)); + } + } + + /** + * @param lookups {@link Map} containing all the lookup as one batch. + * + * @throws IllegalStateException if the manager is closed or if {@link LookupExtractorFactory#start()} returns false + */ + public void put(Map lookups) + { + Map faildExtractorFactoryMap = new HashMap<>(); + synchronized (lock) { + assertStarted(); + for (Map.Entry entry : lookups.entrySet()) { + final String lookupName = entry.getKey(); + final LookupExtractorFactory lookupExtractorFactory = entry.getValue(); + if (lookupMap.containsKey(lookupName)) { + LOGGER.warn("lookup [%s] is not add, another lookup with the same name already exist", lookupName); + continue; + } + if (!lookupExtractorFactory.start()) { + faildExtractorFactoryMap.put(lookupName, lookupExtractorFactory); + continue; + } + lookupMap.put(lookupName, lookupExtractorFactory); + } + if (!faildExtractorFactoryMap.isEmpty()) { + throw new ISE( + "was not able to start the following lookup(s) [%s]", + faildExtractorFactoryMap.keySet().toString() + ); + } + } + } + + /** + * @param lookupName name of {@link LookupExtractorFactory} to delete from the reference registry. + * this function does call the cleaning method {@link LookupExtractorFactory#close()} + * + * @return true only if {@code lookupName} is removed and the lookup correctly stopped + */ + public boolean remove(String lookupName) + { + final LookupExtractorFactory lookupExtractorFactory = lookupMap.remove(lookupName); + if (lookupExtractorFactory != null) { + LOGGER.debug("Removing lookup [%s]", lookupName); + return lookupExtractorFactory.close(); + } + return false; + } + + /** + * @param lookupName key to fetch the reference of the object {@link LookupExtractor} + * + * @return reference of {@link LookupExtractorFactory} that correspond the {@code lookupName} or null if absent + * + * @throws IllegalStateException if the {@link LookupReferencesManager} is closed or did not start yet + */ + @Nullable + public LookupExtractorFactory get(String lookupName) + { + final LookupExtractorFactory lookupExtractorFactory = lookupMap.get(lookupName); + assertStarted(); + return lookupExtractorFactory; + } + + /** + * @return Returns {@link Map} containing a copy of the current state. + * + * @throws ISE if the is is closed or did not start yet. + */ + public Map getAll() + { + assertStarted(); + return Maps.newHashMap(lookupMap); + } + + private void assertStarted() throws ISE + { + if (isClosed()) { + throw new ISE("lookup manager is closed"); + } + } + + public boolean isClosed() + { + return !started.get(); + } +} diff --git a/processing/src/main/java/io/druid/query/extraction/MapLookupExtractor.java b/processing/src/main/java/io/druid/query/extraction/MapLookupExtractor.java index ac2066f50cf..fdd3b5b4613 100644 --- a/processing/src/main/java/io/druid/query/extraction/MapLookupExtractor.java +++ b/processing/src/main/java/io/druid/query/extraction/MapLookupExtractor.java @@ -43,12 +43,16 @@ public class MapLookupExtractor extends LookupExtractor { private final Map map; + private final boolean isOneToOne; + @JsonCreator public MapLookupExtractor( - @JsonProperty("map") Map map + @JsonProperty("map") Map map, + @JsonProperty("isOneToOne") boolean isOneToOne ) { this.map = Preconditions.checkNotNull(map, "map"); + this.isOneToOne = isOneToOne; } @JsonProperty @@ -77,6 +81,13 @@ public class MapLookupExtractor extends LookupExtractor } + @Override + @JsonProperty("isOneToOne") + public boolean isOneToOne() + { + return isOneToOne; + } + @Override public byte[] getCacheKey() { @@ -122,4 +133,5 @@ public class MapLookupExtractor extends LookupExtractor { return map.hashCode(); } + } diff --git a/processing/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java b/processing/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java new file mode 100644 index 00000000000..769a5a82f24 --- /dev/null +++ b/processing/src/test/java/io/druid/query/dimension/LookupDimensionSpecTest.java @@ -0,0 +1,231 @@ +/* + * 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.dimension; + +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.extraction.ExtractionFn; +import io.druid.query.extraction.LookupExtractor; +import io.druid.query.extraction.LookupExtractorFactory; +import io.druid.query.extraction.LookupReferencesManager; +import io.druid.query.extraction.MapLookupExtractor; +import junitparams.JUnitParamsRunner; +import junitparams.Parameters; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Map; + +@RunWith(JUnitParamsRunner.class) +public class LookupDimensionSpecTest +{ + private static final Map STRING_MAP = ImmutableMap.of("key", "value", "key2", "value2"); + private static LookupExtractor MAP_LOOKUP_EXTRACTOR = new MapLookupExtractor( + STRING_MAP, true); + + private static final LookupReferencesManager LOOKUP_REF_MANAGER = EasyMock.createMock(LookupReferencesManager.class); + + static { + EasyMock.expect(LOOKUP_REF_MANAGER.get(EasyMock.eq("lookupName"))).andReturn(new LookupExtractorFactory() + { + @Override + public boolean start() + { + return true; + } + + @Override + public boolean close() + { + return true; + } + + @Override + public LookupExtractor get() + { + return MAP_LOOKUP_EXTRACTOR; + } + }).anyTimes(); + EasyMock.replay(LOOKUP_REF_MANAGER); + } + + private final DimensionSpec lookupDimSpec = new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, + true + ); + + + @Parameters + @Test + public void testSerDesr(DimensionSpec lookupDimSpec) throws IOException + { + ObjectMapper mapper = new DefaultObjectMapper(); + InjectableValues injectableValues = new InjectableValues.Std().addValue( + LookupReferencesManager.class, + LOOKUP_REF_MANAGER + ); + String serLookup = mapper.writeValueAsString(lookupDimSpec); + Assert.assertEquals(lookupDimSpec, mapper.reader(DimensionSpec.class).with(injectableValues).readValue(serLookup)); + } + + private Object[] parametersForTestSerDesr() + { + return new Object[]{ + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, null, null, true), + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "Missing_value", null, null, true), + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, true), + new LookupDimensionSpec("dimName", "outputName", null, false, null, "name", LOOKUP_REF_MANAGER, true) + }; + } + + @Test(expected = Exception.class) + public void testExceptionWhenNameAndLookupNotNull() + { + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "replace", "name", null, true); + } + + @Test(expected = Exception.class) + public void testExceptionWhenNameAndLookupNull() + { + new LookupDimensionSpec("dimName", "outputName", null, false, "replace", "", null, true); + } + + @Test + public void testGetDimension() + { + Assert.assertEquals("dimName", lookupDimSpec.getDimension()); + } + + @Test + public void testGetOutputName() + { + Assert.assertEquals("outputName", lookupDimSpec.getOutputName()); + } + + public Object[] parametersForTestApply() + { + return new Object[]{ + new Object[]{ + new LookupDimensionSpec("dimName", "outputName", null, true, null, "lookupName", LOOKUP_REF_MANAGER, true), + STRING_MAP + }, + new Object[]{ + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, null, null, true), + STRING_MAP + }, + new Object[]{ + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, true), + ImmutableMap.of("not there", "") + }, + new Object[]{ + new LookupDimensionSpec("dimName", "outputName", null, false, null, "lookupName", LOOKUP_REF_MANAGER, true), + ImmutableMap.of("not there", "") + }, + new Object[]{ + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "Missing_value", null, null, + true + ), + ImmutableMap.of("not there", "Missing_value") + }, + new Object[]{ + new LookupDimensionSpec("dimName", "outputName", null, false, "Missing_value", "lookupName", LOOKUP_REF_MANAGER, + true + ), + ImmutableMap.of("not there", "Missing_value") + }, + new Object[]{ + new LookupDimensionSpec("dimName", "outputName", null, true, null, "lookupName", LOOKUP_REF_MANAGER, true), + ImmutableMap.of("not there", "not there") + }, + new Object[]{ + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, "", null, true), + ImmutableMap.of("not there", "not there") + } + + }; + } + + @Test + @Parameters + public void testApply(DimensionSpec dimensionSpec, Map map) + { + for (Map.Entry entry : map.entrySet() + ) { + Assert.assertEquals(Strings.emptyToNull(entry.getValue()), dimensionSpec.getExtractionFn().apply(entry.getKey())); + } + } + + public Object[] parametersForTestGetCacheKey() + { + return new Object[]{ + new Object[]{ + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, true, null, null, null, true), + false + }, + new Object[]{ + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "Missing_value", null, null, + true + ), + false + }, + new Object[]{ + new LookupDimensionSpec("dimName", "outputName2", MAP_LOOKUP_EXTRACTOR, false, null, null, null, true), + false + }, + new Object[]{ + new LookupDimensionSpec("dimName2", "outputName2", MAP_LOOKUP_EXTRACTOR, false, null, null, null, true), + false + }, + new Object[]{ + new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, null, true), + true + }, + new Object[]{ + new LookupDimensionSpec("dimName", "outputName", null, false, null, "name", LOOKUP_REF_MANAGER, true), + false + } + }; + } + + @Test + @Parameters + public void testGetCacheKey(DimensionSpec dimensionSpec, boolean expectedResult) + { + Assert.assertEquals(expectedResult, Arrays.equals(lookupDimSpec.getCacheKey(), dimensionSpec.getCacheKey())); + } + + @Test + public void testPreservesOrdering() + { + Assert.assertFalse(lookupDimSpec.preservesOrdering()); + } + + @Test + public void testIsOneToOne() + { + Assert.assertEquals(lookupDimSpec.getExtractionFn().getExtractionType(), ExtractionFn.ExtractionType.ONE_TO_ONE); + } +} diff --git a/processing/src/test/java/io/druid/query/extraction/LookupExtractorTest.java b/processing/src/test/java/io/druid/query/extraction/LookupExtractorTest.java index 7b08082184a..7fb35119fa6 100644 --- a/processing/src/test/java/io/druid/query/extraction/LookupExtractorTest.java +++ b/processing/src/test/java/io/druid/query/extraction/LookupExtractorTest.java @@ -21,11 +21,14 @@ package io.druid.query.extraction; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import io.druid.jackson.DefaultObjectMapper; import org.junit.Assert; import org.junit.Test; +import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -54,7 +57,14 @@ public class LookupExtractorTest "emptyString", Arrays.asList("") ); - LookupExtractor lookupExtractor = new MapLookupExtractor(EXPECTED_MAP); + LookupExtractor lookupExtractor = new MapLookupExtractor(EXPECTED_MAP, false); + + @Test + public void testSerDes() throws IOException + { + ObjectMapper mapper = new DefaultObjectMapper(); + Assert.assertEquals(lookupExtractor, mapper.reader(LookupExtractor.class).readValue(mapper.writeValueAsBytes(lookupExtractor))); + } @Test public void testApplyAll() diff --git a/processing/src/test/java/io/druid/query/extraction/LookupReferencesManagerTest.java b/processing/src/test/java/io/druid/query/extraction/LookupReferencesManagerTest.java new file mode 100644 index 00000000000..fc7056ad09f --- /dev/null +++ b/processing/src/test/java/io/druid/query/extraction/LookupReferencesManagerTest.java @@ -0,0 +1,172 @@ +/* + * 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.extraction; + +import com.google.common.collect.ImmutableMap; +import com.metamx.common.ISE; +import org.easymock.EasyMock; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +public class LookupReferencesManagerTest +{ + LookupReferencesManager lookupReferencesManager; + + @Before + public void setUp() + { + lookupReferencesManager = new LookupReferencesManager(); + Assert.assertTrue("must be closed before start call", lookupReferencesManager.isClosed()); + lookupReferencesManager.start(); + Assert.assertFalse("must start after start call", lookupReferencesManager.isClosed()); + } + + @After + public void tearDown() + { + lookupReferencesManager.stop(); + Assert.assertTrue("stop call should close it", lookupReferencesManager.isClosed()); + } + + @Test(expected = ISE.class) + public void testGetExceptionWhenClosed() + { + lookupReferencesManager.stop(); + lookupReferencesManager.get("test"); + } + + @Test(expected = ISE.class) + public void testAddExceptionWhenClosed() + { + lookupReferencesManager.stop(); + lookupReferencesManager.put("test", EasyMock.createMock(LookupExtractorFactory.class)); + } + + @Test + public void testPutGetRemove() + { + LookupExtractorFactory lookupExtractorFactory = EasyMock.createMock(LookupExtractorFactory.class); + EasyMock.expect(lookupExtractorFactory.start()).andReturn(true).once(); + EasyMock.expect(lookupExtractorFactory.close()).andReturn(true).once(); + EasyMock.replay(lookupExtractorFactory); + Assert.assertNull(lookupReferencesManager.get("test")); + lookupReferencesManager.put("test", lookupExtractorFactory); + Assert.assertEquals(lookupExtractorFactory, lookupReferencesManager.get("test")); + Assert.assertTrue(lookupReferencesManager.remove("test")); + Assert.assertNull(lookupReferencesManager.get("test")); + } + + @Test + public void testCloseIsCalledAfterStopping() throws IOException + { + LookupExtractorFactory lookupExtractorFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); + EasyMock.expect(lookupExtractorFactory.start()).andReturn(true).once(); + EasyMock.expect(lookupExtractorFactory.close()).andReturn(true).once(); + EasyMock.replay(lookupExtractorFactory); + lookupReferencesManager.put("testMock", lookupExtractorFactory); + lookupReferencesManager.stop(); + EasyMock.verify(lookupExtractorFactory); + } + + @Test + public void testCloseIsCalledAfterRemove() throws IOException + { + LookupExtractorFactory lookupExtractorFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); + EasyMock.expect(lookupExtractorFactory.start()).andReturn(true).once(); + EasyMock.expect(lookupExtractorFactory.close()).andReturn(true).once(); + EasyMock.replay(lookupExtractorFactory); + lookupReferencesManager.put("testMock", lookupExtractorFactory); + lookupReferencesManager.remove("testMock"); + EasyMock.verify(lookupExtractorFactory); + } + + @Test + public void testRemoveInExisting() + { + Assert.assertFalse(lookupReferencesManager.remove("notThere")); + } + + @Test + public void testGetNotThere() + { + Assert.assertNull(lookupReferencesManager.get("notThere")); + } + + @Test + public void testAddingWithSameLookupName() + { + LookupExtractorFactory lookupExtractorFactory = EasyMock.createNiceMock(LookupExtractorFactory.class); + EasyMock.expect(lookupExtractorFactory.start()).andReturn(true).once(); + LookupExtractorFactory lookupExtractorFactory2 = EasyMock.createNiceMock(LookupExtractorFactory.class); + EasyMock.expect(lookupExtractorFactory2.start()).andReturn(true).times(2); + EasyMock.replay(lookupExtractorFactory, lookupExtractorFactory2); + Assert.assertTrue(lookupReferencesManager.put("testName", lookupExtractorFactory)); + Assert.assertFalse(lookupReferencesManager.put("testName", lookupExtractorFactory2)); + ImmutableMap extractorImmutableMap = ImmutableMap.of( + "testName", + lookupExtractorFactory2 + ); + lookupReferencesManager.put(extractorImmutableMap); + Assert.assertEquals(lookupExtractorFactory, lookupReferencesManager.get("testName")); + } + + @Test + public void testAddLookupsThenGetAll() + { + LookupExtractorFactory lookupExtractorFactory = EasyMock.createNiceMock(LookupExtractorFactory.class); + EasyMock.expect(lookupExtractorFactory.start()).andReturn(true).once(); + LookupExtractorFactory lookupExtractorFactory2 = EasyMock.createNiceMock(LookupExtractorFactory.class); + EasyMock.expect(lookupExtractorFactory2.start()).andReturn(true).once(); + EasyMock.replay(lookupExtractorFactory, lookupExtractorFactory2); + ImmutableMap extractorImmutableMap = ImmutableMap.of( + "name1", + lookupExtractorFactory, + "name2", + lookupExtractorFactory2 + ); + lookupReferencesManager.put(extractorImmutableMap); + Assert.assertEquals(extractorImmutableMap, lookupReferencesManager.getAll()); + } + + @Test(expected = ISE.class) + public void testExceptionWhenStartFail() + { + LookupExtractorFactory lookupExtractorFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); + EasyMock.expect(lookupExtractorFactory.start()).andReturn(false).once(); + EasyMock.replay(lookupExtractorFactory); + lookupReferencesManager.put("testMock", lookupExtractorFactory); + } + + @Test(expected = ISE.class) + public void testputAllExceptionWhenStartFail() + { + LookupExtractorFactory lookupExtractorFactory = EasyMock.createStrictMock(LookupExtractorFactory.class); + EasyMock.expect(lookupExtractorFactory.start()).andReturn(false).once(); + ImmutableMap extractorImmutableMap = ImmutableMap.of( + "name1", + lookupExtractorFactory + ); + lookupReferencesManager.put(extractorImmutableMap); + } +} diff --git a/processing/src/test/java/io/druid/query/extraction/MapLookupExtractorTest.java b/processing/src/test/java/io/druid/query/extraction/MapLookupExtractorTest.java index b54a7a84967..507bdcc0865 100644 --- a/processing/src/test/java/io/druid/query/extraction/MapLookupExtractorTest.java +++ b/processing/src/test/java/io/druid/query/extraction/MapLookupExtractorTest.java @@ -32,7 +32,7 @@ import java.util.Map; public class MapLookupExtractorTest { private final Map lookupMap = ImmutableMap.of("foo", "bar", "null", "", "empty String", "", "","empty_string"); - private final MapLookupExtractor fn = new MapLookupExtractor(lookupMap); + private final MapLookupExtractor fn = new MapLookupExtractor(lookupMap, false); @Test public void testUnApply() @@ -62,33 +62,33 @@ public class MapLookupExtractorTest @Test public void testGetCacheKey() throws Exception { - final MapLookupExtractor fn2 = new MapLookupExtractor(ImmutableMap.copyOf(lookupMap)); + final MapLookupExtractor fn2 = new MapLookupExtractor(ImmutableMap.copyOf(lookupMap), false); Assert.assertArrayEquals(fn.getCacheKey(), fn2.getCacheKey()); - final MapLookupExtractor fn3 = new MapLookupExtractor(ImmutableMap.of("foo2", "bar")); + final MapLookupExtractor fn3 = new MapLookupExtractor(ImmutableMap.of("foo2", "bar"), false); Assert.assertFalse(Arrays.equals(fn.getCacheKey(), fn3.getCacheKey())); - final MapLookupExtractor fn4 = new MapLookupExtractor(ImmutableMap.of("foo", "bar2")); + final MapLookupExtractor fn4 = new MapLookupExtractor(ImmutableMap.of("foo", "bar2"), false); Assert.assertFalse(Arrays.equals(fn.getCacheKey(), fn4.getCacheKey())); } @Test public void testEquals() throws Exception { - final MapLookupExtractor fn2 = new MapLookupExtractor(ImmutableMap.copyOf(lookupMap)); + final MapLookupExtractor fn2 = new MapLookupExtractor(ImmutableMap.copyOf(lookupMap), false); Assert.assertEquals(fn, fn2); - final MapLookupExtractor fn3 = new MapLookupExtractor(ImmutableMap.of("foo2", "bar")); + final MapLookupExtractor fn3 = new MapLookupExtractor(ImmutableMap.of("foo2", "bar"), false); Assert.assertNotEquals(fn, fn3); - final MapLookupExtractor fn4 = new MapLookupExtractor(ImmutableMap.of("foo", "bar2")); + final MapLookupExtractor fn4 = new MapLookupExtractor(ImmutableMap.of("foo", "bar2"), false); Assert.assertNotEquals(fn, fn4); } @Test public void testHashCode() throws Exception { - final MapLookupExtractor fn2 = new MapLookupExtractor(ImmutableMap.copyOf(lookupMap)); + final MapLookupExtractor fn2 = new MapLookupExtractor(ImmutableMap.copyOf(lookupMap), false); Assert.assertEquals(fn.hashCode(), fn2.hashCode()); - final MapLookupExtractor fn3 = new MapLookupExtractor(ImmutableMap.of("foo2", "bar")); + final MapLookupExtractor fn3 = new MapLookupExtractor(ImmutableMap.of("foo2", "bar"), false); Assert.assertNotEquals(fn.hashCode(), fn3.hashCode()); - final MapLookupExtractor fn4 = new MapLookupExtractor(ImmutableMap.of("foo", "bar2")); + final MapLookupExtractor fn4 = new MapLookupExtractor(ImmutableMap.of("foo", "bar2"), false); Assert.assertNotEquals(fn.hashCode(), fn4.hashCode()); } } diff --git a/processing/src/test/java/io/druid/query/extraction/extraction/LookupExtractionFnExpectationsTest.java b/processing/src/test/java/io/druid/query/extraction/extraction/LookupExtractionFnExpectationsTest.java index 80113b6cac0..7675445a0cb 100644 --- a/processing/src/test/java/io/druid/query/extraction/extraction/LookupExtractionFnExpectationsTest.java +++ b/processing/src/test/java/io/druid/query/extraction/extraction/LookupExtractionFnExpectationsTest.java @@ -34,7 +34,7 @@ public class LookupExtractionFnExpectationsTest public void testMissingKeyIsNull() { final LookupExtractionFn lookupExtractionFn = new LookupExtractionFn( - new MapLookupExtractor(ImmutableMap.of("foo", "bar")), + new MapLookupExtractor(ImmutableMap.of("foo", "bar"), false), true, null, false, @@ -47,7 +47,7 @@ public class LookupExtractionFnExpectationsTest public void testMissingKeyIsReplaced() { final LookupExtractionFn lookupExtractionFn = new LookupExtractionFn( - new MapLookupExtractor(ImmutableMap.of("foo", "bar")), + new MapLookupExtractor(ImmutableMap.of("foo", "bar"), false), false, "REPLACE", false, @@ -60,7 +60,7 @@ public class LookupExtractionFnExpectationsTest public void testNullKeyIsMappable() { final LookupExtractionFn lookupExtractionFn = new LookupExtractionFn( - new MapLookupExtractor(ImmutableMap.of("", "bar")), + new MapLookupExtractor(ImmutableMap.of("", "bar"), false), false, "REPLACE", false, @@ -73,7 +73,7 @@ public class LookupExtractionFnExpectationsTest public void testNullValue() { final LookupExtractionFn lookupExtractionFn = new LookupExtractionFn( - new MapLookupExtractor(ImmutableMap.of("foo", "")), + new MapLookupExtractor(ImmutableMap.of("foo", ""), false), false, "REPLACE", false, diff --git a/processing/src/test/java/io/druid/query/extraction/extraction/LookupExtractionFnTest.java b/processing/src/test/java/io/druid/query/extraction/extraction/LookupExtractionFnTest.java index 190a4d722f0..46f761edde4 100644 --- a/processing/src/test/java/io/druid/query/extraction/extraction/LookupExtractionFnTest.java +++ b/processing/src/test/java/io/druid/query/extraction/extraction/LookupExtractionFnTest.java @@ -87,14 +87,14 @@ public class LookupExtractionFnTest return; } final LookupExtractionFn lookupExtractionFn1 = new LookupExtractionFn( - new MapLookupExtractor(ImmutableMap.of("foo", "bar")), + new MapLookupExtractor(ImmutableMap.of("foo", "bar"), false), retainMissing, replaceMissing, injective, false ); final LookupExtractionFn lookupExtractionFn2 = new LookupExtractionFn( - new MapLookupExtractor(ImmutableMap.of("foo", "bar")), + new MapLookupExtractor(ImmutableMap.of("foo", "bar"), false), retainMissing, replaceMissing, injective, @@ -103,7 +103,7 @@ public class LookupExtractionFnTest final LookupExtractionFn lookupExtractionFn3 = new LookupExtractionFn( - new MapLookupExtractor(ImmutableMap.of("foo", "bar2")), + new MapLookupExtractor(ImmutableMap.of("foo", "bar2"), false), retainMissing, replaceMissing, injective, @@ -124,7 +124,7 @@ public class LookupExtractionFnTest return; } final LookupExtractionFn lookupExtractionFn = new LookupExtractionFn( - new MapLookupExtractor(ImmutableMap.of("foo", "bar")), + new MapLookupExtractor(ImmutableMap.of("foo", "bar"), false), retainMissing, replaceMissing, injective, @@ -151,7 +151,7 @@ public class LookupExtractionFnTest { if (retainMissing && !Strings.isNullOrEmpty(replaceMissing)) { final LookupExtractionFn lookupExtractionFn = new LookupExtractionFn( - new MapLookupExtractor(ImmutableMap.of("foo", "bar")), + new MapLookupExtractor(ImmutableMap.of("foo", "bar"), false), retainMissing, Strings.emptyToNull(replaceMissing), injective, @@ -173,7 +173,7 @@ public class LookupExtractionFnTest weirdMap.put("foobar", null); final LookupExtractionFn lookupExtractionFn = new LookupExtractionFn( - new MapLookupExtractor(ImmutableMap.of("foo", "bar")), + new MapLookupExtractor(ImmutableMap.of("foo", "bar"), false), retainMissing, replaceMissing, injective, @@ -210,7 +210,7 @@ public class LookupExtractionFnTest Arrays.equals( lookupExtractionFn.getCacheKey(), new LookupExtractionFn( - new MapLookupExtractor(weirdMap), + new MapLookupExtractor(weirdMap, false), lookupExtractionFn.isRetainMissingValue(), lookupExtractionFn.getReplaceMissingValueWith(), lookupExtractionFn.isInjective(), diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 431ab836e24..ff7e230a988 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -268,7 +268,7 @@ public class GroupByQueryRunnerTest .setDimensions( Lists.newArrayList( new ExtractionDimensionSpec( - "quality", "alias", new LookupExtractionFn(new MapLookupExtractor(map), false, null, false, false), null + "quality", "alias", new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, false, false), null ) ) ) @@ -344,7 +344,7 @@ public class GroupByQueryRunnerTest .setDimensions( Lists.newArrayList( new ExtractionDimensionSpec( - "quality", "alias", new LookupExtractionFn(new MapLookupExtractor(map), true, null, false, false), null + "quality", "alias", new LookupExtractionFn(new MapLookupExtractor(map, false), true, null, false, false), null ) ) ) @@ -420,7 +420,7 @@ public class GroupByQueryRunnerTest .setDimensions( Lists.newArrayList( new ExtractionDimensionSpec( - "quality", "alias", new LookupExtractionFn(new MapLookupExtractor(map), true, null, true, false), null + "quality", "alias", new LookupExtractionFn(new MapLookupExtractor(map, false), true, null, true, false), null ) ) ) @@ -498,7 +498,7 @@ public class GroupByQueryRunnerTest new ExtractionDimensionSpec( "quality", "alias", - new LookupExtractionFn(new MapLookupExtractor(map), false, "MISSING", true, false), + new LookupExtractionFn(new MapLookupExtractor(map, false), false, "MISSING", true, false), null ) ) @@ -574,7 +574,7 @@ public class GroupByQueryRunnerTest .setDimensions( Lists.newArrayList( new ExtractionDimensionSpec( - "quality", "alias", new LookupExtractionFn(new MapLookupExtractor(map), false, null, true, false), null + "quality", "alias", new LookupExtractionFn(new MapLookupExtractor(map, false), false, null, true, false), null ) ) ) @@ -3922,7 +3922,8 @@ public class GroupByQueryRunnerTest ImmutableMap.of( "mezzanine", "mezzanine0" - ) + ), + false ), false, null, false, false ), @@ -3996,7 +3997,8 @@ public class GroupByQueryRunnerTest ImmutableMap.of( "mezzanine", "mezzanine0" - ) + ), + false ), false, null, true, false ), @@ -4043,7 +4045,7 @@ public class GroupByQueryRunnerTest extractionMap.put("mezzanine", "automotiveAndBusinessAndNewsAndMezzanine"); extractionMap.put("news", "automotiveAndBusinessAndNewsAndMezzanine"); - MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap); + MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false); LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, false); List dimFilters = Lists.newArrayList( @@ -4115,7 +4117,7 @@ public class GroupByQueryRunnerTest extractionMap.put("technology", "technology0"); extractionMap.put("travel", "travel0"); - MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap); + MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false); LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, false); GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) @@ -4151,7 +4153,7 @@ public class GroupByQueryRunnerTest public void testGroupByWithExtractionDimFilterWhenSearchValueNotInTheMap() { Map extractionMap = new HashMap<>(); - MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap); + MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false); LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, false); GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource) @@ -4192,7 +4194,7 @@ public class GroupByQueryRunnerTest Map extractionMap = new HashMap<>(); extractionMap.put("", "NULLorEMPTY"); - MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap); + MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false); LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, false); GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource) @@ -4244,7 +4246,7 @@ public class GroupByQueryRunnerTest extractionMap.put("technology", "technology0"); extractionMap.put("travel", "travel0"); - MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap); + MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false); LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, "missing", true, false); DimFilter filter = new ExtractionDimFilter("quality","mezzanineANDnews",lookupExtractionFn,null); GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource) @@ -4304,7 +4306,7 @@ public class GroupByQueryRunnerTest extractionMap.put("mezzanine", "newsANDmezzanine"); extractionMap.put("news", "newsANDmezzanine"); - MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap); + MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false); LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, true); GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) @@ -4330,7 +4332,7 @@ public class GroupByQueryRunnerTest Map extractionMap = new HashMap<>(); extractionMap.put("", "EMPTY"); - MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap); + MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false); LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, true); GroupByQuery query = GroupByQuery.builder().setDataSource(QueryRunnerTestHelper.dataSource) diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java index 530aaff87c6..23da1561577 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java @@ -254,7 +254,7 @@ public class SearchQueryRunnerTest final LookupExtractionFn lookupExtractionFn = new LookupExtractionFn( - new MapLookupExtractor(ImmutableMap.of("automotive", automotiveSnowman)), + new MapLookupExtractor(ImmutableMap.of("automotive", automotiveSnowman), false), true, null, true, diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index a77d4e41d36..b1b9d68cc64 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -1629,7 +1629,8 @@ public class TopNQueryRunnerTest "spot", "2spot0", "total_market", "1total_market0", "upfront", "3upfront0" - ) + ), + false ), false, "MISSING", true, false ), @@ -1693,7 +1694,8 @@ public class TopNQueryRunnerTest "spot", "2spot0", "total_market", "1total_market0", "upfront", "3upfront0" - ) + ), + false ), false, "MISSING", false, false ), @@ -1758,7 +1760,8 @@ public class TopNQueryRunnerTest "spot", "2spot0", "total_market", "1total_market0", "upfront", "3upfront0" - ) + ), + false ), true, null, true, false ), @@ -1825,7 +1828,8 @@ public class TopNQueryRunnerTest "total_market0", "upfront", "upfront0" - ) + ), + false ), true, null, false, false ), @@ -1891,7 +1895,8 @@ public class TopNQueryRunnerTest "3total_market", "upfront", "1upfront" - ) + ), + false ), true, null, true, false ), @@ -1957,7 +1962,8 @@ public class TopNQueryRunnerTest "3total_market", "upfront", "1upfront" - ) + ), + false ), true, null, false, false ), @@ -2024,7 +2030,8 @@ public class TopNQueryRunnerTest "3total_market", "upfront", "1upfront" - ) + ), + false ), true, null, true, false ), @@ -3165,7 +3172,7 @@ public class TopNQueryRunnerTest { Map extractionMap = new HashMap<>(); extractionMap.put("spot", "spot0"); - MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap); + MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false); LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, false); TopNQuery query = new TopNQueryBuilder().dataSource(QueryRunnerTestHelper.dataSource) @@ -3215,7 +3222,7 @@ public class TopNQueryRunnerTest Map extractionMap = new HashMap<>(); extractionMap.put("", "NULL"); - MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap); + MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false); LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, false); DimFilter extractionFilter = new ExtractionDimFilter("null_column", "NULL", lookupExtractionFn, null); TopNQueryBuilder topNQueryBuilder = new TopNQueryBuilder() @@ -3284,7 +3291,7 @@ public class TopNQueryRunnerTest Map extractionMap = new HashMap<>(); extractionMap.put("","NULL"); - MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap); + MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false); LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, true); DimFilter extractionFilter = new ExtractionDimFilter("null_column", "NULL", lookupExtractionFn, null); TopNQueryBuilder topNQueryBuilder = new TopNQueryBuilder() diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java index 5c9a43f6e6d..0d82238dc67 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java @@ -93,7 +93,7 @@ public class TopNQueryTest new ExtractionDimensionSpec( marketDimension, marketDimension, - new LookupExtractionFn(new MapLookupExtractor(ImmutableMap.of("foo", "bar")), true, null, false, false), + new LookupExtractionFn(new MapLookupExtractor(ImmutableMap.of("foo", "bar"), false), true, null, false, false), null ) ) diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index f5748e67b67..8af4bd40df2 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -38,10 +38,12 @@ import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; +import io.druid.guice.ManageLifecycle; import io.druid.query.MapQueryToolChestWarehouse; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChestWarehouse; import io.druid.query.RetryQueryRunnerConfig; +import io.druid.query.extraction.LookupReferencesManager; import io.druid.server.ClientInfoResource; import io.druid.server.ClientQuerySegmentWalker; import io.druid.server.QueryResource; @@ -105,6 +107,7 @@ public class CliBroker extends ServerRunnable Jerseys.addResource(binder, ClientInfoResource.class); LifecycleModule.register(binder, QueryResource.class); LifecycleModule.register(binder, DruidBroker.class); + LifecycleModule.register(binder, LookupReferencesManager.class); MetricsModule.register(binder, CacheMonitor.class); diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 56c2025ebec..42ea5b9d8cd 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -35,6 +35,7 @@ import io.druid.guice.LifecycleModule; import io.druid.guice.ManageLifecycle; import io.druid.guice.NodeTypeConfig; import io.druid.query.QuerySegmentWalker; +import io.druid.query.extraction.LookupReferencesManager; import io.druid.server.QueryResource; import io.druid.server.coordination.ServerManager; import io.druid.server.coordination.ZkCoordinator; @@ -83,7 +84,7 @@ public class CliHistorical extends ServerRunnable Jerseys.addResource(binder, QueryResource.class); Jerseys.addResource(binder, HistoricalResource.class); LifecycleModule.register(binder, QueryResource.class); - + LifecycleModule.register(binder, LookupReferencesManager.class); LifecycleModule.register(binder, ZkCoordinator.class); JsonConfigProvider.bind(binder, "druid.historical.cache", CacheConfig.class); diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index ebe67540910..c75ad6d1e0c 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -62,6 +62,7 @@ import io.druid.indexing.worker.executor.ExecutorLifecycle; import io.druid.indexing.worker.executor.ExecutorLifecycleConfig; import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; import io.druid.query.QuerySegmentWalker; +import io.druid.query.extraction.LookupReferencesManager; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentMover; @@ -190,7 +191,7 @@ public class CliPeon extends GuiceRunnable Jerseys.addResource(binder, QueryResource.class); Jerseys.addResource(binder, ChatHandlerResource.class); LifecycleModule.register(binder, QueryResource.class); - + LifecycleModule.register(binder, LookupReferencesManager.class); binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(nodeType)); LifecycleModule.register(binder, Server.class); diff --git a/services/src/main/java/io/druid/cli/CliRouter.java b/services/src/main/java/io/druid/cli/CliRouter.java index 98278657992..b65fa9a2328 100644 --- a/services/src/main/java/io/druid/cli/CliRouter.java +++ b/services/src/main/java/io/druid/cli/CliRouter.java @@ -36,6 +36,7 @@ import io.druid.guice.LifecycleModule; import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.Self; import io.druid.guice.http.JettyHttpClientModule; +import io.druid.query.extraction.LookupReferencesManager; import io.druid.server.initialization.jetty.JettyServerInitializer; import io.druid.server.router.CoordinatorRuleManager; import io.druid.server.router.QueryHostFinder; @@ -91,6 +92,7 @@ public class CliRouter extends ServerRunnable LifecycleModule.register(binder, Server.class); DiscoveryModule.register(binder, Self.class); + LifecycleModule.register(binder, LookupReferencesManager.class); } @Provides diff --git a/services/src/main/java/io/druid/guice/RealtimeModule.java b/services/src/main/java/io/druid/guice/RealtimeModule.java index 1000b94f998..2c716cf5a63 100644 --- a/services/src/main/java/io/druid/guice/RealtimeModule.java +++ b/services/src/main/java/io/druid/guice/RealtimeModule.java @@ -29,6 +29,7 @@ import io.druid.client.cache.CacheConfig; import io.druid.client.coordinator.CoordinatorClient; import io.druid.metadata.MetadataSegmentPublisher; import io.druid.query.QuerySegmentWalker; +import io.druid.query.extraction.LookupReferencesManager; import io.druid.segment.realtime.FireDepartment; import io.druid.segment.realtime.NoopSegmentPublisher; import io.druid.segment.realtime.RealtimeManager; @@ -106,6 +107,7 @@ public class RealtimeModule implements Module Jerseys.addResource(binder, QueryResource.class); Jerseys.addResource(binder, ChatHandlerResource.class); LifecycleModule.register(binder, QueryResource.class); + LifecycleModule.register(binder, LookupReferencesManager.class); LifecycleModule.register(binder, Server.class); } } From 438a4a99708eed151c8a97cae18fdcab14192ea3 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Thu, 11 Feb 2016 10:40:42 -0600 Subject: [PATCH 06/12] fix docs about search query limit --- docs/content/querying/searchquery.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/content/querying/searchquery.md b/docs/content/querying/searchquery.md index 2bc0dedd50b..c11243d7483 100644 --- a/docs/content/querying/searchquery.md +++ b/docs/content/querying/searchquery.md @@ -34,6 +34,7 @@ There are several main parts to a search query: |dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../querying/datasource.html) for more information.|yes| |granularity|Defines the granularity of the query. See [Granularities](../querying/granularities.html).|yes| |filter|See [Filters](../querying/filters.html).|no| +|limit| Defines the maximum number per historical node (parsed as int) of search results to return. |no (default to 1000)| |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| |searchDimensions|The dimensions to run the search over. Excluding this means the search is run over all dimensions.|no| |query|See [SearchQuerySpec](../querying/searchqueryspec.html).|yes| From 29f7758e74199458b8d52bf8ffa9f2d7ed03e506 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 11 Feb 2016 13:51:04 -0800 Subject: [PATCH 07/12] PropertiesModule: Print properties, processors, totalMemory on startup. --- .../main/java/io/druid/guice/PropertiesModule.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/processing/src/main/java/io/druid/guice/PropertiesModule.java b/processing/src/main/java/io/druid/guice/PropertiesModule.java index 51165f64489..dfc1abf11b2 100644 --- a/processing/src/main/java/io/druid/guice/PropertiesModule.java +++ b/processing/src/main/java/io/druid/guice/PropertiesModule.java @@ -21,6 +21,7 @@ package io.druid.guice; import com.google.common.base.Charsets; import com.google.common.base.Throwables; +import com.google.common.collect.Ordering; import com.google.inject.Binder; import com.google.inject.Module; import com.metamx.common.guava.CloseQuietly; @@ -86,6 +87,16 @@ public class PropertiesModule implements Module } } + log.info( + "Loaded properties into JVM with processors[%,d], memory[%,d].", + Runtime.getRuntime().availableProcessors(), + Runtime.getRuntime().totalMemory() + ); + + for (String propertyName : Ordering.natural().sortedCopy(props.stringPropertyNames())) { + log.info("* %s: %s", propertyName, props.getProperty(propertyName)); + } + binder.bind(Properties.class).toInstance(props); } } From f277a54a5cae71ed5f5685b157e0141015bc5f05 Mon Sep 17 00:00:00 2001 From: turu Date: Thu, 11 Feb 2016 23:46:24 +0100 Subject: [PATCH 08/12] removed unsafe heuristics from hll compareTo and provided unit test for regression --- .../hyperloglog/HyperLogLogCollector.java | 17 +------- .../hyperloglog/HyperLogLogCollectorTest.java | 39 +++++++++++++++++++ 2 files changed, 40 insertions(+), 16 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java index 37166e938f9..4803b2ca765 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollector.java @@ -681,21 +681,6 @@ public abstract class HyperLogLogCollector implements Comparable Date: Fri, 29 Jan 2016 08:52:10 +0900 Subject: [PATCH 09/12] Support dimension spec for select query --- docs/content/querying/select-query.md | 2 +- .../src/main/java/io/druid/query/Druids.java | 14 +- .../query/dimension/DefaultDimensionSpec.java | 29 +++- .../dimension/ExtractionDimensionSpec.java | 5 + .../io/druid/query/select/PagingSpec.java | 30 ++++ .../io/druid/query/select/SelectQuery.java | 7 +- .../druid/query/select/SelectQueryEngine.java | 12 +- .../select/SelectQueryQueryToolChest.java | 14 +- .../query/select/SelectQueryRunnerTest.java | 157 ++++++++++++++++-- .../query/select/SelectQuerySpecTest.java | 82 +++++++++ 10 files changed, 317 insertions(+), 35 deletions(-) create mode 100644 processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java diff --git a/docs/content/querying/select-query.md b/docs/content/querying/select-query.md index 9bc5b8b2e06..75eaf310eaa 100644 --- a/docs/content/querying/select-query.md +++ b/docs/content/querying/select-query.md @@ -28,7 +28,7 @@ There are several main parts to a select query: |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| |descending|Whether to make descending ordered result. Default is `false`(ascending). When this is `true`, page identifier and offsets will be negative value.|no| |filter|See [Filters](../querying/filters.html)|no| -|dimensions|A String array of dimensions to select. If left empty, all dimensions are returned.|no| +|dimensions|A JSON list of dimensions to select; or see [DimensionSpec](../querying/dimensionspecs.html) for ways to extract dimensions. If left empty, all dimensions are returned.|no| |metrics|A String array of metrics to select. If left empty, all metrics are returned.|no| |pagingSpec|A JSON object indicating offsets into different scanned segments. Query results will return a `pagingIdentifiers` value that can be reused in the next query for pagination.|yes| |context|An additional JSON Object which can be used to specify certain flags.|no| diff --git a/processing/src/main/java/io/druid/query/Druids.java b/processing/src/main/java/io/druid/query/Druids.java index a971a7cda52..fe6c096160e 100644 --- a/processing/src/main/java/io/druid/query/Druids.java +++ b/processing/src/main/java/io/druid/query/Druids.java @@ -1077,7 +1077,7 @@ public class Druids private Map context; private DimFilter dimFilter; private QueryGranularity granularity; - private List dimensions; + private List dimensions; private List metrics; private PagingSpec pagingSpec; @@ -1101,9 +1101,7 @@ public class Druids descending, dimFilter, granularity, - dimensions, - metrics, - pagingSpec, + dimensions, metrics, pagingSpec, context ); } @@ -1192,12 +1190,18 @@ public class Druids return this; } - public SelectQueryBuilder dimensions(List d) + public SelectQueryBuilder dimensionSpecs(List d) { dimensions = d; return this; } + public SelectQueryBuilder dimensions(List d) + { + dimensions = DefaultDimensionSpec.toSpec(d); + return this; + } + public SelectQueryBuilder metrics(List m) { metrics = m; diff --git a/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java index 86ecfd38f96..4f1b6447db6 100644 --- a/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java @@ -21,16 +21,42 @@ package io.druid.query.dimension; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import com.metamx.common.StringUtils; import io.druid.query.extraction.ExtractionFn; import io.druid.segment.DimensionSelector; import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; /** */ public class DefaultDimensionSpec implements DimensionSpec { + public static List toSpec(String... dimensionNames) + { + return toSpec(Arrays.asList(dimensionNames)); + } + + public static List toSpec(Iterable dimensionNames) + { + return Lists.newArrayList( + Iterables.transform( + dimensionNames, new Function() + { + @Override + public DimensionSpec apply(String input) + { + return new DefaultDimensionSpec(input, input); + } + } + ) + ); + } + private static final byte CACHE_TYPE_ID = 0x0; private final String dimension; private final String outputName; @@ -103,7 +129,8 @@ public class DefaultDimensionSpec implements DimensionSpec public boolean equals(Object o) { if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + // LegacyDimensionSpec can be equal to DefaultDimensionSpec + if (!(o instanceof DefaultDimensionSpec)) return false; DefaultDimensionSpec that = (DefaultDimensionSpec) o; diff --git a/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java index ca30b89b0d4..802f3776b91 100644 --- a/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java @@ -57,6 +57,11 @@ public class ExtractionDimensionSpec implements DimensionSpec this.outputName = outputName == null ? dimension : outputName; } + public ExtractionDimensionSpec(String dimension, String outputName, ExtractionFn extractionFn) + { + this(dimension, outputName, extractionFn, null); + } + @Override @JsonProperty public String getDimension() diff --git a/processing/src/main/java/io/druid/query/select/PagingSpec.java b/processing/src/main/java/io/druid/query/select/PagingSpec.java index 86a61e8f5d8..38d5f141240 100644 --- a/processing/src/main/java/io/druid/query/select/PagingSpec.java +++ b/processing/src/main/java/io/druid/query/select/PagingSpec.java @@ -90,6 +90,36 @@ public class PagingSpec return queryCacheKey.array(); } + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (!(o instanceof PagingSpec)) { + return false; + } + + PagingSpec that = (PagingSpec) o; + + if (threshold != that.threshold) { + return false; + } + if (!pagingIdentifiers.equals(that.pagingIdentifiers)) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = pagingIdentifiers.hashCode(); + result = 31 * result + threshold; + return result; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/select/SelectQuery.java b/processing/src/main/java/io/druid/query/select/SelectQuery.java index 05603c6afd7..2e39c0f547e 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQuery.java +++ b/processing/src/main/java/io/druid/query/select/SelectQuery.java @@ -28,6 +28,7 @@ import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.Query; import io.druid.query.Result; +import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.DimFilter; import io.druid.query.spec.QuerySegmentSpec; @@ -41,7 +42,7 @@ public class SelectQuery extends BaseQuery> { private final DimFilter dimFilter; private final QueryGranularity granularity; - private final List dimensions; + private final List dimensions; private final List metrics; private final PagingSpec pagingSpec; @@ -52,7 +53,7 @@ public class SelectQuery extends BaseQuery> @JsonProperty("descending") boolean descending, @JsonProperty("filter") DimFilter dimFilter, @JsonProperty("granularity") QueryGranularity granularity, - @JsonProperty("dimensions") List dimensions, + @JsonProperty("dimensions") List dimensions, @JsonProperty("metrics") List metrics, @JsonProperty("pagingSpec") PagingSpec pagingSpec, @JsonProperty("context") Map context @@ -104,7 +105,7 @@ public class SelectQuery extends BaseQuery> } @JsonProperty - public List getDimensions() + public List getDimensions() { return dimensions; } diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java index 75e5ac944ad..d6f1bee5f3e 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java @@ -27,6 +27,7 @@ import com.metamx.common.guava.Sequence; import io.druid.query.QueryRunnerHelper; import io.druid.query.Result; import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.DimensionSpec; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; import io.druid.segment.LongColumnSelector; @@ -55,9 +56,9 @@ public class SelectQueryEngine ); } - final Iterable dims; + final Iterable dims; if (query.getDimensions() == null || query.getDimensions().isEmpty()) { - dims = adapter.getAvailableDimensions(); + dims = DefaultDimensionSpec.toSpec(adapter.getAvailableDimensions()); } else { dims = query.getDimensions(); } @@ -89,10 +90,9 @@ public class SelectQueryEngine final LongColumnSelector timestampColumnSelector = cursor.makeLongColumnSelector(Column.TIME_COLUMN_NAME); final Map dimSelectors = Maps.newHashMap(); - for (String dim : dims) { - // switching to using DimensionSpec for select would allow the use of extractionFn here. - final DimensionSelector dimSelector = cursor.makeDimensionSelector(new DefaultDimensionSpec(dim, dim)); - dimSelectors.put(dim, dimSelector); + for (DimensionSpec dim : dims) { + final DimensionSelector dimSelector = cursor.makeDimensionSelector(dim); + dimSelectors.put(dim.getOutputName(), dimSelector); } final Map metSelectors = Maps.newHashMap(); 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 3814fa10427..eba425bd62b 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -40,11 +40,13 @@ import io.druid.query.Result; import io.druid.query.ResultGranularTimestampComparator; import io.druid.query.ResultMergeQueryRunner; import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.DimFilter; import org.joda.time.DateTime; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -138,16 +140,16 @@ public class SelectQueryQueryToolChest extends QueryToolChest dimensions = Sets.newTreeSet(); - if (query.getDimensions() != null) { - dimensions.addAll(query.getDimensions()); + List dimensionSpecs = query.getDimensions(); + if (dimensionSpecs == null) { + dimensionSpecs = Collections.emptyList(); } - final byte[][] dimensionsBytes = new byte[dimensions.size()][]; + final byte[][] dimensionsBytes = new byte[dimensionSpecs.size()][]; int dimensionsBytesSize = 0; int index = 0; - for (String dimension : dimensions) { - dimensionsBytes[index] = StringUtils.toUtf8(dimension); + for (DimensionSpec dimension : dimensionSpecs) { + dimensionsBytes[index] = dimension.getCacheKey(); dimensionsBytesSize += dimensionsBytes[index].length; ++index; } diff --git a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java index bbf3fa2b2c5..3fc958063c8 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -32,6 +32,11 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; import io.druid.query.TableDataSource; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.dimension.ExtractionDimensionSpec; +import io.druid.query.extraction.LookupExtractionFn; +import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.filter.AndDimFilter; import io.druid.query.filter.DimFilter; import io.druid.query.filter.SelectorDimFilter; @@ -130,7 +135,7 @@ public class SelectQueryRunnerTest descending, null, QueryRunnerTestHelper.allGran, - Arrays.asList(), + DefaultDimensionSpec.toSpec(Arrays.asList()), Arrays.asList(), new PagingSpec(null, 3), null @@ -150,6 +155,134 @@ public class SelectQueryRunnerTest verify(expectedResults, results); } + @Test + public void testFullOnSelectWithDimensionSpec() + { + Map map = new HashMap<>(); + map.put("automotive", "automotive0"); + map.put("business", "business0"); + map.put("entertainment", "entertainment0"); + map.put("health", "health0"); + map.put("mezzanine", "mezzanine0"); + map.put("news", "news0"); + map.put("premium", "premium0"); + map.put("technology", "technology0"); + map.put("travel", "travel0"); + + SelectQuery query = new SelectQuery( + new TableDataSource(QueryRunnerTestHelper.dataSource), + QueryRunnerTestHelper.fullOnInterval, + descending, + null, + QueryRunnerTestHelper.allGran, + Arrays.asList( + new DefaultDimensionSpec(QueryRunnerTestHelper.marketDimension, "mar"), + new ExtractionDimensionSpec( + QueryRunnerTestHelper.qualityDimension, + "qual", + new LookupExtractionFn(new MapLookupExtractor(map, true), false, null, true, false) + ), + new DefaultDimensionSpec(QueryRunnerTestHelper.placementDimension, "place") + ), Lists.newArrayList(), new PagingSpec(null, 3), + null + ); + HashMap context = new HashMap(); + Iterable> results = Sequences.toList( + runner.run(query, context), + Lists.>newArrayList() + ); + + List> expectedResultsAsc = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SelectResultValue( + ImmutableMap.of(QueryRunnerTestHelper.segmentId, 2), + Arrays.asList( + new EventHolder( + QueryRunnerTestHelper.segmentId, + 0, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) + .put("mar", "spot") + .put("qual", "automotive0") + .put("place", "preferred") + .put(QueryRunnerTestHelper.indexMetric, 100.000000F) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + 1, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) + .put("mar", "spot") + .put("qual", "business0") + .put("place", "preferred") + .put(QueryRunnerTestHelper.indexMetric, 100.000000F) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + 2, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z")) + .put("mar", "spot") + .put("qual", "entertainment0") + .put("place", "preferred") + .put(QueryRunnerTestHelper.indexMetric, 100.000000F) + .build() + ) + ) + ) + ) + ); + + List> expectedResultsDsc = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SelectResultValue( + ImmutableMap.of(QueryRunnerTestHelper.segmentId, -3), + Arrays.asList( + new EventHolder( + QueryRunnerTestHelper.segmentId, + -1, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-04-15T00:00:00.000Z")) + .put("mar", "upfront") + .put("qual", "premium0") + .put("place", "preferred") + .put(QueryRunnerTestHelper.indexMetric, 780.27197265625F) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + -2, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-04-15T00:00:00.000Z")) + .put("mar", "upfront") + .put("qual", "mezzanine0") + .put("place", "preferred") + .put(QueryRunnerTestHelper.indexMetric, 962.731201171875F) + .build() + ), + new EventHolder( + QueryRunnerTestHelper.segmentId, + -3, + new ImmutableMap.Builder() + .put(EventHolder.timestampKey, new DateTime("2011-04-15T00:00:00.000Z")) + .put("mar", "total_market") + .put("qual", "premium0") + .put("place", "preferred") + .put(QueryRunnerTestHelper.indexMetric, 1029.0570068359375F) + .build() + ) + ) + ) + ) + ); + + verify(descending ? expectedResultsDsc : expectedResultsAsc, results); + } + @Test public void testSelectWithDimsAndMets() { @@ -159,7 +292,7 @@ public class SelectQueryRunnerTest descending, null, QueryRunnerTestHelper.allGran, - Arrays.asList(QueryRunnerTestHelper.marketDimension), + DefaultDimensionSpec.toSpec(Arrays.asList(QueryRunnerTestHelper.marketDimension)), Arrays.asList(QueryRunnerTestHelper.indexMetric), new PagingSpec(null, 3), null @@ -198,7 +331,7 @@ public class SelectQueryRunnerTest descending, null, QueryRunnerTestHelper.allGran, - Arrays.asList(QueryRunnerTestHelper.qualityDimension), + DefaultDimensionSpec.toSpec(Arrays.asList(QueryRunnerTestHelper.qualityDimension)), Arrays.asList(QueryRunnerTestHelper.indexMetric), new PagingSpec(toPagingIdentifier(3, descending), 3), null @@ -236,7 +369,7 @@ public class SelectQueryRunnerTest descending, new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot"), QueryRunnerTestHelper.dayGran, - Lists.newArrayList(QueryRunnerTestHelper.qualityDimension), + DefaultDimensionSpec.toSpec(Lists.newArrayList(QueryRunnerTestHelper.qualityDimension)), Lists.newArrayList(QueryRunnerTestHelper.indexMetric), new PagingSpec(toPagingIdentifier(param[0], descending), param[1]), null @@ -299,15 +432,13 @@ public class SelectQueryRunnerTest I_0112_0114, descending, new AndDimFilter( - Arrays.asList( - new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot"), - new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "foo") - ) - ), + Arrays.asList( + new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot"), + new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "foo") + ) + ), QueryRunnerTestHelper.allGran, - Lists.newArrayList(), - Lists.newArrayList(), - new PagingSpec(null, 3), + DefaultDimensionSpec.toSpec(Lists.newArrayList()), Lists.newArrayList(), new PagingSpec(null, 3), null ); @@ -338,7 +469,7 @@ public class SelectQueryRunnerTest descending, null, QueryRunnerTestHelper.allGran, - Lists.newArrayList("foo"), + DefaultDimensionSpec.toSpec(Lists.newArrayList("foo")), Lists.newArrayList("foo2"), new PagingSpec(null, 3), null diff --git a/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java b/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java new file mode 100644 index 00000000000..d56f51b66ee --- /dev/null +++ b/processing/src/test/java/io/druid/query/select/SelectQuerySpecTest.java @@ -0,0 +1,82 @@ +/* + * 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.jackson.DefaultObjectMapper; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.TableDataSource; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.spec.LegacySegmentSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; + +/** + */ +public class SelectQuerySpecTest +{ + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + @Test + public void testSerializationLegacyString() throws Exception + { + String legacy = + "{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"}," + + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]}," + + "\"descending\":true," + + "\"filter\":null," + + "\"granularity\":{\"type\":\"all\"}," + + "\"dimensions\":[\"market\",\"quality\"]," + + "\"metrics\":[\"index\"]," + + "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3}," + + "\"context\":null}"; + + String current = + "{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"}," + + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]}," + + "\"descending\":true," + + "\"filter\":null," + + "\"granularity\":{\"type\":\"all\"}," + + "\"dimensions\":[{\"type\":\"default\",\"dimension\":\"market\",\"outputName\":\"market\"},{\"type\":\"default\",\"dimension\":\"quality\",\"outputName\":\"quality\"}]," + + "\"metrics\":[\"index\"]," + + "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3}," + + "\"context\":null}"; + + SelectQuery query = new SelectQuery( + new TableDataSource(QueryRunnerTestHelper.dataSource), + new LegacySegmentSpec(new Interval("2011-01-12/2011-01-14")), + true, + null, + QueryRunnerTestHelper.allGran, + DefaultDimensionSpec.toSpec(Arrays.asList("market", "quality")), + Arrays.asList("index"), + new PagingSpec(null, 3), + null + ); + + String actual = jsonMapper.writeValueAsString(query); + Assert.assertEquals(current, actual); + Assert.assertEquals(query, jsonMapper.readValue(actual, SelectQuery.class)); + Assert.assertEquals(query, jsonMapper.readValue(legacy, SelectQuery.class)); + } +} From dd2375477ace5517500d91079f536e3a4a113be4 Mon Sep 17 00:00:00 2001 From: "navis.ryu" Date: Wed, 6 Jan 2016 13:32:38 +0900 Subject: [PATCH 10/12] Support min/max values for metadata query (#2208) --- docs/content/design/broker.md | 2 +- docs/content/design/index.md | 2 +- docs/content/querying/segmentmetadataquery.md | 9 +- .../druid/query/metadata/SegmentAnalyzer.java | 126 ++++++++++++++-- .../metadata/metadata/ColumnAnalysis.java | 61 ++++++-- .../metadata/SegmentMetadataQuery.java | 25 ++- .../segment/QueryableIndexStorageAdapter.java | 23 +++ .../java/io/druid/segment/StorageAdapter.java | 2 + .../segment/incremental/IncrementalIndex.java | 17 +++ .../IncrementalIndexStorageAdapter.java | 14 ++ .../incremental/OnheapIncrementalIndex.java | 16 ++ ...egmentMetadataQueryQueryToolChestTest.java | 4 +- .../metadata/SegmentMetadataQueryTest.java | 101 +++++++++++-- .../metadata/metadata/ColumnAnalysisTest.java | 142 +++++++++++++----- publications/whitepaper/druid.tex | 4 +- 15 files changed, 463 insertions(+), 85 deletions(-) diff --git a/docs/content/design/broker.md b/docs/content/design/broker.md index 3f40b19e8e0..1510030fd60 100644 --- a/docs/content/design/broker.md +++ b/docs/content/design/broker.md @@ -25,7 +25,7 @@ To determine which nodes to forward queries to, the Broker node first builds a v Caching ------- -Broker nodes employ a cache with a LRU cache invalidation strategy. The broker cache stores per segment results. The cache can be local to each broker node or shared across multiple nodes using an external distributed cache such as [memcached](http://memcached.org/). Each time a broker node receives a query, it first maps the query to a set of segments. A subset of these segment results may already exist in the cache and the results can be directly pulled from the cache. For any segment results that do not exist in the cache, the broker node will forward the query to the +Broker nodes employ a cache with a LRU cache invalidation strategy. The broker cache stores per-segment results. The cache can be local to each broker node or shared across multiple nodes using an external distributed cache such as [memcached](http://memcached.org/). Each time a broker node receives a query, it first maps the query to a set of segments. A subset of these segment results may already exist in the cache and the results can be directly pulled from the cache. For any segment results that do not exist in the cache, the broker node will forward the query to the historical nodes. Once the historical nodes return their results, the broker will store those results in the cache. Real-time segments are never cached and hence requests for real-time data will always be forwarded to real-time nodes. Real-time data is perpetually changing and caching the results would be unreliable. HTTP Endpoints diff --git a/docs/content/design/index.md b/docs/content/design/index.md index cb36b5339d3..fb48c6dcf6b 100644 --- a/docs/content/design/index.md +++ b/docs/content/design/index.md @@ -90,7 +90,7 @@ Druid is a column store, which means each individual column is stored separately in that query, and Druid is pretty good about only scanning exactly what it needs for a query. Different columns can also employ different compression methods. Different columns can also have different indexes associated with them. -Druid indexes data on a per shard (segment) level. +Druid indexes data on a per-shard (segment) level. ## Loading the Data diff --git a/docs/content/querying/segmentmetadataquery.md b/docs/content/querying/segmentmetadataquery.md index 469da4b47ec..80a8956a604 100644 --- a/docs/content/querying/segmentmetadataquery.md +++ b/docs/content/querying/segmentmetadataquery.md @@ -2,9 +2,10 @@ layout: doc_page --- # Segment Metadata Queries -Segment metadata queries return per segment information about: +Segment metadata queries return per-segment information about: * Cardinality of all columns in the segment +* Min/max values of string type columns in the segment * Estimated byte size for the segment columns if they were stored in a flat format * Number of rows stored inside the segment * Interval the segment covers @@ -103,13 +104,17 @@ This is a list of properties that determines the amount of information returned By default, all analysis types will be used. If a property is not needed, omitting it from this list will result in a more efficient query. -There are four types of column analyses: +There are five types of column analyses: #### cardinality * `cardinality` in the result will return the estimated floor of cardinality for each column. Only relevant for dimension columns. +#### minmax + +* Estimated min/max values for each column. Only relevant for dimension columns. + #### size * `size` in the result will contain the estimated total segment byte size as if the data were stored in text format diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java index e83edf0a373..1f788c5be8e 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java @@ -21,14 +21,21 @@ package io.druid.query.metadata; import com.google.common.base.Function; import com.google.common.base.Preconditions; +import com.google.common.base.Strings; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.primitives.Longs; import com.metamx.common.StringUtils; +import com.metamx.common.guava.Accumulator; +import com.metamx.common.guava.Sequence; import com.metamx.common.logger.Logger; +import io.druid.granularity.QueryGranularity; +import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.metadata.metadata.ColumnAnalysis; import io.druid.query.metadata.metadata.SegmentMetadataQuery; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; import io.druid.segment.QueryableIndex; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; @@ -38,8 +45,10 @@ import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ComplexColumn; import io.druid.segment.column.ValueType; +import io.druid.segment.data.IndexedInts; import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.ComplexMetrics; +import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.EnumSet; @@ -104,7 +113,11 @@ public class SegmentAnalyzer analysis = analyzeNumericColumn(capabilities, length, NUM_BYTES_IN_TEXT_FLOAT); break; case STRING: - analysis = analyzeStringColumn(capabilities, column, storageAdapter.getDimensionCardinality(columnName)); + if (index != null) { + analysis = analyzeStringColumn(capabilities, column); + } else { + analysis = analyzeStringColumn(capabilities, storageAdapter, columnName); + } break; case COMPLEX: analysis = analyzeComplexColumn(capabilities, column, storageAdapter.getColumnTypeName(columnName)); @@ -140,6 +153,11 @@ public class SegmentAnalyzer return analysisTypes.contains(SegmentMetadataQuery.AnalysisType.CARDINALITY); } + public boolean analyzingMinMax() + { + return analysisTypes.contains(SegmentMetadataQuery.AnalysisType.MINMAX); + } + private ColumnAnalysis analyzeNumericColumn( final ColumnCapabilities capabilities, final int length, @@ -161,28 +179,30 @@ public class SegmentAnalyzer capabilities.hasMultipleValues(), size, null, + null, + null, null ); } private ColumnAnalysis analyzeStringColumn( final ColumnCapabilities capabilities, - @Nullable final Column column, - final int cardinality + final Column column ) { long size = 0; - if (column != null && analyzingSize()) { - if (!capabilities.hasBitmapIndexes()) { - return ColumnAnalysis.error("string_no_bitmap"); - } + Comparable min = null; + Comparable max = null; - final BitmapIndex bitmapIndex = column.getBitmapIndex(); - if (cardinality != bitmapIndex.getCardinality()) { - return ColumnAnalysis.error("bitmap_wrong_cardinality"); - } + if (!capabilities.hasBitmapIndexes()) { + return ColumnAnalysis.error("string_no_bitmap"); + } + final BitmapIndex bitmapIndex = column.getBitmapIndex(); + final int cardinality = bitmapIndex.getCardinality(); + + if (analyzingSize()) { for (int i = 0; i < cardinality; ++i) { String value = bitmapIndex.getValue(i); if (value != null) { @@ -191,11 +211,91 @@ public class SegmentAnalyzer } } + if (analyzingMinMax() && cardinality > 0) { + min = Strings.nullToEmpty(bitmapIndex.getValue(0)); + max = Strings.nullToEmpty(bitmapIndex.getValue(cardinality - 1)); + } + return new ColumnAnalysis( capabilities.getType().name(), capabilities.hasMultipleValues(), size, analyzingCardinality() ? cardinality : 0, + min, + max, + null + ); + } + + private ColumnAnalysis analyzeStringColumn( + final ColumnCapabilities capabilities, + final StorageAdapter storageAdapter, + final String columnName + ) + { + int cardinality = 0; + long size = 0; + + Comparable min = null; + Comparable max = null; + + if (analyzingCardinality()) { + cardinality = storageAdapter.getDimensionCardinality(columnName); + } + + if (analyzingSize()) { + final long start = storageAdapter.getMinTime().getMillis(); + final long end = storageAdapter.getMaxTime().getMillis(); + + final Sequence cursors = + storageAdapter.makeCursors(null, new Interval(start, end), QueryGranularity.ALL, false); + + size = cursors.accumulate( + 0L, + new Accumulator() + { + @Override + public Long accumulate(Long accumulated, Cursor cursor) + { + DimensionSelector selector = cursor.makeDimensionSelector( + new DefaultDimensionSpec( + columnName, + columnName + ) + ); + if (selector == null) { + return accumulated; + } + long current = accumulated; + while (!cursor.isDone()) { + final IndexedInts vals = selector.getRow(); + for (int i = 0; i < vals.size(); ++i) { + final String dimVal = selector.lookupName(vals.get(i)); + if (dimVal != null && !dimVal.isEmpty()) { + current += StringUtils.toUtf8(dimVal).length; + } + } + cursor.advance(); + } + + return current; + } + } + ); + } + + if (analyzingMinMax()) { + min = storageAdapter.getMinValue(columnName); + max = storageAdapter.getMaxValue(columnName); + } + + return new ColumnAnalysis( + capabilities.getType().name(), + capabilities.hasMultipleValues(), + size, + cardinality, + min, + max, null ); } @@ -218,7 +318,7 @@ public class SegmentAnalyzer final Function inputSizeFn = serde.inputSizeFn(); if (inputSizeFn == null) { - return new ColumnAnalysis(typeName, hasMultipleValues, 0, null, null); + return new ColumnAnalysis(typeName, hasMultipleValues, 0, null, null, null, null); } final int length = column.getLength(); @@ -232,6 +332,8 @@ public class SegmentAnalyzer hasMultipleValues, size, null, + null, + null, null ); } diff --git a/processing/src/main/java/io/druid/query/metadata/metadata/ColumnAnalysis.java b/processing/src/main/java/io/druid/query/metadata/metadata/ColumnAnalysis.java index 33552b523a9..b353612e30e 100644 --- a/processing/src/main/java/io/druid/query/metadata/metadata/ColumnAnalysis.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/ColumnAnalysis.java @@ -21,6 +21,7 @@ package io.druid.query.metadata.metadata; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import java.util.Objects; @@ -32,13 +33,15 @@ public class ColumnAnalysis public static ColumnAnalysis error(String reason) { - return new ColumnAnalysis("STRING", false, -1, null, ERROR_PREFIX + reason); + return new ColumnAnalysis("STRING", false, -1, null, null, null, ERROR_PREFIX + reason); } private final String type; private final boolean hasMultipleValues; private final long size; private final Integer cardinality; + private final Comparable minValue; + private final Comparable maxValue; private final String errorMessage; @JsonCreator @@ -47,6 +50,8 @@ public class ColumnAnalysis @JsonProperty("hasMultipleValues") boolean hasMultipleValues, @JsonProperty("size") long size, @JsonProperty("cardinality") Integer cardinality, + @JsonProperty("minValue") Comparable minValue, + @JsonProperty("maxValue") Comparable maxValue, @JsonProperty("errorMessage") String errorMessage ) { @@ -54,6 +59,8 @@ public class ColumnAnalysis this.hasMultipleValues = hasMultipleValues; this.size = size; this.cardinality = cardinality; + this.minValue = minValue; + this.maxValue = maxValue; this.errorMessage = errorMessage; } @@ -81,6 +88,20 @@ public class ColumnAnalysis return cardinality; } + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME) + @JsonProperty + public Comparable getMinValue() + { + return minValue; + } + + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME) + @JsonProperty + public Comparable getMaxValue() + { + return maxValue; + } + @JsonProperty public String getErrorMessage() { @@ -113,21 +134,29 @@ public class ColumnAnalysis Integer cardinality = getCardinality(); final Integer rhsCardinality = rhs.getCardinality(); if (cardinality == null) { - cardinality = rhsCardinality; - } else { - if (rhsCardinality != null) { - cardinality = Math.max(cardinality, rhsCardinality); - } + } else if (rhsCardinality != null) { + cardinality = Math.max(cardinality, rhsCardinality); } - return new ColumnAnalysis( - type, - hasMultipleValues || rhs.isHasMultipleValues(), - size + rhs.getSize(), - cardinality, - null - ); + final boolean multipleValues = hasMultipleValues || rhs.isHasMultipleValues(); + + Comparable newMin = choose(minValue, rhs.minValue, false); + Comparable newMax = choose(maxValue, rhs.maxValue, true); + + return new ColumnAnalysis(type, multipleValues, size + rhs.getSize(), cardinality, newMin, newMax, null); + } + + private T choose(T obj1, T obj2, boolean max) + { + if (obj1 == null) { + return max ? obj2 : null; + } + if (obj2 == null) { + return max ? obj1 : null; + } + int compare = max ? obj1.compareTo(obj2) : obj2.compareTo(obj1); + return compare > 0 ? obj1 : obj2; } @Override @@ -138,6 +167,8 @@ public class ColumnAnalysis ", hasMultipleValues=" + hasMultipleValues + ", size=" + size + ", cardinality=" + cardinality + + ", minValue=" + minValue + + ", maxValue=" + maxValue + ", errorMessage='" + errorMessage + '\'' + '}'; } @@ -156,12 +187,14 @@ public class ColumnAnalysis size == that.size && Objects.equals(type, that.type) && Objects.equals(cardinality, that.cardinality) && + Objects.equals(minValue, that.minValue) && + Objects.equals(maxValue, that.maxValue) && Objects.equals(errorMessage, that.errorMessage); } @Override public int hashCode() { - return Objects.hash(type, hasMultipleValues, size, cardinality, errorMessage); + return Objects.hash(type, hasMultipleValues, size, cardinality, minValue, maxValue, errorMessage); } } diff --git a/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java index 3b270be966c..18a56886219 100644 --- a/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java @@ -53,7 +53,8 @@ public class SegmentMetadataQuery extends BaseQuery CARDINALITY, SIZE, INTERVAL, - AGGREGATORS; + AGGREGATORS, + MINMAX; @JsonValue @Override @@ -81,7 +82,8 @@ public class SegmentMetadataQuery extends BaseQuery public static final EnumSet DEFAULT_ANALYSIS_TYPES = EnumSet.of( AnalysisType.CARDINALITY, AnalysisType.SIZE, - AnalysisType.INTERVAL + AnalysisType.INTERVAL, + AnalysisType.MINMAX ); private final ColumnIncluderator toInclude; @@ -177,6 +179,11 @@ public class SegmentMetadataQuery extends BaseQuery return analysisTypes.contains(AnalysisType.AGGREGATORS); } + public boolean hasMinMax() + { + return analysisTypes.contains(AnalysisType.MINMAX); + } + public byte[] getAnalysisTypesCacheKey() { int size = 1; @@ -242,6 +249,20 @@ public class SegmentMetadataQuery extends BaseQuery ); } + public Query withColumns(ColumnIncluderator includerator) + { + return new SegmentMetadataQuery( + getDataSource(), + getQuerySegmentSpec(), + includerator, + merge, + getContext(), + analysisTypes, + usingDefaultInterval, + lenientAggregatorMerge + ); + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index c9eba3c4203..f9a6467e68c 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -35,6 +35,7 @@ import io.druid.query.QueryInterruptedException; import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.Filter; +import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ComplexColumn; @@ -140,6 +141,28 @@ public class QueryableIndexStorageAdapter implements StorageAdapter } } + @Override + public Comparable getMinValue(String dimension) + { + Column column = index.getColumn(dimension); + if (column != null && column.getCapabilities().hasBitmapIndexes()) { + BitmapIndex bitmap = column.getBitmapIndex(); + return bitmap.getCardinality() > 0 ? bitmap.getValue(0) : null; + } + return null; + } + + @Override + public Comparable getMaxValue(String dimension) + { + Column column = index.getColumn(dimension); + if (column != null && column.getCapabilities().hasBitmapIndexes()) { + BitmapIndex bitmap = column.getBitmapIndex(); + return bitmap.getCardinality() > 0 ? bitmap.getValue(bitmap.getCardinality() - 1) : null; + } + return null; + } + @Override public Capabilities getCapabilities() { diff --git a/processing/src/main/java/io/druid/segment/StorageAdapter.java b/processing/src/main/java/io/druid/segment/StorageAdapter.java index b557757dffa..418a45e4ac1 100644 --- a/processing/src/main/java/io/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/StorageAdapter.java @@ -44,6 +44,8 @@ public interface StorageAdapter extends CursorFactory public int getDimensionCardinality(String column); public DateTime getMinTime(); public DateTime getMaxTime(); + public Comparable getMinValue(String column); + public Comparable getMaxValue(String column); public Capabilities getCapabilities(); public ColumnCapabilities getColumnCapabilities(String column); diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 46f1a79194a..6d45e547987 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -51,6 +51,7 @@ import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ValueType; +import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.serde.ComplexMetricExtractor; import io.druid.segment.serde.ComplexMetricSerde; @@ -847,6 +848,10 @@ public abstract class IncrementalIndex implements Iterable, public int size(); + public String getMinValue(); + + public String getMaxValue(); + public int add(String value); public SortedDimLookup sort(); @@ -899,6 +904,18 @@ public abstract class IncrementalIndex implements Iterable, return delegate.size(); } + @Override + public String getMinValue() + { + return Strings.nullToEmpty(delegate.getMinValue()); + } + + @Override + public String getMaxValue() + { + return Strings.nullToEmpty(delegate.getMaxValue()); + } + @Override public int add(String value) { diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java index d430f2e8695..3cc5825568c 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -136,6 +136,20 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter return index.getMaxTime(); } + @Override + public Comparable getMinValue(String column) + { + IncrementalIndex.DimDim dimDim = index.getDimensionValues(column); + return dimDim == null ? null : dimDim.getMinValue(); + } + + @Override + public Comparable getMaxValue(String column) + { + IncrementalIndex.DimDim dimDim = index.getDimensionValues(column); + return dimDim == null ? null : dimDim.getMaxValue(); + } + @Override public Capabilities getCapabilities() { diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index dcf6c0d2946..6b4e6b39214 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -278,6 +278,8 @@ public class OnheapIncrementalIndex extends IncrementalIndex static class OnHeapDimDim implements DimDim { private final Map valueToId = Maps.newHashMap(); + private String minValue = null; + private String maxValue = null; private final List idToValue = Lists.newArrayList(); private final Object lock; @@ -326,10 +328,24 @@ public class OnheapIncrementalIndex extends IncrementalIndex final int index = size(); valueToId.put(value, index); idToValue.add(value); + minValue = minValue == null || minValue.compareTo(value) > 0 ? value : minValue; + maxValue = maxValue == null || maxValue.compareTo(value) < 0 ? value : maxValue; return index; } } + @Override + public String getMinValue() + { + return minValue; + } + + @Override + public String getMaxValue() + { + return maxValue; + } + public OnHeapDimLookup sort() { synchronized (lock) { diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java index b01c94e52f2..2502b81654b 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java @@ -63,7 +63,7 @@ public class SegmentMetadataQueryQueryToolChestTest new SegmentMetadataQueryQueryToolChest(null).getCacheStrategy(query); // Test cache key generation - byte[] expectedKey = {0x04, 0x01, (byte) 0xFF, 0x00, 0x01, 0x02}; + byte[] expectedKey = {0x04, 0x01, (byte) 0xFF, 0x00, 0x01, 0x02, 0x04}; byte[] actualKey = strategy.computeCacheKey(query); Assert.assertArrayEquals(expectedKey, actualKey); @@ -79,6 +79,8 @@ public class SegmentMetadataQueryQueryToolChestTest true, 10881, 1, + "preferred", + "preferred", null ) ), 71982, diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java index 3b199f15edb..628fcfa6a20 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java @@ -153,14 +153,18 @@ public class SegmentMetadataQueryTest false, 12090, null, + null, + null, null ), "placement", new ColumnAnalysis( ValueType.STRING.toString(), false, - mmap1 ? 10881 : 0, + mmap1 ? 10881 : 10764, 1, + "preferred", + "preferred", null ), "index", @@ -169,9 +173,11 @@ public class SegmentMetadataQueryTest false, 9672, null, + null, + null, null ) - ), mmap1 ? 71982 : 32643, + ), mmap1 ? 71982 : 72755, 1209, null ); @@ -187,6 +193,8 @@ public class SegmentMetadataQueryTest false, 12090, null, + null, + null, null ), "placement", @@ -195,6 +203,8 @@ public class SegmentMetadataQueryTest false, mmap2 ? 10881 : 0, 1, + null, + null, null ), "index", @@ -203,9 +213,12 @@ public class SegmentMetadataQueryTest false, 9672, null, + null, + null, null ) - ), mmap2 ? 71982 : 32643, + // null_column will be included only for incremental index, which makes a little bigger result than expected + ), mmap2 ? 71982 : 72755, 1209, null ); @@ -236,6 +249,8 @@ public class SegmentMetadataQueryTest false, 0, 1, + null, + null, null ), "placementish", @@ -244,6 +259,8 @@ public class SegmentMetadataQueryTest true, 0, 9, + null, + null, null ) ), @@ -298,6 +315,8 @@ public class SegmentMetadataQueryTest false, 0, 1, + null, + null, null ), "quality_uniques", @@ -306,6 +325,8 @@ public class SegmentMetadataQueryTest false, 0, null, + null, + null, null ) ), @@ -349,6 +370,53 @@ public class SegmentMetadataQueryTest @Test public void testSegmentMetadataQueryWithDefaultAnalysisMerge() + { + ColumnAnalysis analysis = new ColumnAnalysis( + ValueType.STRING.toString(), + false, + (mmap1 ? 10881 : 10764) + (mmap2 ? 10881 : 10764), + 1, + "preferred", + "preferred", + null + ); + testSegmentMetadataQueryWithDefaultAnalysisMerge("placement", analysis); + } + + @Test + public void testSegmentMetadataQueryWithDefaultAnalysisMerge2() + { + ColumnAnalysis analysis = new ColumnAnalysis( + ValueType.STRING.toString(), + false, + (mmap1 ? 6882 : 6808) + (mmap2 ? 6882 : 6808), + 3, + "spot", + "upfront", + null + ); + testSegmentMetadataQueryWithDefaultAnalysisMerge("market", analysis); + } + + @Test + public void testSegmentMetadataQueryWithDefaultAnalysisMerge3() + { + ColumnAnalysis analysis = new ColumnAnalysis( + ValueType.STRING.toString(), + false, + (mmap1 ? 9765 : 9660) + (mmap2 ? 9765 : 9660), + 9, + "automotive", + "travel", + null + ); + testSegmentMetadataQueryWithDefaultAnalysisMerge("quality", analysis); + } + + private void testSegmentMetadataQueryWithDefaultAnalysisMerge( + String column, + ColumnAnalysis analysis + ) { SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis( differentIds ? "merged" : "testSegment", @@ -360,14 +428,8 @@ public class SegmentMetadataQueryTest false, 12090 * 2, null, - null - ), - "placement", - new ColumnAnalysis( - ValueType.STRING.toString(), - false, - 10881 * ((mmap1 ? 1 : 0) + (mmap2 ? 1 : 0)), - 1, + null, + null, null ), "index", @@ -376,8 +438,12 @@ public class SegmentMetadataQueryTest false, 9672 * 2, null, + null, + null, null - ) + ), + column, + analysis ), expectedSegmentAnalysis1.getSize() + expectedSegmentAnalysis2.getSize(), expectedSegmentAnalysis1.getNumRows() + expectedSegmentAnalysis2.getNumRows(), @@ -400,12 +466,11 @@ public class SegmentMetadataQueryTest toolChest ); + Query query = testQuery.withColumns(new ListColumnIncluderator(Arrays.asList("__time", "index", column))); + TestHelper.assertExpectedObjects( ImmutableList.of(mergedSegmentAnalysis), - myRunner.run( - testQuery, - Maps.newHashMap() - ), + myRunner.run(query, Maps.newHashMap()), "failed SegmentMetadata merging query" ); exec.shutdownNow(); @@ -424,6 +489,8 @@ public class SegmentMetadataQueryTest false, 0, 0, + null, + null, null ) ), @@ -482,6 +549,8 @@ public class SegmentMetadataQueryTest false, 0, 0, + null, + null, null ) ), diff --git a/processing/src/test/java/io/druid/query/metadata/metadata/ColumnAnalysisTest.java b/processing/src/test/java/io/druid/query/metadata/metadata/ColumnAnalysisTest.java index 90cc2a8ea77..224fdd29296 100644 --- a/processing/src/test/java/io/druid/query/metadata/metadata/ColumnAnalysisTest.java +++ b/processing/src/test/java/io/druid/query/metadata/metadata/ColumnAnalysisTest.java @@ -19,75 +19,149 @@ package io.druid.query.metadata.metadata; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.segment.TestHelper; import org.junit.Assert; import org.junit.Test; public class ColumnAnalysisTest { - @Test - public void testFoldStringColumns() + private final ObjectMapper MAPPER = TestHelper.getObjectMapper(); + + private void assertSerDe(ColumnAnalysis analysis) throws Exception { - final ColumnAnalysis analysis1 = new ColumnAnalysis("STRING", false, 1L, 2, null); - final ColumnAnalysis analysis2 = new ColumnAnalysis("STRING", true, 3L, 4, null); - final ColumnAnalysis expected = new ColumnAnalysis("STRING", true, 4L, 4, null); - Assert.assertEquals(expected, analysis1.fold(analysis2)); - Assert.assertEquals(expected, analysis2.fold(analysis1)); + Assert.assertEquals(analysis, MAPPER.readValue(MAPPER.writeValueAsString(analysis), ColumnAnalysis.class)); } @Test - public void testFoldWithNull() + public void testFoldStringColumns() throws Exception { - final ColumnAnalysis analysis1 = new ColumnAnalysis("STRING", false, 1L, 2, null); + final ColumnAnalysis analysis1 = new ColumnAnalysis("STRING", false, 1L, 2, "aaA", "Zzz", null); + final ColumnAnalysis analysis2 = new ColumnAnalysis("STRING", true, 3L, 4, "aAA", "ZZz", null); + + assertSerDe(analysis1); + assertSerDe(analysis2); + + final ColumnAnalysis expected = new ColumnAnalysis("STRING", true, 4L, 4, "aAA", "Zzz", null); + + ColumnAnalysis fold1 = analysis1.fold(analysis2); + ColumnAnalysis fold2 = analysis2.fold(analysis1); + Assert.assertEquals(expected, fold1); + Assert.assertEquals(expected, fold2); + + assertSerDe(fold1); + assertSerDe(fold2); + } + + @Test + public void testFoldWithNull() throws Exception + { + final ColumnAnalysis analysis1 = new ColumnAnalysis("STRING", false, 1L, 2, null, null, null); Assert.assertEquals(analysis1, analysis1.fold(null)); + assertSerDe(analysis1); } @Test - public void testFoldComplexColumns() + public void testFoldComplexColumns() throws Exception { - final ColumnAnalysis analysis1 = new ColumnAnalysis("hyperUnique", false, 0L, null, null); - final ColumnAnalysis analysis2 = new ColumnAnalysis("hyperUnique", false, 0L, null, null); - final ColumnAnalysis expected = new ColumnAnalysis("hyperUnique", false, 0L, null, null); - Assert.assertEquals(expected, analysis1.fold(analysis2)); - Assert.assertEquals(expected, analysis2.fold(analysis1)); + final ColumnAnalysis analysis1 = new ColumnAnalysis("hyperUnique", false, 0L, null, null, null, null); + final ColumnAnalysis analysis2 = new ColumnAnalysis("hyperUnique", false, 0L, null, null, null, null); + + assertSerDe(analysis1); + assertSerDe(analysis2); + + final ColumnAnalysis expected = new ColumnAnalysis("hyperUnique", false, 0L, null, null, null, null); + + ColumnAnalysis fold1 = analysis1.fold(analysis2); + ColumnAnalysis fold2 = analysis2.fold(analysis1); + Assert.assertEquals(expected, fold1); + Assert.assertEquals(expected, fold2); + + assertSerDe(fold1); + assertSerDe(fold2); } @Test - public void testFoldDifferentTypes() + public void testFoldDifferentTypes() throws Exception { - final ColumnAnalysis analysis1 = new ColumnAnalysis("hyperUnique", false, 1L, 1, null); - final ColumnAnalysis analysis2 = new ColumnAnalysis("COMPLEX", false, 2L, 2, null); - final ColumnAnalysis expected = new ColumnAnalysis("STRING", false, -1L, null, "error:cannot_merge_diff_types"); - Assert.assertEquals(expected, analysis1.fold(analysis2)); - Assert.assertEquals(expected, analysis2.fold(analysis1)); + final ColumnAnalysis analysis1 = new ColumnAnalysis("hyperUnique", false, 1L, 1, null, null, null); + final ColumnAnalysis analysis2 = new ColumnAnalysis("COMPLEX", false, 2L, 2, null, null, null); + + assertSerDe(analysis1); + assertSerDe(analysis2); + + final ColumnAnalysis expected = new ColumnAnalysis( + "STRING", + false, + -1L, + null, + null, + null, + "error:cannot_merge_diff_types" + ); + ColumnAnalysis fold1 = analysis1.fold(analysis2); + ColumnAnalysis fold2 = analysis2.fold(analysis1); + Assert.assertEquals(expected, fold1); + Assert.assertEquals(expected, fold2); + + assertSerDe(fold1); + assertSerDe(fold2); } @Test - public void testFoldSameErrors() + public void testFoldSameErrors() throws Exception { final ColumnAnalysis analysis1 = ColumnAnalysis.error("foo"); final ColumnAnalysis analysis2 = ColumnAnalysis.error("foo"); - final ColumnAnalysis expected = new ColumnAnalysis("STRING", false, -1L, null, "error:foo"); - Assert.assertEquals(expected, analysis1.fold(analysis2)); - Assert.assertEquals(expected, analysis2.fold(analysis1)); + + assertSerDe(analysis1); + assertSerDe(analysis2); + + final ColumnAnalysis expected = new ColumnAnalysis("STRING", false, -1L, null, null, null, "error:foo"); + ColumnAnalysis fold1 = analysis1.fold(analysis2); + ColumnAnalysis fold2 = analysis2.fold(analysis1); + Assert.assertEquals(expected, fold1); + Assert.assertEquals(expected, fold2); + + assertSerDe(fold1); + assertSerDe(fold2); } @Test - public void testFoldErrorAndNoError() + public void testFoldErrorAndNoError() throws Exception { final ColumnAnalysis analysis1 = ColumnAnalysis.error("foo"); - final ColumnAnalysis analysis2 = new ColumnAnalysis("STRING", false, 2L, 2, null); - final ColumnAnalysis expected = new ColumnAnalysis("STRING", false, -1L, null, "error:foo"); - Assert.assertEquals(expected, analysis1.fold(analysis2)); - Assert.assertEquals(expected, analysis2.fold(analysis1)); + final ColumnAnalysis analysis2 = new ColumnAnalysis("STRING", false, 2L, 2, "a", "z", null); + + assertSerDe(analysis1); + assertSerDe(analysis2); + + final ColumnAnalysis expected = new ColumnAnalysis("STRING", false, -1L, null, null, null, "error:foo"); + ColumnAnalysis fold1 = analysis1.fold(analysis2); + ColumnAnalysis fold2 = analysis2.fold(analysis1); + Assert.assertEquals(expected, fold1); + Assert.assertEquals(expected, fold2); + + assertSerDe(fold1); + assertSerDe(fold2); } @Test - public void testFoldDifferentErrors() + public void testFoldDifferentErrors() throws Exception { final ColumnAnalysis analysis1 = ColumnAnalysis.error("foo"); final ColumnAnalysis analysis2 = ColumnAnalysis.error("bar"); - final ColumnAnalysis expected = new ColumnAnalysis("STRING", false, -1L, null, "error:multiple_errors"); - Assert.assertEquals(expected, analysis1.fold(analysis2)); - Assert.assertEquals(expected, analysis2.fold(analysis1)); + + assertSerDe(analysis1); + assertSerDe(analysis2); + + final ColumnAnalysis expected = new ColumnAnalysis("STRING", false, -1L, null, null, null, "error:multiple_errors"); + ColumnAnalysis fold1 = analysis1.fold(analysis2); + ColumnAnalysis fold2 = analysis2.fold(analysis1); + Assert.assertEquals(expected, fold1); + Assert.assertEquals(expected, fold2); + + assertSerDe(fold1); + assertSerDe(fold2); } } diff --git a/publications/whitepaper/druid.tex b/publications/whitepaper/druid.tex index 088fd77b71c..b746c972153 100644 --- a/publications/whitepaper/druid.tex +++ b/publications/whitepaper/druid.tex @@ -419,7 +419,7 @@ first maps the query to a set of segments. Results for certain segments may already exist in the cache and there is no need to recompute them. For any results that do not exist in the cache, the broker node will forward the query to the correct historical and real-time nodes. Once historical nodes return -their results, the broker will cache these results on a per segment basis for +their results, the broker will cache these results on a per-segment basis for future use. This process is illustrated in Figure~\ref{fig:caching}. Real-time data is never cached and hence requests for real-time data will always be forwarded to real-time nodes. Real-time data is perpetually changing and @@ -428,7 +428,7 @@ caching the results is unreliable. \begin{figure*} \centering \includegraphics[width = 4.5in]{caching} -\caption{Results are cached per segment. Queries combine cached results with results computed on historical and real-time nodes.} +\caption{Results are cached per-segment. Queries combine cached results with results computed on historical and real-time nodes.} \label{fig:caching} \end{figure*} From da5fcd0124fd83c6a2c0a7729c4229c80c5f0255 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Fri, 12 Feb 2016 13:04:46 -0600 Subject: [PATCH 11/12] before facts get it , indexAndOffsets should already know about it --- .../incremental/OffheapIncrementalIndex.java | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java index f5d4d8248cd..fdbb74ce579 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -200,14 +200,18 @@ public class OffheapIncrementalIndex extends IncrementalIndex bufferOffset = indexAndOffset[1]; aggBuffer = aggBuffers.get(bufferIndex).get(); } else { - rowContainer.set(row); - for (int i = 0; i < metrics.length; i++) { - final AggregatorFactory agg = metrics[i]; - getAggs()[i] = agg.factorizeBuffered( - makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics) - ); + if (metrics.length > 0 && getAggs()[0] == null) { + // note: creation of Aggregators is done lazily when at least one row from input is available + // so that FilteredAggregators could be initialized correctly. + rowContainer.set(row); + for (int i = 0; i < metrics.length; i++) { + final AggregatorFactory agg = metrics[i]; + getAggs()[i] = agg.factorizeBuffered( + makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics) + ); + } + rowContainer.set(null); } - rowContainer.set(null); bufferIndex = aggBuffers.size() - 1; ByteBuffer lastBuffer = aggBuffers.isEmpty() ? null : aggBuffers.get(aggBuffers.size() - 1).get(); @@ -241,10 +245,13 @@ public class OffheapIncrementalIndex extends IncrementalIndex } final Integer rowIndex = indexIncrement.getAndIncrement(); + + // note that indexAndOffsets must be updated before facts, because as soon as we update facts + // concurrent readers get hold of it and might ask for newly added row + indexAndOffsets.add(new int[]{bufferIndex, bufferOffset}); final Integer prev = facts.putIfAbsent(key, rowIndex); if (null == prev) { numEntries.incrementAndGet(); - indexAndOffsets.add(new int[]{bufferIndex, bufferOffset}); } else { throw new ISE("WTF! we are in sychronized block."); } From e0c049c0b092be48449943a6072e582446a389f0 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 12 Feb 2016 14:12:16 -0800 Subject: [PATCH 12/12] Make startup properties logging optional. Off by default, but enabled in the example config files. See also #2452. --- docs/content/configuration/index.md | 10 ++++++ .../druid/_common/common.runtime.properties | 12 ++++++- .../druid/_common/common.runtime.properties | 12 ++++++- .../java/io/druid/guice/PropertiesModule.java | 10 ------ .../io/druid/guice/StartupLoggingModule.java | 33 +++++++++++++++++++ .../druid/initialization/Initialization.java | 4 ++- .../server/log/StartupLoggingConfig.java | 33 +++++++++++++++++++ .../main/java/io/druid/cli/GuiceRunnable.java | 18 ++++++++++ 8 files changed, 119 insertions(+), 13 deletions(-) create mode 100644 server/src/main/java/io/druid/guice/StartupLoggingModule.java create mode 100644 server/src/main/java/io/druid/server/log/StartupLoggingConfig.java diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index a3f934a6844..31cf108955b 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -74,6 +74,16 @@ The following path is used for service discovery. It is **not** affected by `dru |--------|-----------|-------| |`druid.discovery.curator.path`|Services announce themselves under this ZooKeeper path.|`/druid/discovery`| +### Startup Logging + +All nodes can log debugging information on startup. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.startup.logging.logProperties`|Log all properties on startup (from common.runtime.properties, runtime.properties, and the JVM command line).|false| + +Note that some sensitive information may be logged if these settings are enabled. + ### Request Logging All nodes that can serve queries can also log the query requests they see. diff --git a/examples/conf-quickstart/druid/_common/common.runtime.properties b/examples/conf-quickstart/druid/_common/common.runtime.properties index 23e94bb89c3..38f0f262e42 100644 --- a/examples/conf-quickstart/druid/_common/common.runtime.properties +++ b/examples/conf-quickstart/druid/_common/common.runtime.properties @@ -30,6 +30,13 @@ druid.extensions.loadList=[] # and uncomment the line below to point to your directory. #druid.extensions.hadoopDependenciesDir=/my/dir/hadoop-dependencies +# +# Logging +# + +# Log all runtime properties on startup. Disable to avoid logging properties on startup: +druid.startup.logging.logProperties=true + # # Zookeeper # @@ -41,6 +48,7 @@ druid.zk.paths.base=/druid # Metadata storage # +# For Derby server on your Druid Coordinator (only viable in a cluster with a single Coordinator, no fail-over): druid.metadata.storage.type=derby druid.metadata.storage.connector.connectURI=jdbc:derby://localhost:1527/var/druid/metadata.db;create=true druid.metadata.storage.connector.host=localhost @@ -62,6 +70,7 @@ druid.metadata.storage.connector.port=1527 # Deep storage # +# For local disk (only viable in a cluster if this is a network mount): druid.storage.type=local druid.storage.storageDirectory=var/druid/segments @@ -80,6 +89,7 @@ druid.storage.storageDirectory=var/druid/segments # Indexing service logs # +# For local disk (only viable in a cluster if this is a network mount): druid.indexer.logs.type=file druid.indexer.logs.directory=var/druid/indexing-logs @@ -105,4 +115,4 @@ druid.selectors.coordinator.serviceName=druid/coordinator druid.monitoring.monitors=["com.metamx.metrics.JvmMonitor"] druid.emitter=logging -druid.emitter.logging.logLevel=debug +druid.emitter.logging.logLevel=info diff --git a/examples/conf/druid/_common/common.runtime.properties b/examples/conf/druid/_common/common.runtime.properties index edd2712b873..19b00525736 100644 --- a/examples/conf/druid/_common/common.runtime.properties +++ b/examples/conf/druid/_common/common.runtime.properties @@ -29,6 +29,13 @@ druid.extensions.loadList=["druid-kafka-eight", "druid-s3-extensions", "druid-hi # and uncomment the line below to point to your directory. #druid.extensions.hadoopDependenciesDir=/my/dir/hadoop-dependencies +# +# Logging +# + +# Log all runtime properties on startup. Disable to avoid logging properties on startup: +druid.startup.logging.logProperties=true + # # Zookeeper # @@ -40,6 +47,7 @@ druid.zk.paths.base=/druid # Metadata storage # +# For Derby server on your Druid Coordinator (only viable in a cluster with a single Coordinator, no fail-over): druid.metadata.storage.type=derby druid.metadata.storage.connector.connectURI=jdbc:derby://metadata.store.ip:1527/var/druid/metadata.db;create=true druid.metadata.storage.connector.host=metadata.store.ip @@ -61,6 +69,7 @@ druid.metadata.storage.connector.port=1527 # Deep storage # +# For local disk (only viable in a cluster if this is a network mount): druid.storage.type=local druid.storage.storageDirectory=var/druid/segments @@ -79,12 +88,13 @@ druid.storage.storageDirectory=var/druid/segments # Indexing service logs # +# For local disk (only viable in a cluster if this is a network mount): druid.indexer.logs.type=file druid.indexer.logs.directory=var/druid/indexing-logs # For HDFS (make sure to include the HDFS extension and that your Hadoop config files in the cp): #druid.indexer.logs.type=hdfs -#druid.indexer.logs.directory=hdfs://namenode.example.com:9000/druid/indexing-logs +#druid.indexer.logs.directory=/druid/indexing-logs # For S3: #druid.indexer.logs.type=s3 diff --git a/processing/src/main/java/io/druid/guice/PropertiesModule.java b/processing/src/main/java/io/druid/guice/PropertiesModule.java index dfc1abf11b2..408b964906c 100644 --- a/processing/src/main/java/io/druid/guice/PropertiesModule.java +++ b/processing/src/main/java/io/druid/guice/PropertiesModule.java @@ -87,16 +87,6 @@ public class PropertiesModule implements Module } } - log.info( - "Loaded properties into JVM with processors[%,d], memory[%,d].", - Runtime.getRuntime().availableProcessors(), - Runtime.getRuntime().totalMemory() - ); - - for (String propertyName : Ordering.natural().sortedCopy(props.stringPropertyNames())) { - log.info("* %s: %s", propertyName, props.getProperty(propertyName)); - } - binder.bind(Properties.class).toInstance(props); } } diff --git a/server/src/main/java/io/druid/guice/StartupLoggingModule.java b/server/src/main/java/io/druid/guice/StartupLoggingModule.java new file mode 100644 index 00000000000..876589b55f9 --- /dev/null +++ b/server/src/main/java/io/druid/guice/StartupLoggingModule.java @@ -0,0 +1,33 @@ +/* + * 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.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import io.druid.server.log.StartupLoggingConfig; + +public class StartupLoggingModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.startup.logging", StartupLoggingConfig.class); + } +} diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index 0a13b6d7871..2448070ec3f 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -50,6 +50,7 @@ import io.druid.guice.QueryRunnerFactoryModule; import io.druid.guice.QueryableModule; import io.druid.guice.ServerModule; import io.druid.guice.ServerViewModule; +import io.druid.guice.StartupLoggingModule; import io.druid.guice.StorageNodeModule; import io.druid.guice.annotations.Client; import io.druid.guice.annotations.Json; @@ -302,7 +303,8 @@ public class Initialization new CoordinatorDiscoveryModule(), new LocalDataStorageDruidModule(), new FirehoseModule(), - new ParsersModule() + new ParsersModule(), + new StartupLoggingModule() ); ModuleList actualModules = new ModuleList(baseInjector); diff --git a/server/src/main/java/io/druid/server/log/StartupLoggingConfig.java b/server/src/main/java/io/druid/server/log/StartupLoggingConfig.java new file mode 100644 index 00000000000..77cb33b58ec --- /dev/null +++ b/server/src/main/java/io/druid/server/log/StartupLoggingConfig.java @@ -0,0 +1,33 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.log; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class StartupLoggingConfig +{ + @JsonProperty + private boolean logProperties = false; + + public boolean isLogProperties() + { + return logProperties; + } +} diff --git a/services/src/main/java/io/druid/cli/GuiceRunnable.java b/services/src/main/java/io/druid/cli/GuiceRunnable.java index 78658ac0e61..d75184dfe24 100644 --- a/services/src/main/java/io/druid/cli/GuiceRunnable.java +++ b/services/src/main/java/io/druid/cli/GuiceRunnable.java @@ -20,6 +20,7 @@ package io.druid.cli; import com.google.common.base.Throwables; +import com.google.common.collect.Ordering; import com.google.inject.Inject; import com.google.inject.Injector; import com.google.inject.Module; @@ -27,8 +28,10 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import io.druid.initialization.Initialization; import io.druid.initialization.LogLevelAdjuster; +import io.druid.server.log.StartupLoggingConfig; import java.util.List; +import java.util.Properties; /** */ @@ -68,6 +71,21 @@ public abstract class GuiceRunnable implements Runnable try { LogLevelAdjuster.register(); final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + final StartupLoggingConfig startupLoggingConfig = injector.getInstance(StartupLoggingConfig.class); + + log.info( + "Starting up with processors[%,d], memory[%,d].", + Runtime.getRuntime().availableProcessors(), + Runtime.getRuntime().totalMemory() + ); + + if (startupLoggingConfig.isLogProperties()) { + final Properties props = injector.getInstance(Properties.class); + + for (String propertyName : Ordering.natural().sortedCopy(props.stringPropertyNames())) { + log.info("* %s: %s", propertyName, props.getProperty(propertyName)); + } + } try { lifecycle.start();