diff --git a/api/pom.xml b/api/pom.xml index 839a65d8951b..907d8193f02f 100644 --- a/api/pom.xml +++ b/api/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT diff --git a/api/src/main/java/io/druid/data/input/impl/CSVParseSpec.java b/api/src/main/java/io/druid/data/input/impl/CSVParseSpec.java index 36a6b670fb7c..ec787bd21856 100644 --- a/api/src/main/java/io/druid/data/input/impl/CSVParseSpec.java +++ b/api/src/main/java/io/druid/data/input/impl/CSVParseSpec.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; import io.druid.java.util.common.parsers.CSVParser; import io.druid.java.util.common.parsers.Parser; @@ -114,7 +113,7 @@ public void verify(List usedCols) @Override public Parser makeParser() { - return new CSVParser(Optional.fromNullable(listDelimiter), columns, hasHeaderRow, skipHeaderRows); + return new CSVParser(listDelimiter, columns, hasHeaderRow, skipHeaderRows); } @Override diff --git a/api/src/main/java/io/druid/data/input/impl/DelimitedParseSpec.java b/api/src/main/java/io/druid/data/input/impl/DelimitedParseSpec.java index 4facbca3e01c..be360675bab7 100644 --- a/api/src/main/java/io/druid/data/input/impl/DelimitedParseSpec.java +++ b/api/src/main/java/io/druid/data/input/impl/DelimitedParseSpec.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; import io.druid.java.util.common.parsers.DelimitedParser; import io.druid.java.util.common.parsers.Parser; @@ -125,8 +124,8 @@ public void verify(List usedCols) public Parser makeParser() { return new DelimitedParser( - Optional.fromNullable(delimiter), - Optional.fromNullable(listDelimiter), + delimiter, + listDelimiter, columns, hasHeaderRow, skipHeaderRows diff --git a/api/src/main/java/io/druid/data/input/impl/MapInputRowParser.java b/api/src/main/java/io/druid/data/input/impl/MapInputRowParser.java index 8847dea0278e..1fafa37a6241 100644 --- a/api/src/main/java/io/druid/data/input/impl/MapInputRowParser.java +++ b/api/src/main/java/io/druid/data/input/impl/MapInputRowParser.java @@ -23,11 +23,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; import com.google.common.collect.Sets; - import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.java.util.common.parsers.ParseException; - import org.joda.time.DateTime; import java.util.List; diff --git a/api/src/test/java/io/druid/jackson/JacksonExtremeDoubleValuesSerdeTest.java b/api/src/test/java/io/druid/jackson/JacksonExtremeDoubleValuesSerdeTest.java new file mode 100644 index 000000000000..789fd82def97 --- /dev/null +++ b/api/src/test/java/io/druid/jackson/JacksonExtremeDoubleValuesSerdeTest.java @@ -0,0 +1,41 @@ +/* + * 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.jackson; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class JacksonExtremeDoubleValuesSerdeTest +{ + @Test + public void testExtremeDoubleValuesSerde() throws IOException + { + ObjectMapper objectMapper = new ObjectMapper(); + for (double value : new double[] {Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY}) { + String serialized = objectMapper.writeValueAsString(value); + Assert.assertEquals(new Double(value), objectMapper.readValue(serialized, Double.class)); + } + String negativeInfinityString = objectMapper.writeValueAsString(Double.NaN); + Assert.assertTrue(objectMapper.readValue(negativeInfinityString, Double.class).isNaN()); + } +} diff --git a/aws-common/pom.xml b/aws-common/pom.xml index 1b063211a036..3f0890e95740 100644 --- a/aws-common/pom.xml +++ b/aws-common/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml index 42c966daa2a1..8a082d30ecba 100644 --- a/benchmarks/pom.xml +++ b/benchmarks/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT diff --git a/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java index 8cfc191aa28a..2cc3ba6c7365 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -32,6 +32,8 @@ import io.druid.benchmark.datagen.BenchmarkSchemas; import io.druid.benchmark.query.QueryBenchmarkUtil; import io.druid.collections.BlockingPool; +import io.druid.collections.DefaultBlockingPool; +import io.druid.collections.NonBlockingPool; import io.druid.collections.StupidPool; import io.druid.concurrent.Execs; import io.druid.data.input.InputRow; @@ -348,7 +350,7 @@ public void setup() throws IOException } } - StupidPool bufferPool = new StupidPool<>( + NonBlockingPool bufferPool = new StupidPool<>( "GroupByBenchmark-computeBufferPool", new OffheapBufferGenerator("compute", 250_000_000), 0, @@ -356,7 +358,7 @@ public void setup() throws IOException ); // limit of 2 is required since we simulate both historical merge and broker merge in the same process - BlockingPool mergePool = new BlockingPool<>( + BlockingPool mergePool = new DefaultBlockingPool<>( new OffheapBufferGenerator("merge", 250_000_000), 2 ); diff --git a/benchmarks/src/main/java/io/druid/benchmark/StupidPoolConcurrencyBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/StupidPoolConcurrencyBenchmark.java index 43ae737495cd..887baf432687 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/StupidPoolConcurrencyBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/StupidPoolConcurrencyBenchmark.java @@ -21,6 +21,7 @@ import com.google.common.base.Supplier; +import io.druid.collections.NonBlockingPool; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidPool; import io.druid.java.util.common.logger.Logger; @@ -64,7 +65,7 @@ public void teardown() public static class BenchmarkPool { private final AtomicLong numPools = new AtomicLong(0L); - private final StupidPool pool = new StupidPool<>( + private final NonBlockingPool pool = new StupidPool<>( "simpleObject pool", new Supplier() { diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java index 6e028b99642b..09f8a1fd2f88 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java @@ -32,6 +32,8 @@ import io.druid.benchmark.datagen.BenchmarkSchemaInfo; import io.druid.benchmark.datagen.BenchmarkSchemas; import io.druid.collections.BlockingPool; +import io.druid.collections.DefaultBlockingPool; +import io.druid.collections.NonBlockingPool; import io.druid.collections.StupidPool; import io.druid.concurrent.Execs; import io.druid.data.input.InputRow; @@ -392,7 +394,7 @@ public void setup() throws IOException } } - StupidPool bufferPool = new StupidPool<>( + NonBlockingPool bufferPool = new StupidPool<>( "GroupByBenchmark-computeBufferPool", new OffheapBufferGenerator("compute", 250_000_000), 0, @@ -400,7 +402,7 @@ public void setup() throws IOException ); // limit of 2 is required since we simulate both historical merge and broker merge in the same process - BlockingPool mergePool = new BlockingPool<>( + BlockingPool mergePool = new DefaultBlockingPool<>( new OffheapBufferGenerator("merge", 250_000_000), 2 ); diff --git a/bytebuffer-collections/pom.xml b/bytebuffer-collections/pom.xml index 36d62b51155d..4b4ed0f14b20 100755 --- a/bytebuffer-collections/pom.xml +++ b/bytebuffer-collections/pom.xml @@ -18,14 +18,13 @@ ~ under the License. --> - + 4.0.0 io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT bytebuffer-collections diff --git a/bytebuffer-collections/src/main/java/io/druid/collections/spatial/Node.java b/bytebuffer-collections/src/main/java/io/druid/collections/spatial/Node.java index 1206083799c1..e93df949c433 100755 --- a/bytebuffer-collections/src/main/java/io/druid/collections/spatial/Node.java +++ b/bytebuffer-collections/src/main/java/io/druid/collections/spatial/Node.java @@ -151,9 +151,9 @@ public boolean enclose() { boolean retVal = false; float[] minCoords = new float[getNumDims()]; - Arrays.fill(minCoords, Float.MAX_VALUE); + Arrays.fill(minCoords, Float.POSITIVE_INFINITY); float[] maxCoords = new float[getNumDims()]; - Arrays.fill(maxCoords, -Float.MAX_VALUE); + Arrays.fill(maxCoords, Float.NEGATIVE_INFINITY); for (Node child : getChildren()) { for (int i = 0; i < getNumDims(); i++) { diff --git a/bytebuffer-collections/src/main/java/io/druid/collections/spatial/RTree.java b/bytebuffer-collections/src/main/java/io/druid/collections/spatial/RTree.java index 74ad9426802a..6cce6b90e75f 100755 --- a/bytebuffer-collections/src/main/java/io/druid/collections/spatial/RTree.java +++ b/bytebuffer-collections/src/main/java/io/druid/collections/spatial/RTree.java @@ -128,8 +128,8 @@ private Node buildRoot(boolean isLeaf) { float[] initMinCoords = new float[numDims]; float[] initMaxCoords = new float[numDims]; - Arrays.fill(initMinCoords, -Float.MAX_VALUE); - Arrays.fill(initMaxCoords, Float.MAX_VALUE); + Arrays.fill(initMinCoords, Float.NEGATIVE_INFINITY); + Arrays.fill(initMaxCoords, Float.POSITIVE_INFINITY); return new Node(initMinCoords, initMaxCoords, isLeaf, bitmapFactory); } @@ -178,7 +178,7 @@ private Node chooseLeaf(Node node, Point point) return node; } - double minCost = Double.MAX_VALUE; + double minCost = Double.POSITIVE_INFINITY; Node optimal = node.getChildren().get(0); for (Node child : node.getChildren()) { double cost = RTreeUtils.getExpansionCost(child, point); diff --git a/bytebuffer-collections/src/main/java/io/druid/collections/spatial/split/LinearGutmanSplitStrategy.java b/bytebuffer-collections/src/main/java/io/druid/collections/spatial/split/LinearGutmanSplitStrategy.java index a193f466cff5..e4249f35f39a 100755 --- a/bytebuffer-collections/src/main/java/io/druid/collections/spatial/split/LinearGutmanSplitStrategy.java +++ b/bytebuffer-collections/src/main/java/io/druid/collections/spatial/split/LinearGutmanSplitStrategy.java @@ -58,10 +58,10 @@ public Node[] pickSeeds(List nodes) double bestNormalized = 0.0; for (int i = 0; i < numDims; i++) { - float minCoord = Float.MAX_VALUE; - float maxCoord = -Float.MAX_VALUE; - float highestLowSide = -Float.MAX_VALUE; - float lowestHighside = Float.MAX_VALUE; + float minCoord = Float.POSITIVE_INFINITY; + float maxCoord = Float.NEGATIVE_INFINITY; + float lowestHighside = Float.POSITIVE_INFINITY; + float highestLowSide = Float.NEGATIVE_INFINITY; int highestLowSideIndex = 0; int lowestHighSideIndex = 0; diff --git a/bytebuffer-collections/src/main/java/io/druid/collections/spatial/split/QuadraticGutmanSplitStrategy.java b/bytebuffer-collections/src/main/java/io/druid/collections/spatial/split/QuadraticGutmanSplitStrategy.java index 444fc5ecaeb0..a1808919d13c 100755 --- a/bytebuffer-collections/src/main/java/io/druid/collections/spatial/split/QuadraticGutmanSplitStrategy.java +++ b/bytebuffer-collections/src/main/java/io/druid/collections/spatial/split/QuadraticGutmanSplitStrategy.java @@ -37,7 +37,7 @@ public QuadraticGutmanSplitStrategy(int minNumChildren, int maxNumChildren, Bitm @Override public Node[] pickSeeds(List nodes) { - double highestCost = Double.MIN_VALUE; + double highestCost = Double.NEGATIVE_INFINITY; int[] highestCostIndices = new int[2]; for (int i = 0; i < nodes.size() - 1; i++) { @@ -58,7 +58,7 @@ public Node[] pickSeeds(List nodes) @Override public Node pickNext(List nodes, Node[] groups) { - double highestCost = Double.MIN_VALUE; + double highestCost = Double.NEGATIVE_INFINITY; Node costlyNode = null; int counter = 0; int index = -1; diff --git a/codestyle/checkstyle.xml b/codestyle/checkstyle.xml index 17570da445dd..0c994b818c03 100644 --- a/codestyle/checkstyle.xml +++ b/codestyle/checkstyle.xml @@ -71,5 +71,31 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/common/pom.xml b/common/pom.xml index d1c6be234351..caa1ff5b40d8 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT diff --git a/common/src/main/java/io/druid/collections/BlockingPool.java b/common/src/main/java/io/druid/collections/BlockingPool.java index af48f53cf098..fdcf7f2356d4 100644 --- a/common/src/main/java/io/druid/collections/BlockingPool.java +++ b/common/src/main/java/io/druid/collections/BlockingPool.java @@ -19,59 +19,11 @@ package io.druid.collections; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; -import io.druid.java.util.common.ISE; - -import java.io.Closeable; -import java.io.IOException; -import java.util.ArrayDeque; import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.ReentrantLock; -/** - * Pool that pre-generates objects up to a limit, then permits possibly-blocking "take" operations. - */ -public class BlockingPool +public interface BlockingPool { - private static final TimeUnit TIME_UNIT = TimeUnit.MILLISECONDS; - - private final ArrayDeque objects; - private final ReentrantLock lock; - private final Condition notEnough; - private final int maxSize; - - public BlockingPool( - Supplier generator, - int limit - ) - { - this.objects = new ArrayDeque<>(limit); - this.maxSize = limit; - - for (int i = 0; i < limit; i++) { - objects.add(generator.get()); - } - - this.lock = new ReentrantLock(); - this.notEnough = lock.newCondition(); - } - - public int maxSize() - { - return maxSize; - } - - @VisibleForTesting - public int getPoolSize() - { - return objects.size(); - } + int maxSize(); /** * Take a resource from the pool, waiting up to the @@ -81,91 +33,14 @@ public int getPoolSize() * * @return a resource, or null if the timeout was reached */ - public ReferenceCountingResourceHolder take(final long timeoutMs) - { - Preconditions.checkArgument(timeoutMs >= 0, "timeoutMs must be a non-negative value, but was [%s]", timeoutMs); - checkInitialized(); - try { - return wrapObject(timeoutMs > 0 ? pollObject(timeoutMs) : pollObject()); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - } + ReferenceCountingResourceHolder take(long timeoutMs); /** * Take a resource from the pool, waiting if necessary until an element becomes available. * * @return a resource */ - public ReferenceCountingResourceHolder take() - { - checkInitialized(); - try { - return wrapObject(takeObject()); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - } - - private ReferenceCountingResourceHolder wrapObject(T theObject) - { - return theObject == null ? null : new ReferenceCountingResourceHolder<>( - theObject, - new Closeable() - { - @Override - public void close() throws IOException - { - offer(theObject); - } - } - ); - } - - private T pollObject() - { - final ReentrantLock lock = this.lock; - lock.lock(); - try { - return objects.isEmpty() ? null : objects.pop(); - } finally { - lock.unlock(); - } - } - - private T pollObject(long timeoutMs) throws InterruptedException - { - long nanos = TIME_UNIT.toNanos(timeoutMs); - final ReentrantLock lock = this.lock; - lock.lockInterruptibly(); - try { - while (objects.isEmpty()) { - if (nanos <= 0) { - return null; - } - nanos = notEnough.awaitNanos(nanos); - } - return objects.pop(); - } finally { - lock.unlock(); - } - } - - private T takeObject() throws InterruptedException - { - final ReentrantLock lock = this.lock; - lock.lockInterruptibly(); - try { - while (objects.isEmpty()) { - notEnough.await(); - } - return objects.pop(); - } finally { - lock.unlock(); - } - } + ReferenceCountingResourceHolder take(); /** * Take resources from the pool, waiting up to the @@ -176,17 +51,7 @@ private T takeObject() throws InterruptedException * * @return a resource, or null if the timeout was reached */ - public ReferenceCountingResourceHolder> takeBatch(final int elementNum, final long timeoutMs) - { - Preconditions.checkArgument(timeoutMs >= 0, "timeoutMs must be a non-negative value, but was [%s]", timeoutMs); - checkInitialized(); - try { - return wrapObjects(timeoutMs > 0 ? pollObjects(elementNum, timeoutMs) : pollObjects(elementNum)); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - } + ReferenceCountingResourceHolder> takeBatch(int elementNum, long timeoutMs); /** * Take resources from the pool, waiting if necessary until the elements of the given number become available. @@ -195,127 +60,5 @@ public ReferenceCountingResourceHolder> takeBatch(final int elementNum, * * @return a resource */ - public ReferenceCountingResourceHolder> takeBatch(final int elementNum) - { - checkInitialized(); - try { - return wrapObjects(takeObjects(elementNum)); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - } - - private ReferenceCountingResourceHolder> wrapObjects(List theObjects) - { - return theObjects == null ? null : new ReferenceCountingResourceHolder<>( - theObjects, - new Closeable() - { - @Override - public void close() throws IOException - { - offerBatch(theObjects); - } - } - ); - } - - private List pollObjects(int elementNum) throws InterruptedException - { - final List list = Lists.newArrayListWithCapacity(elementNum); - final ReentrantLock lock = this.lock; - lock.lockInterruptibly(); - try { - if (objects.size() < elementNum) { - return null; - } else { - for (int i = 0; i < elementNum; i++) { - list.add(objects.pop()); - } - return list; - } - } finally { - lock.unlock(); - } - } - - private List pollObjects(int elementNum, long timeoutMs) throws InterruptedException - { - long nanos = TIME_UNIT.toNanos(timeoutMs); - final List list = Lists.newArrayListWithCapacity(elementNum); - final ReentrantLock lock = this.lock; - lock.lockInterruptibly(); - try { - while (objects.size() < elementNum) { - if (nanos <= 0) { - return null; - } - nanos = notEnough.awaitNanos(nanos); - } - for (int i = 0; i < elementNum; i++) { - list.add(objects.pop()); - } - return list; - } finally { - lock.unlock(); - } - } - - private List takeObjects(int elementNum) throws InterruptedException - { - final List list = Lists.newArrayListWithCapacity(elementNum); - final ReentrantLock lock = this.lock; - lock.lockInterruptibly(); - try { - while (objects.size() < elementNum) { - notEnough.await(); - } - for (int i = 0; i < elementNum; i++) { - list.add(objects.pop()); - } - return list; - } finally { - lock.unlock(); - } - } - - private void checkInitialized() - { - Preconditions.checkState(maxSize > 0, "Pool was initialized with limit = 0, there are no objects to take."); - } - - private void offer(T theObject) - { - final ReentrantLock lock = this.lock; - lock.lock(); - try { - if (objects.size() < maxSize) { - objects.push(theObject); - notEnough.signal(); - } else { - throw new ISE("Cannot exceed pre-configured maximum size"); - } - } finally { - lock.unlock(); - } - } - - private void offerBatch(List offers) - { - final ReentrantLock lock = this.lock; - lock.lock(); - try { - if (objects.size() + offers.size() <= maxSize) { - for (T offer : offers) { - objects.push(offer); - } - notEnough.signal(); - } else { - throw new ISE("Cannot exceed pre-configured maximum size"); - } - } finally { - lock.unlock(); - } - } + ReferenceCountingResourceHolder> takeBatch(int elementNum); } diff --git a/common/src/main/java/io/druid/collections/DefaultBlockingPool.java b/common/src/main/java/io/druid/collections/DefaultBlockingPool.java new file mode 100644 index 000000000000..6e30add171c6 --- /dev/null +++ b/common/src/main/java/io/druid/collections/DefaultBlockingPool.java @@ -0,0 +1,297 @@ +/* + * 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.collections; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import io.druid.java.util.common.ISE; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Pool that pre-generates objects up to a limit, then permits possibly-blocking "take" operations. + */ +public class DefaultBlockingPool implements BlockingPool +{ + private static final TimeUnit TIME_UNIT = TimeUnit.MILLISECONDS; + + private final ArrayDeque objects; + private final ReentrantLock lock; + private final Condition notEnough; + private final int maxSize; + + public DefaultBlockingPool( + Supplier generator, + int limit + ) + { + this.objects = new ArrayDeque<>(limit); + this.maxSize = limit; + + for (int i = 0; i < limit; i++) { + objects.add(generator.get()); + } + + this.lock = new ReentrantLock(); + this.notEnough = lock.newCondition(); + } + + @Override + public int maxSize() + { + return maxSize; + } + + @VisibleForTesting + public int getPoolSize() + { + return objects.size(); + } + + @Override + public ReferenceCountingResourceHolder take(final long timeoutMs) + { + Preconditions.checkArgument(timeoutMs >= 0, "timeoutMs must be a non-negative value, but was [%s]", timeoutMs); + checkInitialized(); + try { + return wrapObject(timeoutMs > 0 ? pollObject(timeoutMs) : pollObject()); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + } + + @Override + public ReferenceCountingResourceHolder take() + { + checkInitialized(); + try { + return wrapObject(takeObject()); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + } + + private ReferenceCountingResourceHolder wrapObject(T theObject) + { + return theObject == null ? null : new ReferenceCountingResourceHolder<>( + theObject, + new Closeable() + { + @Override + public void close() throws IOException + { + offer(theObject); + } + } + ); + } + + private T pollObject() + { + final ReentrantLock lock = this.lock; + lock.lock(); + try { + return objects.isEmpty() ? null : objects.pop(); + } finally { + lock.unlock(); + } + } + + private T pollObject(long timeoutMs) throws InterruptedException + { + long nanos = TIME_UNIT.toNanos(timeoutMs); + final ReentrantLock lock = this.lock; + lock.lockInterruptibly(); + try { + while (objects.isEmpty()) { + if (nanos <= 0) { + return null; + } + nanos = notEnough.awaitNanos(nanos); + } + return objects.pop(); + } finally { + lock.unlock(); + } + } + + private T takeObject() throws InterruptedException + { + final ReentrantLock lock = this.lock; + lock.lockInterruptibly(); + try { + while (objects.isEmpty()) { + notEnough.await(); + } + return objects.pop(); + } finally { + lock.unlock(); + } + } + + @Override + public ReferenceCountingResourceHolder> takeBatch(final int elementNum, final long timeoutMs) + { + Preconditions.checkArgument(timeoutMs >= 0, "timeoutMs must be a non-negative value, but was [%s]", timeoutMs); + checkInitialized(); + try { + return wrapObjects(timeoutMs > 0 ? pollObjects(elementNum, timeoutMs) : pollObjects(elementNum)); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + } + + @Override + public ReferenceCountingResourceHolder> takeBatch(final int elementNum) + { + checkInitialized(); + try { + return wrapObjects(takeObjects(elementNum)); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + } + + private ReferenceCountingResourceHolder> wrapObjects(List theObjects) + { + return theObjects == null ? null : new ReferenceCountingResourceHolder<>( + theObjects, + new Closeable() + { + @Override + public void close() throws IOException + { + offerBatch(theObjects); + } + } + ); + } + + private List pollObjects(int elementNum) throws InterruptedException + { + final List list = Lists.newArrayListWithCapacity(elementNum); + final ReentrantLock lock = this.lock; + lock.lockInterruptibly(); + try { + if (objects.size() < elementNum) { + return null; + } else { + for (int i = 0; i < elementNum; i++) { + list.add(objects.pop()); + } + return list; + } + } finally { + lock.unlock(); + } + } + + private List pollObjects(int elementNum, long timeoutMs) throws InterruptedException + { + long nanos = TIME_UNIT.toNanos(timeoutMs); + final List list = Lists.newArrayListWithCapacity(elementNum); + final ReentrantLock lock = this.lock; + lock.lockInterruptibly(); + try { + while (objects.size() < elementNum) { + if (nanos <= 0) { + return null; + } + nanos = notEnough.awaitNanos(nanos); + } + for (int i = 0; i < elementNum; i++) { + list.add(objects.pop()); + } + return list; + } finally { + lock.unlock(); + } + } + + private List takeObjects(int elementNum) throws InterruptedException + { + final List list = Lists.newArrayListWithCapacity(elementNum); + final ReentrantLock lock = this.lock; + lock.lockInterruptibly(); + try { + while (objects.size() < elementNum) { + notEnough.await(); + } + for (int i = 0; i < elementNum; i++) { + list.add(objects.pop()); + } + return list; + } finally { + lock.unlock(); + } + } + + private void checkInitialized() + { + Preconditions.checkState(maxSize > 0, "Pool was initialized with limit = 0, there are no objects to take."); + } + + private void offer(T theObject) + { + final ReentrantLock lock = this.lock; + lock.lock(); + try { + if (objects.size() < maxSize) { + objects.push(theObject); + notEnough.signal(); + } else { + throw new ISE("Cannot exceed pre-configured maximum size"); + } + } finally { + lock.unlock(); + } + } + + private void offerBatch(List offers) + { + final ReentrantLock lock = this.lock; + lock.lock(); + try { + if (objects.size() + offers.size() <= maxSize) { + for (T offer : offers) { + objects.push(offer); + } + notEnough.signal(); + } else { + throw new ISE("Cannot exceed pre-configured maximum size"); + } + } finally { + lock.unlock(); + } + } +} diff --git a/common/src/main/java/io/druid/collections/DummyBlockingPool.java b/common/src/main/java/io/druid/collections/DummyBlockingPool.java new file mode 100644 index 000000000000..2752e68a8ad2 --- /dev/null +++ b/common/src/main/java/io/druid/collections/DummyBlockingPool.java @@ -0,0 +1,70 @@ +/* + * 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.collections; + +import java.util.List; + +/** + * BlockingPool with 0 maxSize, all take*() methods immediately throw {@link UnsupportedOperationException}. + */ +public final class DummyBlockingPool implements BlockingPool +{ + private static final DummyBlockingPool INSTANCE = new DummyBlockingPool(); + + @SuppressWarnings("unchecked") + public static BlockingPool instance() + { + return INSTANCE; + } + + private DummyBlockingPool() + { + } + + @Override + public int maxSize() + { + return 0; + } + + @Override + public ReferenceCountingResourceHolder take(long timeoutMs) + { + throw new UnsupportedOperationException(); + } + + @Override + public ReferenceCountingResourceHolder take() + { + throw new UnsupportedOperationException(); + } + + @Override + public ReferenceCountingResourceHolder> takeBatch(int elementNum, long timeoutMs) + { + throw new UnsupportedOperationException(); + } + + @Override + public ReferenceCountingResourceHolder> takeBatch(int elementNum) + { + throw new UnsupportedOperationException(); + } +} diff --git a/common/src/main/java/io/druid/collections/DummyNonBlockingPool.java b/common/src/main/java/io/druid/collections/DummyNonBlockingPool.java new file mode 100644 index 000000000000..c03dfd70c90f --- /dev/null +++ b/common/src/main/java/io/druid/collections/DummyNonBlockingPool.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.collections; + +/** + * NonBlockingPool which is not able to allocate objects, {@link #take()} throws {@link UnsupportedOperationException}. + */ +public final class DummyNonBlockingPool implements NonBlockingPool +{ + private static final DummyNonBlockingPool INSTANCE = new DummyNonBlockingPool(); + + @SuppressWarnings("unchecked") + public static NonBlockingPool instance() + { + return INSTANCE; + } + + private DummyNonBlockingPool() + { + } + @Override + public ResourceHolder take() + { + throw new UnsupportedOperationException(); + } +} diff --git a/common/src/main/java/io/druid/collections/NonBlockingPool.java b/common/src/main/java/io/druid/collections/NonBlockingPool.java new file mode 100644 index 000000000000..9941dcf47870 --- /dev/null +++ b/common/src/main/java/io/druid/collections/NonBlockingPool.java @@ -0,0 +1,25 @@ +/* + * 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.collections; + +public interface NonBlockingPool +{ + ResourceHolder take(); +} diff --git a/common/src/main/java/io/druid/collections/StupidPool.java b/common/src/main/java/io/druid/collections/StupidPool.java index 939377a3eec1..dd1918732de1 100644 --- a/common/src/main/java/io/druid/collections/StupidPool.java +++ b/common/src/main/java/io/druid/collections/StupidPool.java @@ -35,7 +35,7 @@ /** */ -public class StupidPool +public class StupidPool implements NonBlockingPool { private static final Logger log = new Logger(StupidPool.class); @@ -95,6 +95,7 @@ public String toString() "}"; } + @Override public ResourceHolder take() { ObjectResourceHolder resourceHolder = objects.poll(); @@ -233,7 +234,7 @@ private static class ObjectLeakNotifier implements Runnable ObjectLeakNotifier(StupidPool pool) { - poolReference = new WeakReference>(pool); + poolReference = new WeakReference<>(pool); leakedObjectsCounter = pool.leakedObjectsCounter; } diff --git a/common/src/main/java/io/druid/concurrent/DummyExecutorService.java b/common/src/main/java/io/druid/concurrent/DummyExecutorService.java new file mode 100644 index 000000000000..aa7e0428f055 --- /dev/null +++ b/common/src/main/java/io/druid/concurrent/DummyExecutorService.java @@ -0,0 +1,123 @@ +/* + * 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.concurrent; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * ExecutorService which is terminated and shutdown from the moment of creation and not able to accept any tasks. + */ +final class DummyExecutorService implements ExecutorService +{ + static final DummyExecutorService INSTANCE = new DummyExecutorService(); + + private DummyExecutorService() + { + } + + @Override + public void shutdown() + { + // Do nothing, alread shutdown + } + + @Override + public List shutdownNow() + { + return Collections.emptyList(); + } + + @Override + public boolean isShutdown() + { + return true; + } + + @Override + public boolean isTerminated() + { + return true; + } + + @Override + public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException + { + return true; + } + + @Override + public Future submit(Callable task) + { + throw new UnsupportedOperationException(); + } + + @Override + public Future submit(Runnable task, T result) + { + throw new UnsupportedOperationException(); + } + + @Override + public Future submit(Runnable task) + { + throw new UnsupportedOperationException(); + } + + @Override + public List> invokeAll(Collection> tasks) throws InterruptedException + { + throw new UnsupportedOperationException(); + } + + @Override + public List> invokeAll( + Collection> tasks, long timeout, TimeUnit unit + ) throws InterruptedException + { + throw new UnsupportedOperationException(); + } + + @Override + public T invokeAny(Collection> tasks) throws InterruptedException, ExecutionException + { + throw new UnsupportedOperationException(); + } + + @Override + public T invokeAny(Collection> tasks, long timeout, TimeUnit unit) + throws InterruptedException, ExecutionException, TimeoutException + { + throw new UnsupportedOperationException(); + } + + @Override + public void execute(Runnable command) + { + throw new UnsupportedOperationException(); + } +} diff --git a/common/src/main/java/io/druid/concurrent/Execs.java b/common/src/main/java/io/druid/concurrent/Execs.java index 04b84f135bbc..65c5181341e4 100644 --- a/common/src/main/java/io/druid/concurrent/Execs.java +++ b/common/src/main/java/io/druid/concurrent/Execs.java @@ -41,6 +41,13 @@ */ public class Execs { + /** + * Returns an ExecutorService which is terminated and shutdown from the beginning and not able to accept any tasks. + */ + public static ExecutorService dummy() + { + return DummyExecutorService.INSTANCE; + } public static ExecutorService singleThreaded(@NotNull String nameFormat) { diff --git a/common/src/test/java/io/druid/collections/BlockingPoolTest.java b/common/src/test/java/io/druid/collections/BlockingPoolTest.java index 73a3d86c3075..f76339ca99c2 100644 --- a/common/src/test/java/io/druid/collections/BlockingPoolTest.java +++ b/common/src/test/java/io/druid/collections/BlockingPoolTest.java @@ -42,8 +42,8 @@ public class BlockingPoolTest { private static final ExecutorService SERVICE = Executors.newFixedThreadPool(2); - private static final BlockingPool POOL = new BlockingPool<>(Suppliers.ofInstance(1), 10); - private static final BlockingPool EMPTY_POOL = new BlockingPool<>(Suppliers.ofInstance(1), 0); + private static final DefaultBlockingPool POOL = new DefaultBlockingPool<>(Suppliers.ofInstance(1), 10); + private static final BlockingPool EMPTY_POOL = new DefaultBlockingPool<>(Suppliers.ofInstance(1), 0); @AfterClass public static void teardown() diff --git a/distribution/pom.xml b/distribution/pom.xml index 0636378d0343..d7d6a80cb6ac 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 pom @@ -29,7 +28,7 @@ druid io.druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT @@ -60,7 +59,7 @@ java -classpath - + -Ddruid.extensions.loadList=[] -Ddruid.extensions.directory=${project.build.directory}/extensions @@ -188,7 +187,7 @@ java -classpath - + -Ddruid.extensions.loadList=[] -Ddruid.extensions.directory=${project.build.directory}/extensions diff --git a/distribution/src/assembly/assembly.xml b/distribution/src/assembly/assembly.xml index 02a71d3ede65..88732f441216 100644 --- a/distribution/src/assembly/assembly.xml +++ b/distribution/src/assembly/assembly.xml @@ -191,6 +191,7 @@ ../ LICENSE + NOTICE diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 4a259b1d5500..fa93f1d7373d 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -28,6 +28,12 @@ Many of Druid's external dependencies can be plugged in as modules. Extensions c |`druid.extensions.hadoopContainerDruidClasspath`|Hadoop Indexing launches hadoop jobs and this configuration provides way to explicitly set the user classpath for the hadoop job. By default this is computed automatically by druid based on the druid process classpath and set of extensions. However, sometimes you might want to be explicit to resolve dependency conflicts between druid and hadoop.|null| |`druid.extensions.addExtensionsToHadoopContainer`|Only applicable if `druid.extensions.hadoopContainerDruidClasspath` is provided. If set to true, then extensions specified in the loadList are added to hadoop container classpath. Note that when `druid.extensions.hadoopContainerDruidClasspath` is not provided then extensions are always added to hadoop container classpath.|false| +### Modules + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.modules.excludeList`|A JSON array of canonical class names (e. g. `"io.druid.somepackage.SomeModule"`) of module classes which shouldn't be loaded, even if they are found in extensions specified by `druid.extensions.loadList`, or in the list of core modules specified to be loaded on a particular Druid node type. Useful when some useful extension contains some module, which shouldn't be loaded on some Druid node type because some dependencies of that module couldn't be satisfied.|[]| + ### Zookeeper We recommend just setting the base ZK path and the ZK service host, but all ZK paths that Druid uses can be overwritten to absolute paths. diff --git a/docs/content/development/router.md b/docs/content/development/router.md index 8fdb21ad3035..63ba585d4427 100644 --- a/docs/content/development/router.md +++ b/docs/content/development/router.md @@ -49,7 +49,6 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/router -druid.processing.numThreads=1 druid.router.defaultBrokerServiceName=druid:broker-cold druid.router.coordinatorServiceName=druid:coordinator druid.router.tierToBrokerMap={"hot":"druid:broker-hot","_default_tier":"druid:broker-cold"} diff --git a/docs/content/querying/aggregations.md b/docs/content/querying/aggregations.md index 77aec92786f9..7b986b57f303 100644 --- a/docs/content/querying/aggregations.md +++ b/docs/content/querying/aggregations.md @@ -17,7 +17,7 @@ Available aggregations are: ``` Please note the count aggregator counts the number of Druid rows, which does not always reflect the number of raw events ingested. -This is because Druid rolls up data at ingestion time. To +This is because Druid can be configured to roll up data at ingestion time. To count the number of ingested rows of data, include a count aggregator at ingestion time, and a longSum aggregator at query time. diff --git a/examples/conf-quickstart/druid/middleManager/runtime.properties b/examples/conf-quickstart/druid/middleManager/runtime.properties index dc1f6aceddb3..254148fc51dd 100644 --- a/examples/conf-quickstart/druid/middleManager/runtime.properties +++ b/examples/conf-quickstart/druid/middleManager/runtime.properties @@ -11,9 +11,9 @@ druid.indexer.task.baseTaskDir=var/druid/task # HTTP server threads druid.server.http.numThreads=9 -# Processing threads and buffers -druid.processing.buffer.sizeBytes=256000000 -druid.processing.numThreads=2 +# Processing threads and buffers on Peons +druid.indexer.fork.property.druid.processing.buffer.sizeBytes=256000000 +druid.indexer.fork.property.druid.processing.numThreads=2 # Hadoop indexing druid.indexer.task.hadoopWorkingPath=var/druid/hadoop-tmp diff --git a/examples/conf/druid/middleManager/runtime.properties b/examples/conf/druid/middleManager/runtime.properties index ce857a0d09d8..50f03013c5a7 100644 --- a/examples/conf/druid/middleManager/runtime.properties +++ b/examples/conf/druid/middleManager/runtime.properties @@ -11,9 +11,9 @@ druid.indexer.task.baseTaskDir=var/druid/task # HTTP server threads druid.server.http.numThreads=25 -# Processing threads and buffers -druid.processing.buffer.sizeBytes=536870912 -druid.processing.numThreads=2 +# Processing threads and buffers on Peons +druid.indexer.fork.property.druid.processing.buffer.sizeBytes=536870912 +druid.indexer.fork.property.druid.processing.numThreads=2 # Hadoop indexing druid.indexer.task.hadoopWorkingPath=var/druid/hadoop-tmp diff --git a/examples/pom.xml b/examples/pom.xml index 167ab83ce91d..312671a65cde 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT diff --git a/extendedset/pom.xml b/extendedset/pom.xml index 9d76baf22206..53d0733c4800 100755 --- a/extendedset/pom.xml +++ b/extendedset/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 extendedset @@ -32,7 +31,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT diff --git a/extensions-contrib/ambari-metrics-emitter/pom.xml b/extensions-contrib/ambari-metrics-emitter/pom.xml index 7d7c2b9dec58..15aac3832eeb 100644 --- a/extensions-contrib/ambari-metrics-emitter/pom.xml +++ b/extensions-contrib/ambari-metrics-emitter/pom.xml @@ -18,14 +18,13 @@ ~ under the License. --> - + 4.0.0 io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/azure-extensions/pom.xml b/extensions-contrib/azure-extensions/pom.xml index 5629af951fd6..e7858fd19ef0 100644 --- a/extensions-contrib/azure-extensions/pom.xml +++ b/extensions-contrib/azure-extensions/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions.contrib @@ -30,7 +29,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/cassandra-storage/pom.xml b/extensions-contrib/cassandra-storage/pom.xml index e5bda9d742ba..fd808ba8d449 100644 --- a/extensions-contrib/cassandra-storage/pom.xml +++ b/extensions-contrib/cassandra-storage/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml index e48db2c3f7c9..132473e91557 100644 --- a/extensions-contrib/cloudfiles-extensions/pom.xml +++ b/extensions-contrib/cloudfiles-extensions/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions.contrib @@ -30,7 +29,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/distinctcount/pom.xml b/extensions-contrib/distinctcount/pom.xml index 336174333923..5c4ec7980820 100644 --- a/extensions-contrib/distinctcount/pom.xml +++ b/extensions-contrib/distinctcount/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions.contrib @@ -30,7 +29,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/druid-rocketmq/pom.xml b/extensions-contrib/druid-rocketmq/pom.xml index 9b91b413e9a5..c4bae43d7f04 100644 --- a/extensions-contrib/druid-rocketmq/pom.xml +++ b/extensions-contrib/druid-rocketmq/pom.xml @@ -17,14 +17,12 @@ ~ specific language governing permissions and limitations ~ under the License. --> - + 4.0.0 druid io.druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/google-extensions/pom.xml b/extensions-contrib/google-extensions/pom.xml index f7193e875203..03a00c7d16e3 100644 --- a/extensions-contrib/google-extensions/pom.xml +++ b/extensions-contrib/google-extensions/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/graphite-emitter/pom.xml b/extensions-contrib/graphite-emitter/pom.xml index 39ba49d236c2..ce8498f7d4a7 100644 --- a/extensions-contrib/graphite-emitter/pom.xml +++ b/extensions-contrib/graphite-emitter/pom.xml @@ -18,14 +18,13 @@ ~ under the License. --> - + 4.0.0 io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml index 2da284dd194b..0142346dbb9e 100644 --- a/extensions-contrib/kafka-eight-simpleConsumer/pom.xml +++ b/extensions-contrib/kafka-eight-simpleConsumer/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 io.druid.extensions.contrib druid-kafka-eight-simple-consumer @@ -27,7 +26,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/kafka-emitter/pom.xml b/extensions-contrib/kafka-emitter/pom.xml index abad6933d891..4a241ec8414b 100644 --- a/extensions-contrib/kafka-emitter/pom.xml +++ b/extensions-contrib/kafka-emitter/pom.xml @@ -18,14 +18,13 @@ ~ under the License. --> - + 4.0.0 io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/orc-extensions/pom.xml b/extensions-contrib/orc-extensions/pom.xml index b0bb2c9b749f..de8787217473 100644 --- a/extensions-contrib/orc-extensions/pom.xml +++ b/extensions-contrib/orc-extensions/pom.xml @@ -15,9 +15,7 @@ ~ See the License for the specific language governing permissions and ~ limitations under the License. --> - + io.druid.extensions.contrib druid-orc-extensions druid-orc-extensions @@ -26,7 +24,7 @@ druid io.druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml 4.0.0 @@ -47,6 +45,7 @@ org.apache.hadoop hadoop-client + compile commons-cli diff --git a/extensions-contrib/parquet-extensions/pom.xml b/extensions-contrib/parquet-extensions/pom.xml index d488f709eb61..a4bbde8a99c3 100644 --- a/extensions-contrib/parquet-extensions/pom.xml +++ b/extensions-contrib/parquet-extensions/pom.xml @@ -1,7 +1,5 @@ - + io.druid.extensions.contrib druid-parquet-extensions druid-parquet-extensions @@ -10,7 +8,7 @@ druid io.druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/rabbitmq/pom.xml b/extensions-contrib/rabbitmq/pom.xml index 51f15b974f14..d5645cd564f1 100644 --- a/extensions-contrib/rabbitmq/pom.xml +++ b/extensions-contrib/rabbitmq/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/scan-query/pom.xml b/extensions-contrib/scan-query/pom.xml index 0d3b78d462df..506ab00bc0e5 100644 --- a/extensions-contrib/scan-query/pom.xml +++ b/extensions-contrib/scan-query/pom.xml @@ -18,14 +18,12 @@ ~ under the License. --> - + io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/sqlserver-metadata-storage/pom.xml b/extensions-contrib/sqlserver-metadata-storage/pom.xml index d8d5939374ae..2398bf882813 100644 --- a/extensions-contrib/sqlserver-metadata-storage/pom.xml +++ b/extensions-contrib/sqlserver-metadata-storage/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/statsd-emitter/pom.xml b/extensions-contrib/statsd-emitter/pom.xml index fde59f8e2cbb..1b578eba0438 100644 --- a/extensions-contrib/statsd-emitter/pom.xml +++ b/extensions-contrib/statsd-emitter/pom.xml @@ -17,13 +17,11 @@ ~ specific language governing permissions and limitations ~ under the License. --> - + druid io.druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/thrift-extensions/pom.xml b/extensions-contrib/thrift-extensions/pom.xml index 369a78b7f2b6..a30fc896d418 100644 --- a/extensions-contrib/thrift-extensions/pom.xml +++ b/extensions-contrib/thrift-extensions/pom.xml @@ -1,7 +1,5 @@ - + io.druid.extensions.contrib druid-thrift-extensions @@ -11,7 +9,7 @@ druid io.druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/time-min-max/pom.xml b/extensions-contrib/time-min-max/pom.xml index c37772726918..06b99043c52f 100644 --- a/extensions-contrib/time-min-max/pom.xml +++ b/extensions-contrib/time-min-max/pom.xml @@ -17,13 +17,11 @@ ~ specific language governing permissions and limitations ~ under the License. --> - + druid io.druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/virtual-columns/pom.xml b/extensions-contrib/virtual-columns/pom.xml index c2c158e0b4ba..d36fe73a5608 100644 --- a/extensions-contrib/virtual-columns/pom.xml +++ b/extensions-contrib/virtual-columns/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/avro-extensions/pom.xml b/extensions-core/avro-extensions/pom.xml index 1b46f490473b..89813ae05868 100644 --- a/extensions-core/avro-extensions/pom.xml +++ b/extensions-core/avro-extensions/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions @@ -30,7 +29,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/caffeine-cache/pom.xml b/extensions-core/caffeine-cache/pom.xml index 5047cb9f8843..9cfd1589ee67 100644 --- a/extensions-core/caffeine-cache/pom.xml +++ b/extensions-core/caffeine-cache/pom.xml @@ -30,7 +30,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml index eaca6d3e5e9e..6084540d2d41 100644 --- a/extensions-core/datasketches/pom.xml +++ b/extensions-core/datasketches/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions @@ -30,7 +29,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/druid-kerberos/pom.xml b/extensions-core/druid-kerberos/pom.xml index a5baa5ba82f2..0de021e2dd5c 100644 --- a/extensions-core/druid-kerberos/pom.xml +++ b/extensions-core/druid-kerberos/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions @@ -30,7 +29,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/hdfs-storage/pom.xml b/extensions-core/hdfs-storage/pom.xml index 8dd014629b99..78523c45cffa 100644 --- a/extensions-core/hdfs-storage/pom.xml +++ b/extensions-core/hdfs-storage/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml @@ -140,11 +140,21 @@ emitter provided - - org.apache.hadoop - hadoop-aws - ${hadoop.compile.version} - + + org.apache.hadoop + hadoop-aws + ${hadoop.compile.version} + + + com.amazonaws + aws-java-sdk + + + + + com.amazonaws + aws-java-sdk-s3 + commons-io commons-io diff --git a/extensions-core/histogram/pom.xml b/extensions-core/histogram/pom.xml index c2a317dd628c..ec1f30377807 100644 --- a/extensions-core/histogram/pom.xml +++ b/extensions-core/histogram/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java index 50d941f0af22..da4b08513751 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java @@ -335,11 +335,11 @@ public void offer(float value) // or merge existing bins before inserting the new one int minPos = minDeltaIndex(); - float minDelta = minPos >= 0 ? positions[minPos + 1] - positions[minPos] : Float.MAX_VALUE; + float minDelta = minPos >= 0 ? positions[minPos + 1] - positions[minPos] : Float.POSITIVE_INFINITY; // determine the distance of new value to the nearest bins - final float deltaRight = insertAt < binCount ? positions[insertAt] - value : Float.MAX_VALUE; - final float deltaLeft = insertAt > 0 ? value - positions[insertAt - 1] : Float.MAX_VALUE; + final float deltaRight = insertAt < binCount ? positions[insertAt] - value : Float.POSITIVE_INFINITY; + final float deltaLeft = insertAt > 0 ? value - positions[insertAt - 1] : Float.POSITIVE_INFINITY; boolean mergeValue = false; if (deltaRight < minDelta) { @@ -368,7 +368,7 @@ public void offer(float value) protected int minDeltaIndex() { // determine minimum distance between existing bins - float minDelta = Float.MAX_VALUE; + float minDelta = Float.POSITIVE_INFINITY; int minPos = -1; for (int i = 0; i < binCount - 1; ++i) { float delta = (positions[i + 1] - positions[i]); @@ -886,9 +886,6 @@ private static void mergeBins( while (i < numMerge) { // find the smallest delta within the range used for bins - // pick minimum delta by scanning array - //int currentIndex = minIndex(deltas, lastValidIndex); - // pick minimum delta index using min-heap int currentIndex = heap[0]; @@ -908,17 +905,13 @@ private static void mergeBins( final float mm0 = (m0 - m1) * w + m1; mergedPositions[currentIndex] = mm0; - //mergedPositions[nextIndex] = Float.MAX_VALUE; // for debugging mergedBins[currentIndex] = sum | APPROX_FLAG_BIT; - //mergedBins[nextIndex] = -1; // for debugging // update deltas and min-heap if (nextIndex == lastValidIndex) { // merged bin is the last => remove the current bin delta from the heap heapSize = heapDelete(heap, reverseIndex, heapSize, reverseIndex[currentIndex], deltas); - - //deltas[currentIndex] = Float.MAX_VALUE; // for debugging } else { // merged bin is not the last => remove the merged bin delta from the heap heapSize = heapDelete(heap, reverseIndex, heapSize, reverseIndex[nextIndex], deltas); @@ -938,9 +931,6 @@ private static void mergeBins( siftDown(heap, reverseIndex, reverseIndex[prevIndex], heapSize - 1, deltas); } - // mark the merged bin as invalid - // deltas[nextIndex] = Float.MAX_VALUE; // for debugging - // update last valid index if we merged the last bin if (nextIndex == lastValidIndex) { lastValidIndex = currentIndex; @@ -1037,7 +1027,7 @@ private static int heapDelete(int[] heap, int[] reverseIndex, int count, int hea private static int minIndex(float[] deltas, int lastValidIndex) { int minIndex = -1; - float min = Float.MAX_VALUE; + float min = Float.POSITIVE_INFINITY; for (int k = 0; k < lastValidIndex; ++k) { float value = deltas[k]; if (value < min) { diff --git a/extensions-core/kafka-eight/pom.xml b/extensions-core/kafka-eight/pom.xml index 83cc8bc7353b..3bf5d61edfb3 100644 --- a/extensions-core/kafka-eight/pom.xml +++ b/extensions-core/kafka-eight/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml index 5e14b1018852..1cef0e4b6047 100644 --- a/extensions-core/kafka-extraction-namespace/pom.xml +++ b/extensions-core/kafka-extraction-namespace/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions druid-kafka-extraction-namespace @@ -29,7 +28,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index b9d6a4e98e37..a751907ed0d0 100644 --- a/extensions-core/kafka-indexing-service/pom.xml +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index b908af860475..fe1214091b4d 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -1549,7 +1549,7 @@ public void close() new TestDataSegmentAnnouncer(), EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, - makeTimeseriesOnlyConglomerate(), + this::makeTimeseriesOnlyConglomerate, MoreExecutors.sameThreadExecutor(), // queryExecutorService EasyMock.createMock(MonitorScheduler.class), new SegmentLoaderFactory( diff --git a/extensions-core/lookups-cached-global/pom.xml b/extensions-core/lookups-cached-global/pom.xml index da9a3e563a2f..46f8cb32dea0 100644 --- a/extensions-core/lookups-cached-global/pom.xml +++ b/extensions-core/lookups-cached-global/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions druid-lookups-cached-global @@ -29,7 +28,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/UriExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/UriExtractionNamespace.java index e36275909c44..1922b7572a5f 100644 --- a/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/UriExtractionNamespace.java +++ b/extensions-core/lookups-cached-global/src/main/java/io/druid/query/lookup/namespace/UriExtractionNamespace.java @@ -29,7 +29,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.base.Throwables; @@ -295,7 +294,7 @@ public CSVFlatDataParser( ); this.parser = new DelegateParser( - new CSVParser(Optional.absent(), columns, hasHeaderRow, skipHeaderRows), + new CSVParser(null, columns, hasHeaderRow, skipHeaderRows), this.keyColumn, this.valueColumn ); @@ -395,8 +394,8 @@ public TSVFlatDataParser( "Must specify more than one column to have a key value pair" ); final DelimitedParser delegate = new DelimitedParser( - Optional.fromNullable(Strings.emptyToNull(delimiter)), - Optional.fromNullable(Strings.emptyToNull(listDelimiter)), + Strings.emptyToNull(delimiter), + Strings.emptyToNull(listDelimiter), hasHeaderRow, skipHeaderRows ); diff --git a/extensions-core/lookups-cached-single/pom.xml b/extensions-core/lookups-cached-single/pom.xml index a424d0530aa2..ee90f2c3d2bb 100644 --- a/extensions-core/lookups-cached-single/pom.xml +++ b/extensions-core/lookups-cached-single/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 io.druid.extensions druid-lookups-cached-single @@ -29,7 +28,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/mysql-metadata-storage/pom.xml b/extensions-core/mysql-metadata-storage/pom.xml index 59d65deae3d8..a64fae95dd7e 100644 --- a/extensions-core/mysql-metadata-storage/pom.xml +++ b/extensions-core/mysql-metadata-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/postgresql-metadata-storage/pom.xml b/extensions-core/postgresql-metadata-storage/pom.xml index 737a9be8bc86..25c083691ee4 100644 --- a/extensions-core/postgresql-metadata-storage/pom.xml +++ b/extensions-core/postgresql-metadata-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/protobuf-extensions/pom.xml b/extensions-core/protobuf-extensions/pom.xml index 01173413f238..321a12c01b4f 100644 --- a/extensions-core/protobuf-extensions/pom.xml +++ b/extensions-core/protobuf-extensions/pom.xml @@ -16,9 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 io.druid.extensions @@ -29,7 +27,7 @@ druid io.druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/s3-extensions/pom.xml b/extensions-core/s3-extensions/pom.xml index 35ea4c65e650..b7d11d105335 100644 --- a/extensions-core/s3-extensions/pom.xml +++ b/extensions-core/s3-extensions/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 io.druid.extensions @@ -28,7 +27,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/extensions-core/stats/pom.xml b/extensions-core/stats/pom.xml index df5db8eab11a..d34500c28873 100644 --- a/extensions-core/stats/pom.xml +++ b/extensions-core/stats/pom.xml @@ -29,7 +29,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT ../../pom.xml diff --git a/hll/pom.xml b/hll/pom.xml index eca0a3f8a065..d21bbf18d166 100644 --- a/hll/pom.xml +++ b/hll/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT druid-hll diff --git a/hll/src/main/java/io/druid/hll/HyperLogLogCollector.java b/hll/src/main/java/io/druid/hll/HyperLogLogCollector.java index f5465bb53e1c..e09f35d602ed 100644 --- a/hll/src/main/java/io/druid/hll/HyperLogLogCollector.java +++ b/hll/src/main/java/io/druid/hll/HyperLogLogCollector.java @@ -146,7 +146,7 @@ public static double applyCorrection(double e, int zeroCount) final double ratio = e / TWO_TO_THE_SIXTY_FOUR; if (ratio >= 1) { // handle very unlikely case that value is > 2^64 - return Double.MAX_VALUE; + return Double.POSITIVE_INFINITY; } else { return -TWO_TO_THE_SIXTY_FOUR * Math.log(1 - ratio); } diff --git a/hll/src/test/java/io/druid/hll/HyperLogLogCollectorTest.java b/hll/src/test/java/io/druid/hll/HyperLogLogCollectorTest.java index 5c7744963ed8..fc3b05ae4a69 100644 --- a/hll/src/test/java/io/druid/hll/HyperLogLogCollectorTest.java +++ b/hll/src/test/java/io/druid/hll/HyperLogLogCollectorTest.java @@ -693,7 +693,7 @@ public void testHighBits() throws Exception fillBuckets(collector, (byte) 0, (byte) 63); collector.add(new byte[]{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}); - Assert.assertEquals(Double.MAX_VALUE, collector.estimateCardinality(), 1000); + Assert.assertEquals(Double.POSITIVE_INFINITY, collector.estimateCardinality(), 1000); } @Test diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 6e9f5ecece0a..ed9ef0cf88c8 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java index e64c0e7884df..b131a5ed5d7d 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java @@ -267,7 +267,7 @@ public HadoopTuningConfig withWorkingPath(String path) numBackgroundPersistThreads, forceExtendableShardSpecs, useExplicitVersion, - null + allowedHadoopPrefix ); } @@ -292,7 +292,7 @@ public HadoopTuningConfig withVersion(String ver) numBackgroundPersistThreads, forceExtendableShardSpecs, useExplicitVersion, - null + allowedHadoopPrefix ); } @@ -317,7 +317,7 @@ public HadoopTuningConfig withShardSpecs(Map> specs numBackgroundPersistThreads, forceExtendableShardSpecs, useExplicitVersion, - null + allowedHadoopPrefix ); } diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 2308670cd8c0..8994f1686e13 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index 8671432b77be..e9461d98ae35 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -26,6 +26,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.Multimap; import com.google.common.collect.Multimaps; +import com.google.inject.Provider; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import io.druid.client.cache.Cache; @@ -33,7 +34,6 @@ import io.druid.indexing.common.actions.SegmentInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.config.TaskConfig; -import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; @@ -63,7 +63,6 @@ public class TaskToolbox { private final TaskConfig config; - private final Task task; private final TaskActionClient taskActionClient; private final ServiceEmitter emitter; private final DataSegmentPusher segmentPusher; @@ -73,7 +72,12 @@ public class TaskToolbox private final DataSegmentAnnouncer segmentAnnouncer; private final DataSegmentServerAnnouncer serverAnnouncer; private final SegmentHandoffNotifierFactory handoffNotifierFactory; - private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate; + /** + * Using Provider, not {@link QueryRunnerFactoryConglomerate} directly, to not require {@link + * io.druid.indexing.overlord.TaskRunner} implementations that create TaskToolboxes to inject query stuff eagerly, + * because it may be unavailable, e. g. for batch tasks running in Spark or Hadoop. + */ + private final Provider queryRunnerFactoryConglomerateProvider; private final MonitorScheduler monitorScheduler; private final ExecutorService queryExecutorService; private final SegmentLoader segmentLoader; @@ -87,7 +91,6 @@ public class TaskToolbox public TaskToolbox( TaskConfig config, - Task task, TaskActionClient taskActionClient, ServiceEmitter emitter, DataSegmentPusher segmentPusher, @@ -97,7 +100,7 @@ public TaskToolbox( DataSegmentAnnouncer segmentAnnouncer, DataSegmentServerAnnouncer serverAnnouncer, SegmentHandoffNotifierFactory handoffNotifierFactory, - QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate, + Provider queryRunnerFactoryConglomerateProvider, ExecutorService queryExecutorService, MonitorScheduler monitorScheduler, SegmentLoader segmentLoader, @@ -111,7 +114,6 @@ public TaskToolbox( ) { this.config = config; - this.task = task; this.taskActionClient = taskActionClient; this.emitter = emitter; this.segmentPusher = segmentPusher; @@ -121,7 +123,7 @@ public TaskToolbox( this.segmentAnnouncer = segmentAnnouncer; this.serverAnnouncer = serverAnnouncer; this.handoffNotifierFactory = handoffNotifierFactory; - this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate; + this.queryRunnerFactoryConglomerateProvider = queryRunnerFactoryConglomerateProvider; this.queryExecutorService = queryExecutorService; this.monitorScheduler = monitorScheduler; this.segmentLoader = segmentLoader; @@ -186,7 +188,7 @@ public SegmentHandoffNotifierFactory getSegmentHandoffNotifierFactory() public QueryRunnerFactoryConglomerate getQueryRunnerFactoryConglomerate() { - return queryRunnerFactoryConglomerate; + return queryRunnerFactoryConglomerateProvider.get(); } public ExecutorService getQueryExecutorService() diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index 7527d1b8761b..44be3e9ebcaa 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.inject.Inject; +import com.google.inject.Provider; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import io.druid.client.cache.Cache; @@ -60,7 +61,7 @@ public class TaskToolboxFactory private final DataSegmentAnnouncer segmentAnnouncer; private final DataSegmentServerAnnouncer serverAnnouncer; private final SegmentHandoffNotifierFactory handoffNotifierFactory; - private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate; + private final Provider queryRunnerFactoryConglomerateProvider; private final ExecutorService queryExecutorService; private final MonitorScheduler monitorScheduler; private final SegmentLoaderFactory segmentLoaderFactory; @@ -83,7 +84,7 @@ public TaskToolboxFactory( DataSegmentAnnouncer segmentAnnouncer, DataSegmentServerAnnouncer serverAnnouncer, SegmentHandoffNotifierFactory handoffNotifierFactory, - QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate, + Provider queryRunnerFactoryConglomerateProvider, @Processing ExecutorService queryExecutorService, MonitorScheduler monitorScheduler, SegmentLoaderFactory segmentLoaderFactory, @@ -105,7 +106,7 @@ public TaskToolboxFactory( this.segmentAnnouncer = segmentAnnouncer; this.serverAnnouncer = serverAnnouncer; this.handoffNotifierFactory = handoffNotifierFactory; - this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate; + this.queryRunnerFactoryConglomerateProvider = queryRunnerFactoryConglomerateProvider; this.queryExecutorService = queryExecutorService; this.monitorScheduler = monitorScheduler; this.segmentLoaderFactory = segmentLoaderFactory; @@ -122,7 +123,6 @@ public TaskToolbox build(Task task) final File taskWorkDir = config.getTaskWorkDir(task.getId()); return new TaskToolbox( config, - task, taskActionClientFactory.create(task), emitter, segmentPusher, @@ -132,7 +132,7 @@ public TaskToolbox build(Task task) segmentAnnouncer, serverAnnouncer, handoffNotifierFactory, - queryRunnerFactoryConglomerate, + queryRunnerFactoryConglomerateProvider, queryExecutorService, monitorScheduler, segmentLoaderFactory.manufacturate(taskWorkDir), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index f6774843ee7c..2f2862fdbe80 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -261,6 +261,7 @@ private Map> determineShardSpecs( // determine intervals containing data and prime HLL collectors final Map> hllCollectors = Maps.newHashMap(); int thrownAway = 0; + int unparseable = 0; log.info("Determining intervals and shardSpecs"); long determineShardSpecsStartMillis = System.currentTimeMillis(); @@ -269,48 +270,61 @@ private Map> determineShardSpecs( firehoseTempDir) ) { while (firehose.hasMore()) { - final InputRow inputRow = firehose.nextRow(); + try { + final InputRow inputRow = firehose.nextRow(); - // The null inputRow means the caller must skip this row. - if (inputRow == null) { - continue; - } + // The null inputRow means the caller must skip this row. + if (inputRow == null) { + continue; + } - final Interval interval; - if (determineIntervals) { - interval = granularitySpec.getSegmentGranularity().bucket(inputRow.getTimestamp()); - } else { - final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); - if (!optInterval.isPresent()) { - thrownAway++; + final Interval interval; + if (determineIntervals) { + interval = granularitySpec.getSegmentGranularity().bucket(inputRow.getTimestamp()); + } else { + final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); + if (!optInterval.isPresent()) { + thrownAway++; + continue; + } + interval = optInterval.get(); + } + + if (!determineNumPartitions) { + // we don't need to determine partitions but we still need to determine intervals, so add an Optional.absent() + // for the interval and don't instantiate a HLL collector + if (!hllCollectors.containsKey(interval)) { + hllCollectors.put(interval, Optional.absent()); + } continue; } - interval = optInterval.get(); - } - if (!determineNumPartitions) { - // we don't need to determine partitions but we still need to determine intervals, so add an Optional.absent() - // for the interval and don't instantiate a HLL collector if (!hllCollectors.containsKey(interval)) { - hllCollectors.put(interval, Optional.absent()); + hllCollectors.put(interval, Optional.of(HyperLogLogCollector.makeLatestCollector())); } - continue; - } - if (!hllCollectors.containsKey(interval)) { - hllCollectors.put(interval, Optional.of(HyperLogLogCollector.makeLatestCollector())); + List groupKey = Rows.toGroupKey( + queryGranularity.bucketStart(inputRow.getTimestamp()).getMillis(), + inputRow + ); + hllCollectors.get(interval).get().add(hashFunction.hashBytes(jsonMapper.writeValueAsBytes(groupKey)).asBytes()); + } + catch (ParseException e) { + if (ingestionSchema.getTuningConfig().isReportParseExceptions()) { + throw e; + } else { + unparseable++; + } } - - List groupKey = Rows.toGroupKey( - queryGranularity.bucketStart(inputRow.getTimestamp()).getMillis(), - inputRow - ); - hllCollectors.get(interval).get().add(hashFunction.hashBytes(jsonMapper.writeValueAsBytes(groupKey)).asBytes()); } } + // These metrics are reported in generateAndPublishSegments() if (thrownAway > 0) { - log.warn("Unable to to find a matching interval for [%,d] events", thrownAway); + log.warn("Unable to find a matching interval for [%,d] events", thrownAway); + } + if (unparseable > 0) { + log.warn("Unable to parse [%,d] events", unparseable); } final ImmutableSortedMap> sortedMap = ImmutableSortedMap.copyOf( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java index 76e23a4f393f..6dc827306ac8 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java @@ -105,7 +105,7 @@ public void setUp() throws IOException mockSegmentAnnouncer, EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), mockHandoffNotifierFactory, - mockQueryRunnerFactoryConglomerate, + () -> mockQueryRunnerFactoryConglomerate, mockQueryExecutorService, mockMonitorScheduler, new SegmentLoaderFactory(mockSegmentLoaderLocalCacheManager), diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 2881ab0c5196..29478292727b 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import com.google.common.io.Files; import io.druid.data.input.impl.CSVParseSpec; import io.druid.data.input.impl.DimensionsSpec; @@ -37,8 +38,11 @@ import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskAction; import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import io.druid.indexing.overlord.SegmentPublishResult; +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; +import io.druid.java.util.common.parsers.ParseException; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.segment.IndexIO; @@ -62,6 +66,7 @@ import org.junit.Assert; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import java.io.BufferedWriter; @@ -71,14 +76,19 @@ import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; public class IndexTaskTest { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule + public ExpectedException expectedException = ExpectedException.none(); + private static final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec( new TimestampSpec( "ts", @@ -446,7 +456,6 @@ public void testCSVFileWithHeaderColumnOverride() throws Exception writer.write("2014-01-01T00:00:10Z,a,1\n"); } - IndexTask indexTask = new IndexTask( null, null, @@ -482,18 +491,260 @@ public void testCSVFileWithHeaderColumnOverride() throws Exception Assert.assertEquals(1, segments.size()); - Assert.assertEquals(Arrays.asList("dim"), segments.get(0).getDimensions()); + Assert.assertEquals(Arrays.asList("d"), segments.get(0).getDimensions()); + Assert.assertEquals(Arrays.asList("val"), segments.get(0).getMetrics()); + Assert.assertEquals(new Interval("2014/P1D"), segments.get(0).getInterval()); + } + + @Test + public void testIgnoreParseException() throws Exception + { + final File tmpDir = temporaryFolder.newFolder(); + + final File tmpFile = File.createTempFile("druid", "index", tmpDir); + + try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { + writer.write("time,d,val\n"); + writer.write("unparseable,a,1\n"); + writer.write("2014-01-01T00:00:10Z,a,1\n"); + } + + // GranularitySpec.intervals and numShards must be null to verify reportParseException=false is respected both in + // IndexTask.determineShardSpecs() and IndexTask.generateAndPublishSegments() + final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + tmpDir, + new CSVParseSpec( + new TimestampSpec( + "time", + "auto", + null + ), + new DimensionsSpec( + null, + Lists.newArrayList(), + Lists.newArrayList() + ), + null, + Arrays.asList("time", "dim", "val"), + true, + 0 + ), + null, + 2, + null, + false, + false, + false // ignore parse exception + ); + + IndexTask indexTask = new IndexTask( + null, + null, + parseExceptionIgnoreSpec, + null, + jsonMapper + ); + + final List segments = runTask(indexTask); + + Assert.assertEquals(Arrays.asList("d"), segments.get(0).getDimensions()); Assert.assertEquals(Arrays.asList("val"), segments.get(0).getMetrics()); Assert.assertEquals(new Interval("2014/P1D"), segments.get(0).getInterval()); } + @Test + public void testReportParseException() throws Exception + { + expectedException.expect(ParseException.class); + expectedException.expectMessage("Unparseable timestamp found!"); + + final File tmpDir = temporaryFolder.newFolder(); + + final File tmpFile = File.createTempFile("druid", "index", tmpDir); + + try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { + writer.write("time,d,val\n"); + writer.write("unparseable,a,1\n"); + writer.write("2014-01-01T00:00:10Z,a,1\n"); + } + + final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + tmpDir, + new CSVParseSpec( + new TimestampSpec( + "time", + "auto", + null + ), + new DimensionsSpec( + null, + Lists.newArrayList(), + Lists.newArrayList() + ), + null, + Arrays.asList("time", "dim", "val"), + true, + 0 + ), + null, + 2, + null, + false, + false, + true // report parse exception + ); + + IndexTask indexTask = new IndexTask( + null, + null, + parseExceptionIgnoreSpec, + null, + jsonMapper + ); + + runTask(indexTask); + } + + @Test + public void testCsvWithHeaderOfEmptyColumns() throws Exception + { + final File tmpDir = temporaryFolder.newFolder(); + + File tmpFile = File.createTempFile("druid", "index", tmpDir); + + try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { + writer.write("time,,\n"); + writer.write("2014-01-01T00:00:10Z,a,1\n"); + } + + tmpFile = File.createTempFile("druid", "index", tmpDir); + + try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { + writer.write("time,dim,\n"); + writer.write("2014-01-01T00:00:10Z,a,1\n"); + } + + tmpFile = File.createTempFile("druid", "index", tmpDir); + + try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { + writer.write("time,,val\n"); + writer.write("2014-01-01T00:00:10Z,a,1\n"); + } + + final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + tmpDir, + new CSVParseSpec( + new TimestampSpec( + "time", + "auto", + null + ), + new DimensionsSpec( + null, + null, + null + ), + null, + null, + true, + 0 + ), + null, + 2, + null, + false, + false, + true // report parse exception + ); + + IndexTask indexTask = new IndexTask( + null, + null, + parseExceptionIgnoreSpec, + null, + jsonMapper + ); + + final List segments = runTask(indexTask); + // the order of result segments can be changed because hash shardSpec is used. + // the below loop is to make this test deterministic. + Assert.assertEquals(2, segments.size()); + Assert.assertNotEquals(segments.get(0), segments.get(1)); + + for (int i = 0; i < 2; i++) { + final DataSegment segment = segments.get(i); + final Set dimensions = new HashSet<>(segment.getDimensions()); + + Assert.assertTrue( + StringUtils.safeFormat("Actual dimensions: %s", dimensions), + dimensions.equals(Sets.newHashSet("dim", "column_3")) || + dimensions.equals(Sets.newHashSet("column_2", "column_3")) + ); + + Assert.assertEquals(Arrays.asList("val"), segment.getMetrics()); + Assert.assertEquals(new Interval("2014/P1D"), segment.getInterval()); + } + } + + @Test + public void testCsvWithHeaderOfEmptyTimestamp() throws Exception + { + expectedException.expect(ParseException.class); + expectedException.expectMessage("Unparseable timestamp found!"); + + final File tmpDir = temporaryFolder.newFolder(); + + final File tmpFile = File.createTempFile("druid", "index", tmpDir); + + try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { + writer.write(",,\n"); + writer.write("2014-01-01T00:00:10Z,a,1\n"); + } + + final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + tmpDir, + new CSVParseSpec( + new TimestampSpec( + "time", + "auto", + null + ), + new DimensionsSpec( + null, + Lists.newArrayList(), + Lists.newArrayList() + ), + null, + Arrays.asList("time", "", ""), + true, + 0 + ), + null, + 2, + null, + false, + false, + true // report parse exception + ); + + IndexTask indexTask = new IndexTask( + null, + null, + parseExceptionIgnoreSpec, + null, + jsonMapper + ); + + runTask(indexTask); + } + private final List runTask(final IndexTask indexTask) throws Exception { final List segments = Lists.newArrayList(); indexTask.run( new TaskToolbox( - null, null, new TaskActionClient() + null, new TaskActionClient() { @Override public RetType submit(TaskAction taskAction) throws IOException @@ -577,6 +828,29 @@ private IndexTask.IndexIngestionSpec createIngestionSpec( boolean forceExtendableShardSpecs, boolean appendToExisting ) + { + return createIngestionSpec( + baseDir, + parseSpec, + granularitySpec, + targetPartitionSize, + numShards, + forceExtendableShardSpecs, + appendToExisting, + true + ); + } + + private IndexTask.IndexIngestionSpec createIngestionSpec( + File baseDir, + ParseSpec parseSpec, + GranularitySpec granularitySpec, + Integer targetPartitionSize, + Integer numShards, + boolean forceExtendableShardSpecs, + boolean appendToExisting, + boolean reportParseException + ) { return new IndexTask.IndexIngestionSpec( new DataSchema( @@ -614,7 +888,7 @@ private IndexTask.IndexIngestionSpec createIngestionSpec( null, true, forceExtendableShardSpecs, - true, + reportParseException, null ) ); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 78c48b0734dd..bd8a001853c9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -1045,7 +1045,7 @@ Map> getHandOffCallbacks() new TestDataSegmentAnnouncer(), EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, - conglomerate, + () -> conglomerate, MoreExecutors.sameThreadExecutor(), // queryExecutorService EasyMock.createMock(MonitorScheduler.class), new SegmentLoaderFactory( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java index 9f2bd7f78232..4f75c9962cf8 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java @@ -141,7 +141,7 @@ public RetType submit(TaskAction taskAction) throws IOExcepti mergeTask.run( new TaskToolbox( - null, null, new TaskActionClient() + null, new TaskActionClient() { @Override public RetType submit(TaskAction taskAction) throws IOException diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index b001cfceda8c..0c76f718b659 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -577,7 +577,7 @@ public void unannounceSegments(Iterable segments) throws IOExceptio }, // segment announcer EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, - queryRunnerFactoryConglomerate, // query runner factory conglomerate corporation unionized collective + () -> queryRunnerFactoryConglomerate, // query runner factory conglomerate corporation unionized collective MoreExecutors.sameThreadExecutor(), // query executor service monitorScheduler, // monitor scheduler new SegmentLoaderFactory( diff --git a/integration-tests/docker/router.conf b/integration-tests/docker/router.conf index 17d5146bdf7b..c24c6995562f 100644 --- a/integration-tests/docker/router.conf +++ b/integration-tests/docker/router.conf @@ -9,9 +9,7 @@ command=java -Dfile.encoding=UTF-8 -Ddruid.host=%(ENV_HOST_IP)s -Ddruid.zk.service.host=druid-zookeeper-kafka - -Ddruid.computation.buffer.size=75000000 -Ddruid.server.http.numThreads=100 - -Ddruid.processing.numThreads=1 -cp /usr/local/druid/lib/* io.druid.cli.Main server router redirect_stderr=true diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 981811065f9f..7597a421819b 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java index dbdc49f01074..f83b2f44179f 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java @@ -63,7 +63,7 @@ public class ITUnionQueryTest extends AbstractIndexerTest @Test public void testUnionQuery() throws Exception { - final int numTasks = 4; + final int numTasks = 3; try { // Load 4 datasources with same dimensions diff --git a/integration-tests/src/test/resources/indexer/union_queries.json b/integration-tests/src/test/resources/indexer/union_queries.json index 08b7776e7ab9..ec13d507d42f 100644 --- a/integration-tests/src/test/resources/indexer/union_queries.json +++ b/integration-tests/src/test/resources/indexer/union_queries.json @@ -53,11 +53,11 @@ { "timestamp": "2013-08-31T01:02:33.000Z", "result": { - "added": 2064.0, - "count": 8, - "delta": 748.0, - "deleted": 1316.0, - "rows": 8 + "added": 1548.0, + "count": 6, + "delta": 561.0, + "deleted": 987.0, + "rows": 6 } } ] @@ -115,28 +115,28 @@ "timestamp": "2013-08-31T01:02:33.000Z", "result": [ { - "added": 3620.0, - "count": 4, + "added": 2715.0, + "count": 3, "page": "Crimson Typhoon", - "delta": 3600.0, - "deleted": 20.0, - "rows": 4 + "delta": 2700.0, + "deleted": 15.0, + "rows": 3 }, { - "added": 1836.0, - "count": 4, + "added": 1377.0, + "count": 3, "page": "Striker Eureka", - "delta": 1320.0, - "deleted": 516.0, - "rows": 4 + "delta": 990.0, + "deleted": 387.0, + "rows": 3 }, { - "added": 492.0, - "count": 4, + "added": 369.0, + "count": 3, "page": "Cherno Alpha", - "delta": 444.0, - "deleted": 48.0, - "rows": 4 + "delta": 333.0, + "deleted": 36.0, + "rows": 3 } ] } @@ -226,31 +226,31 @@ "timestamp": "2013-08-31T01:02:33.000Z", "result": [ { - "added": 3620.0, - "count": 4, + "added": 2715.0, + "count": 3, "page": "Crimson Typhoon", - "delta": 3600.0, - "deleted": 20.0, - "sumOfAddedDeletedConst": 4640.0, - "rows": 4 + "delta": 2700.0, + "deleted": 15.0, + "sumOfAddedDeletedConst": 3730.0, + "rows": 3 }, { - "added": 1836.0, - "count": 4, + "added": 1377.0, + "count": 3, "page": "Striker Eureka", - "delta": 1320.0, - "deleted": 516.0, - "sumOfAddedDeletedConst": 3352.0, - "rows": 4 + "delta": 990.0, + "deleted": 387.0, + "sumOfAddedDeletedConst": 2764.0, + "rows": 3 }, { - "added": 492.0, - "count": 4, + "added": 369.0, + "count": 3, "page": "Cherno Alpha", - "delta": 444.0, - "deleted": 48.0, - "sumOfAddedDeletedConst": 1540.0, - "rows": 4 + "delta": 333.0, + "deleted": 36.0, + "sumOfAddedDeletedConst": 1405.0, + "rows": 3 } ] } @@ -316,22 +316,22 @@ "timestamp": "2013-08-31T01:02:33.000Z", "result": [ { - "sumOfRowsAndCount": 16.0, - "count": 8, + "sumOfRowsAndCount": 12.0, + "count": 6, "language": "en", - "rows": 8 + "rows": 6 }, { - "sumOfRowsAndCount": 8.0, - "count": 4, + "sumOfRowsAndCount": 6.0, + "count": 3, "language": "ja", - "rows": 4 + "rows": 3 }, { - "sumOfRowsAndCount": 8.0, - "count": 4, + "sumOfRowsAndCount": 6.0, + "count": 3, "language": "ru", - "rows": 4 + "rows": 3 } ] } @@ -392,9 +392,9 @@ "version": "v1", "timestamp": "2013-08-31T00:00:00.000Z", "event": { - "sumOfRowsAndCount": 16.0, - "count": 8, - "rows": 8, + "sumOfRowsAndCount": 12.0, + "count": 6, + "rows": 6, "namespace": "article" } }, @@ -402,9 +402,9 @@ "version": "v1", "timestamp": "2013-08-31T00:00:00.000Z", "event": { - "sumOfRowsAndCount": 24.0, - "count": 12, - "rows": 12, + "sumOfRowsAndCount": 18.0, + "count": 9, + "rows": 9, "namespace": "wikipedia" } } @@ -470,10 +470,10 @@ "version": "v1", "timestamp": "2013-08-31T00:00:00.000Z", "event": { - "sumOfRowsAndCount": 8.0, - "count": 4, + "sumOfRowsAndCount": 6.0, + "count": 3, "robot": "false", - "rows": 4, + "rows": 3, "namespace": "article" } }, @@ -481,10 +481,10 @@ "version": "v1", "timestamp": "2013-08-31T00:00:00.000Z", "event": { - "sumOfRowsAndCount": 8.0, - "count": 4, + "sumOfRowsAndCount": 6.0, + "count": 3, "robot": "true", - "rows": 4, + "rows": 3, "namespace": "article" } }, @@ -492,10 +492,10 @@ "version": "v1", "timestamp": "2013-08-31T00:00:00.000Z", "event": { - "sumOfRowsAndCount": 24.0, - "count": 12, + "sumOfRowsAndCount": 18.0, + "count": 9, "robot": "true", - "rows": 12, + "rows": 9, "namespace": "wikipedia" } } @@ -530,12 +530,12 @@ { "dimension": "user", "value": "triplets", - "count":4 + "count":3 }, { "dimension": "namespace", "value": "wikipedia", - "count":12 + "count":9 } ] } diff --git a/java-util/pom.xml b/java-util/pom.xml index cc7d858b7536..00c1e4d82dd3 100644 --- a/java-util/pom.xml +++ b/java-util/pom.xml @@ -24,7 +24,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT java-util diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/ExecutorServiceConfig.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/ExecutorServiceConfig.java index 2b0ae7756f5f..9c4c139a75be 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/ExecutorServiceConfig.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/ExecutorServiceConfig.java @@ -26,13 +26,31 @@ */ public abstract class ExecutorServiceConfig { + public static final int DEFAULT_NUM_THREADS = -1; + @Config(value = "${base_path}.formatString") @Default("processing-%s") public abstract String getFormatString(); - @Config(value = "${base_path}.numThreads") public int getNumThreads() { - return Runtime.getRuntime().availableProcessors() - 1; + int numThreadsConfigured = getNumThreadsConfigured(); + if (numThreadsConfigured != DEFAULT_NUM_THREADS) { + return numThreadsConfigured; + } else { + return Math.max(Runtime.getRuntime().availableProcessors() - 1, 1); + } + } + + /** + * Returns the number of threads _explicitly_ configured, or -1 if it is not explicitly configured, that is not + * a valid number of threads. To get the configured value or the default (valid) number, use {@link #getNumThreads()}. + * This method exists for ability to distinguish between the default value set when there is no explicit config, and + * an explicitly configured value. + */ + @Config(value = "${base_path}.numThreads") + public int getNumThreadsConfigured() + { + return DEFAULT_NUM_THREADS; } } diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/AbstractFlatTextFormatParser.java b/java-util/src/main/java/io/druid/java/util/common/parsers/AbstractFlatTextFormatParser.java new file mode 100644 index 000000000000..7d3f43c9c336 --- /dev/null +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/AbstractFlatTextFormatParser.java @@ -0,0 +1,165 @@ +/* + * 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.java.util.common.parsers; + +import com.google.common.base.Function; +import com.google.common.base.Splitter; +import com.google.common.base.Strings; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import io.druid.java.util.common.collect.Utils; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public abstract class AbstractFlatTextFormatParser implements Parser +{ + public enum FlatTextFormat + { + CSV(","), + DELIMITED("\t"); + + private final String defaultDelimiter; + + FlatTextFormat(String defaultDelimiter) + { + this.defaultDelimiter = defaultDelimiter; + } + + public String getDefaultDelimiter() + { + return defaultDelimiter; + } + } + + private final String listDelimiter; + private final Splitter listSplitter; + private final Function valueFunction; + private final boolean hasHeaderRow; + private final int maxSkipHeaderRows; + + private List fieldNames = null; + private boolean hasParsedHeader = false; + private int skippedHeaderRows; + private boolean supportSkipHeaderRows; + + public AbstractFlatTextFormatParser( + @Nullable final String listDelimiter, + final boolean hasHeaderRow, + final int maxSkipHeaderRows + ) + { + this.listDelimiter = listDelimiter != null ? listDelimiter : Parsers.DEFAULT_LIST_DELIMITER; + this.listSplitter = Splitter.on(this.listDelimiter); + this.valueFunction = ParserUtils.getMultiValueFunction(this.listDelimiter, this.listSplitter); + + this.hasHeaderRow = hasHeaderRow; + this.maxSkipHeaderRows = maxSkipHeaderRows; + } + + @Override + public void startFileFromBeginning() + { + if (hasHeaderRow) { + fieldNames = null; + } + hasParsedHeader = false; + skippedHeaderRows = 0; + supportSkipHeaderRows = true; + } + + public String getListDelimiter() + { + return listDelimiter; + } + + @Override + public List getFieldNames() + { + return fieldNames; + } + + @Override + public void setFieldNames(final Iterable fieldNames) + { + if (fieldNames != null) { + final List fieldsList = Lists.newArrayList(fieldNames); + this.fieldNames = new ArrayList<>(fieldsList.size()); + for (int i = 0; i < fieldsList.size(); i++) { + if (Strings.isNullOrEmpty(fieldsList.get(i))) { + this.fieldNames.add(ParserUtils.getDefaultColumnName(i)); + } else { + this.fieldNames.add(fieldsList.get(i)); + } + } + ParserUtils.validateFields(this.fieldNames); + } + } + + public void setFieldNames(final String header) + { + try { + setFieldNames(parseLine(header)); + } + catch (Exception e) { + throw new ParseException(e, "Unable to parse header [%s]", header); + } + } + + @Override + public Map parse(final String input) + { + if (!supportSkipHeaderRows && (hasHeaderRow || maxSkipHeaderRows > 0)) { + throw new UnsupportedOperationException("hasHeaderRow or maxSkipHeaderRows is not supported. " + + "Please check the indexTask supports these options."); + } + + try { + List values = parseLine(input); + + if (skippedHeaderRows < maxSkipHeaderRows) { + skippedHeaderRows++; + return null; + } + + if (hasHeaderRow && !hasParsedHeader) { + if (fieldNames == null) { + setFieldNames(values); + } + hasParsedHeader = true; + return null; + } + + if (fieldNames == null) { + setFieldNames(ParserUtils.generateFieldNames(values.size())); + } + + return Utils.zipMapPartial(fieldNames, Iterables.transform(values, valueFunction)); + } + catch (Exception e) { + throw new ParseException(e, "Unable to parse row [%s]", input); + } + } + + protected abstract List parseLine(String input) throws IOException; +} diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/CSVParser.java b/java-util/src/main/java/io/druid/java/util/common/parsers/CSVParser.java index 957e72560d2a..c5d18ae41ced 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/CSVParser.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/CSVParser.java @@ -20,73 +20,27 @@ package io.druid.java.util.common.parsers; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Optional; -import com.google.common.base.Splitter; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import io.druid.java.util.common.collect.Utils; -import java.util.ArrayList; +import javax.annotation.Nullable; +import java.io.IOException; import java.util.Arrays; import java.util.List; -import java.util.Map; -public class CSVParser implements Parser +public class CSVParser extends AbstractFlatTextFormatParser { - private static final Function getValueFunction( - final String listDelimiter, - final Splitter listSplitter - ) - { - return new Function() - { - @Override - public Object apply(String input) - { - if (input.contains(listDelimiter)) { - return Lists.newArrayList( - Iterables.transform( - listSplitter.split(input), - ParserUtils.nullEmptyStringFunction - ) - ); - } else { - return ParserUtils.nullEmptyStringFunction.apply(input); - } - } - }; - } - - private final String listDelimiter; - private final Splitter listSplitter; - private final Function valueFunction; - private final boolean hasHeaderRow; - private final int maxSkipHeaderRows; - private final au.com.bytecode.opencsv.CSVParser parser = new au.com.bytecode.opencsv.CSVParser(); - private ArrayList fieldNames = null; - private boolean hasParsedHeader = false; - private int skippedHeaderRows; - private boolean supportSkipHeaderRows; - public CSVParser( - final Optional listDelimiter, + @Nullable final String listDelimiter, final boolean hasHeaderRow, final int maxSkipHeaderRows ) { - this.listDelimiter = listDelimiter.isPresent() ? listDelimiter.get() : Parsers.DEFAULT_LIST_DELIMITER; - this.listSplitter = Splitter.on(this.listDelimiter); - this.valueFunction = getValueFunction(this.listDelimiter, this.listSplitter); - - this.hasHeaderRow = hasHeaderRow; - this.maxSkipHeaderRows = maxSkipHeaderRows; + super(listDelimiter, hasHeaderRow, maxSkipHeaderRows); } public CSVParser( - final Optional listDelimiter, + @Nullable final String listDelimiter, final Iterable fieldNames, final boolean hasHeaderRow, final int maxSkipHeaderRows @@ -97,84 +51,17 @@ public CSVParser( setFieldNames(fieldNames); } - @VisibleForTesting - CSVParser(final Optional listDelimiter, final String header) - { - this(listDelimiter, false, 0); - - setFieldNames(header); - } - - public String getListDelimiter() - { - return listDelimiter; - } - - @Override - public void startFileFromBeginning() - { - supportSkipHeaderRows = true; - hasParsedHeader = false; - skippedHeaderRows = 0; - } - @Override - public List getFieldNames() - { - return fieldNames; - } - - @Override - public void setFieldNames(final Iterable fieldNames) - { - if (fieldNames != null) { - ParserUtils.validateFields(fieldNames); - this.fieldNames = Lists.newArrayList(fieldNames); - } - } - - public void setFieldNames(final String header) + protected List parseLine(String input) throws IOException { - try { - setFieldNames(Arrays.asList(parser.parseLine(header))); - } - catch (Exception e) { - throw new ParseException(e, "Unable to parse header [%s]", header); - } + return Arrays.asList(parser.parseLine(input)); } - @Override - public Map parse(final String input) + @VisibleForTesting + CSVParser(@Nullable final String listDelimiter, final String header) { - if (!supportSkipHeaderRows && (hasHeaderRow || maxSkipHeaderRows > 0)) { - throw new UnsupportedOperationException("hasHeaderRow or maxSkipHeaderRows is not supported. " - + "Please check the indexTask supports these options."); - } - - try { - String[] values = parser.parseLine(input); - - if (skippedHeaderRows < maxSkipHeaderRows) { - skippedHeaderRows++; - return null; - } - - if (hasHeaderRow && !hasParsedHeader) { - if (fieldNames == null) { - setFieldNames(Arrays.asList(values)); - } - hasParsedHeader = true; - return null; - } - - if (fieldNames == null) { - setFieldNames(ParserUtils.generateFieldNames(values.length)); - } + this(listDelimiter, false, 0); - return Utils.zipMapPartial(fieldNames, Iterables.transform(Lists.newArrayList(values), valueFunction)); - } - catch (Exception e) { - throw new ParseException(e, "Unable to parse row [%s]", input); - } + setFieldNames(header); } } diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/DelimitedParser.java b/java-util/src/main/java/io/druid/java/util/common/parsers/DelimitedParser.java index 27114df31e95..e8477324c8cf 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/DelimitedParser.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/DelimitedParser.java @@ -20,81 +20,40 @@ package io.druid.java.util.common.parsers; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Splitter; -import com.google.common.collect.Iterables; -import com.google.common.collect.Iterators; -import com.google.common.collect.Lists; -import io.druid.java.util.common.collect.Utils; -import java.util.ArrayList; +import javax.annotation.Nullable; +import java.io.IOException; import java.util.List; -import java.util.Map; -public class DelimitedParser implements Parser +public class DelimitedParser extends AbstractFlatTextFormatParser { - private static final String DEFAULT_DELIMITER = "\t"; - - private static Function getValueFunction( - final String listDelimiter, - final Splitter listSplitter - ) - { - return (input) -> { - if (input.contains(listDelimiter)) { - return Lists.newArrayList( - Iterables.transform( - listSplitter.split(input), - ParserUtils.nullEmptyStringFunction - ) - ); - } else { - return ParserUtils.nullEmptyStringFunction.apply(input); - } - }; - } - private final String delimiter; - private final String listDelimiter; private final Splitter splitter; - private final Splitter listSplitter; - private final Function valueFunction; - private final boolean hasHeaderRow; - private final int maxSkipHeaderRows; - - private ArrayList fieldNames = null; - private boolean hasParsedHeader = false; - private int skippedHeaderRows; - private boolean supportSkipHeaderRows; public DelimitedParser( - final Optional delimiter, - final Optional listDelimiter, + @Nullable final String delimiter, + @Nullable final String listDelimiter, final boolean hasHeaderRow, final int maxSkipHeaderRows ) { - this.delimiter = delimiter.isPresent() ? delimiter.get() : DEFAULT_DELIMITER; - this.listDelimiter = listDelimiter.isPresent() ? listDelimiter.get() : Parsers.DEFAULT_LIST_DELIMITER; + super(listDelimiter, hasHeaderRow, maxSkipHeaderRows); + this.delimiter = delimiter != null ? delimiter : FlatTextFormat.DELIMITED.getDefaultDelimiter(); Preconditions.checkState( - !this.delimiter.equals(this.listDelimiter), + !this.delimiter.equals(getListDelimiter()), "Cannot have same delimiter and list delimiter of [%s]", this.delimiter ); this.splitter = Splitter.on(this.delimiter); - this.listSplitter = Splitter.on(this.listDelimiter); - this.valueFunction = getValueFunction(this.listDelimiter, this.listSplitter); - this.hasHeaderRow = hasHeaderRow; - this.maxSkipHeaderRows = maxSkipHeaderRows; } public DelimitedParser( - final Optional delimiter, - final Optional listDelimiter, + @Nullable final String delimiter, + @Nullable final String listDelimiter, final Iterable fieldNames, final boolean hasHeaderRow, final int maxSkipHeaderRows @@ -106,7 +65,7 @@ public DelimitedParser( } @VisibleForTesting - DelimitedParser(final Optional delimiter, final Optional listDelimiter, final String header) + DelimitedParser(@Nullable final String delimiter, @Nullable final String listDelimiter, final String header) { this(delimiter, listDelimiter, false, 0); @@ -118,76 +77,9 @@ public String getDelimiter() return delimiter; } - public String getListDelimiter() - { - return listDelimiter; - } - @Override - public void startFileFromBeginning() + protected List parseLine(String input) throws IOException { - supportSkipHeaderRows = true; - hasParsedHeader = false; - skippedHeaderRows = 0; - } - - @Override - public List getFieldNames() - { - return fieldNames; - } - - @Override - public void setFieldNames(final Iterable fieldNames) - { - if (fieldNames != null) { - ParserUtils.validateFields(fieldNames); - this.fieldNames = Lists.newArrayList(fieldNames); - } - } - - public void setFieldNames(String header) - { - try { - setFieldNames(splitter.split(header)); - } - catch (Exception e) { - throw new ParseException(e, "Unable to parse header [%s]", header); - } - } - - @Override - public Map parse(final String input) - { - if (!supportSkipHeaderRows && (hasHeaderRow || maxSkipHeaderRows > 0)) { - throw new UnsupportedOperationException("hasHeaderRow or maxSkipHeaderRows is not supported. " - + "Please check the indexTask supports these options."); - } - - try { - Iterable values = splitter.split(input); - - if (skippedHeaderRows < maxSkipHeaderRows) { - skippedHeaderRows++; - return null; - } - - if (hasHeaderRow && !hasParsedHeader) { - if (fieldNames == null) { - setFieldNames(values); - } - hasParsedHeader = true; - return null; - } - - if (fieldNames == null) { - setFieldNames(ParserUtils.generateFieldNames(Iterators.size(values.iterator()))); - } - - return Utils.zipMapPartial(fieldNames, Iterables.transform(values, valueFunction)); - } - catch (Exception e) { - throw new ParseException(e, "Unable to parse row [%s]", input); - } + return splitter.splitToList(input); } } diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/ParserUtils.java b/java-util/src/main/java/io/druid/java/util/common/parsers/ParserUtils.java index 58fdd9dd4fa2..8adc74fec812 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/ParserUtils.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/ParserUtils.java @@ -20,31 +20,41 @@ package io.druid.java.util.common.parsers; import com.google.common.base.Function; +import com.google.common.base.Splitter; +import com.google.common.base.Strings; import com.google.common.collect.Sets; import org.joda.time.DateTime; import java.util.ArrayList; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; public class ParserUtils { - public static final Function nullEmptyStringFunction = new Function() + private static final String DEFAULT_COLUMN_NAME_PREFIX = "column_"; + + public static Function getMultiValueFunction( + final String listDelimiter, + final Splitter listSplitter + ) { - @Override - public String apply(String input) - { - if (input == null || input.isEmpty()) { - return null; + return (input) -> { + if (input != null && input.contains(listDelimiter)) { + return StreamSupport.stream(listSplitter.split(input).spliterator(), false) + .map(Strings::emptyToNull) + .collect(Collectors.toList()); + } else { + return Strings.emptyToNull(input); } - return input; - } - }; + }; + } public static ArrayList generateFieldNames(int length) { - ArrayList names = new ArrayList<>(length); + final ArrayList names = new ArrayList<>(length); for (int i = 0; i < length; ++i) { - names.add("column_" + (i + 1)); + names.add(getDefaultColumnName(i)); } return names; } @@ -94,4 +104,15 @@ public static String stripQuotes(String input) } return input; } + + /** + * Return a function to generate default column names. + * Note that the postfix for default column names starts from 1. + * + * @return column name generating function + */ + public static String getDefaultColumnName(int ordinal) + { + return DEFAULT_COLUMN_NAME_PREFIX + (ordinal + 1); + } } diff --git a/java-util/src/main/java/io/druid/java/util/common/parsers/RegexParser.java b/java-util/src/main/java/io/druid/java/util/common/parsers/RegexParser.java index 329b02aa944a..c9ca448ce98d 100644 --- a/java-util/src/main/java/io/druid/java/util/common/parsers/RegexParser.java +++ b/java-util/src/main/java/io/druid/java/util/common/parsers/RegexParser.java @@ -22,6 +22,7 @@ import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Splitter; +import com.google.common.base.Strings; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.java.util.common.collect.Utils; @@ -30,6 +31,8 @@ import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; /** */ @@ -56,12 +59,9 @@ public RegexParser( @Override public Object apply(String input) { - final List retVal = Lists.newArrayList( - Iterables.transform( - listSplitter.split(input), - ParserUtils.nullEmptyStringFunction - ) - ); + final List retVal = StreamSupport.stream(listSplitter.split(input).spliterator(), false) + .map(Strings::emptyToNull) + .collect(Collectors.toList()); if (retVal.size() == 1) { return retVal.get(0); } else { diff --git a/java-util/src/test/java/io/druid/java/util/common/parsers/CSVParserTest.java b/java-util/src/test/java/io/druid/java/util/common/parsers/CSVParserTest.java deleted file mode 100644 index 37a589b276a2..000000000000 --- a/java-util/src/test/java/io/druid/java/util/common/parsers/CSVParserTest.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.java.util.common.parsers; - -import com.google.common.base.Optional; -import com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Map; - -public class CSVParserTest -{ - - @Test - public void testValidHeader() - { - String csv = "time,value1,value2"; - final Parser csvParser; - boolean parseable = true; - try { - csvParser = new CSVParser(Optional.fromNullable(null), csv); - } - catch (Exception e) { - parseable = false; - } - finally { - Assert.assertTrue(parseable); - } - } - - @Test - public void testInvalidHeader() - { - String csv = "time,value1,value2,value2"; - final Parser csvParser; - boolean parseable = true; - try { - csvParser = new CSVParser(Optional.fromNullable(null), csv); - } - catch (Exception e) { - parseable = false; - } - finally { - Assert.assertFalse(parseable); - } - } - - @Test - public void testCSVParserWithHeader() - { - String header = "time,value1,value2"; - final Parser csvParser = new CSVParser(Optional.fromNullable(null), header); - String body = "hello,world,foo"; - final Map jsonMap = csvParser.parse(body); - Assert.assertEquals( - "jsonMap", - ImmutableMap.of("time", "hello", "value1", "world", "value2", "foo"), - jsonMap - ); - } - - @Test - public void testCSVParserWithoutHeader() - { - final Parser csvParser = new CSVParser(Optional.fromNullable(null), false, 0); - String body = "hello,world,foo"; - final Map jsonMap = csvParser.parse(body); - Assert.assertEquals( - "jsonMap", - ImmutableMap.of("column_1", "hello", "column_2", "world", "column_3", "foo"), - jsonMap - ); - } - - @Test - public void testCSVParserWithSkipHeaderRows() - { - final int skipHeaderRows = 2; - final Parser csvParser = new CSVParser( - Optional.absent(), - false, - skipHeaderRows - ); - csvParser.startFileFromBeginning(); - final String[] body = new String[] { - "header,line,1", - "header,line,2", - "hello,world,foo" - }; - int index; - for (index = 0; index < skipHeaderRows; index++) { - Assert.assertNull(csvParser.parse(body[index])); - } - final Map jsonMap = csvParser.parse(body[index]); - Assert.assertEquals( - "jsonMap", - ImmutableMap.of("column_1", "hello", "column_2", "world", "column_3", "foo"), - jsonMap - ); - } - - @Test(expected = UnsupportedOperationException.class) - public void testCSVParserWithoutStartFileFromBeginning() - { - final int skipHeaderRows = 2; - final Parser csvParser = new CSVParser( - Optional.absent(), - false, - skipHeaderRows - ); - final String[] body = new String[] { - "header\tline\t1", - "header\tline\t2", - "hello\tworld\tfoo" - }; - csvParser.parse(body[0]); - } -} diff --git a/java-util/src/test/java/io/druid/java/util/common/parsers/DelimitedParserTest.java b/java-util/src/test/java/io/druid/java/util/common/parsers/DelimitedParserTest.java deleted file mode 100644 index d91ed25cbbc2..000000000000 --- a/java-util/src/test/java/io/druid/java/util/common/parsers/DelimitedParserTest.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.java.util.common.parsers; - -import com.google.common.base.Optional; -import com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Map; - -public class DelimitedParserTest -{ - - @Test - public void testValidHeader() - { - String tsv = "time\tvalue1\tvalue2"; - final Parser delimitedParser; - boolean parseable = true; - try { - delimitedParser = new DelimitedParser(Optional.of("\t"), Optional.absent(), tsv); - } - catch (Exception e) { - parseable = false; - } - finally { - Assert.assertTrue(parseable); - } - } - - @Test - public void testInvalidHeader() - { - String tsv = "time\tvalue1\tvalue2\tvalue2"; - final Parser delimitedParser; - boolean parseable = true; - try { - delimitedParser = new DelimitedParser(Optional.of("\t"), Optional.absent(), tsv); - } - catch (Exception e) { - parseable = false; - } - finally { - Assert.assertFalse(parseable); - } - } - - @Test - public void testTSVParserWithHeader() - { - String header = "time\tvalue1\tvalue2"; - final Parser delimitedParser = new DelimitedParser( - Optional.of("\t"), - Optional.absent(), - header - ); - String body = "hello\tworld\tfoo"; - final Map jsonMap = delimitedParser.parse(body); - Assert.assertEquals( - "jsonMap", - ImmutableMap.of("time", "hello", "value1", "world", "value2", "foo"), - jsonMap - ); - } - - @Test - public void testTSVParserWithoutHeader() - { - final Parser delimitedParser = new DelimitedParser( - Optional.of("\t"), - Optional.absent(), - false, - 0 - ); - String body = "hello\tworld\tfoo"; - final Map jsonMap = delimitedParser.parse(body); - Assert.assertEquals( - "jsonMap", - ImmutableMap.of("column_1", "hello", "column_2", "world", "column_3", "foo"), - jsonMap - ); - } - - @Test - public void testTSVParserWithSkipHeaderRows() - { - final int skipHeaderRows = 2; - final Parser delimitedParser = new DelimitedParser( - Optional.of("\t"), - Optional.absent(), - false, - skipHeaderRows - ); - delimitedParser.startFileFromBeginning(); - final String[] body = new String[] { - "header\tline\t1", - "header\tline\t2", - "hello\tworld\tfoo" - }; - int index; - for (index = 0; index < skipHeaderRows; index++) { - Assert.assertNull(delimitedParser.parse(body[index])); - } - final Map jsonMap = delimitedParser.parse(body[index]); - Assert.assertEquals( - "jsonMap", - ImmutableMap.of("column_1", "hello", "column_2", "world", "column_3", "foo"), - jsonMap - ); - } - - @Test(expected = UnsupportedOperationException.class) - public void testTSVParserWithoutStartFileFromBeginning() - { - final int skipHeaderRows = 2; - final Parser delimitedParser = new DelimitedParser( - Optional.of("\t"), - Optional.absent(), - false, - skipHeaderRows - ); - final String[] body = new String[] { - "header\tline\t1", - "header\tline\t2", - "hello\tworld\tfoo" - }; - delimitedParser.parse(body[0]); - } -} diff --git a/java-util/src/test/java/io/druid/java/util/common/parsers/FlatTextFormatParserTest.java b/java-util/src/test/java/io/druid/java/util/common/parsers/FlatTextFormatParserTest.java new file mode 100644 index 000000000000..b8fa2d354a4d --- /dev/null +++ b/java-util/src/test/java/io/druid/java/util/common/parsers/FlatTextFormatParserTest.java @@ -0,0 +1,254 @@ +/* + * 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.java.util.common.parsers; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.druid.java.util.common.IAE; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.parsers.AbstractFlatTextFormatParser.FlatTextFormat; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Map; +import java.util.stream.Collectors; + +@RunWith(Parameterized.class) +public class FlatTextFormatParserTest +{ + @Parameters(name = "{0}") + public static Collection constructorFeeder() + { + return ImmutableList.of( + new Object[]{FlatTextFormat.CSV}, + new Object[]{FlatTextFormat.DELIMITED} + ); + } + + private static final FlatTextFormatParserFactory parserFactory = new FlatTextFormatParserFactory(); + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private final FlatTextFormat format; + + public FlatTextFormatParserTest(FlatTextFormat format) + { + this.format = format; + } + + @Test + public void testValidHeader() + { + final String header = concat(format, "time", "value1", "value2"); + final Parser parser = parserFactory.get(format, header); + Assert.assertEquals(ImmutableList.of("time", "value1", "value2"), parser.getFieldNames()); + } + + @Test + public void testDuplicatedColumnName() + { + final String header = concat(format, "time", "value1", "value2", "value2"); + + expectedException.expect(ParseException.class); + expectedException.expectMessage(StringUtils.safeFormat("Unable to parse header [%s]", header)); + + parserFactory.get(format, header); + } + + @Test + public void testWithHeader() + { + final String header = concat(format, "time", "value1", "value2"); + final Parser parser = parserFactory.get(format, header); + final String body = concat(format, "hello", "world", "foo"); + final Map jsonMap = parser.parse(body); + Assert.assertEquals( + "jsonMap", + ImmutableMap.of("time", "hello", "value1", "world", "value2", "foo"), + jsonMap + ); + } + + @Test + public void testWithoutHeader() + { + final Parser parser = parserFactory.get(format); + final String body = concat(format, "hello", "world", "foo"); + final Map jsonMap = parser.parse(body); + Assert.assertEquals( + "jsonMap", + ImmutableMap.of("column_1", "hello", "column_2", "world", "column_3", "foo"), + jsonMap + ); + } + + @Test + public void testWithSkipHeaderRows() + { + final int skipHeaderRows = 2; + final Parser parser = parserFactory.get(format, false, skipHeaderRows); + parser.startFileFromBeginning(); + final String[] body = new String[]{ + concat(format, "header", "line", "1"), + concat(format, "header", "line", "2"), + concat(format, "hello", "world", "foo") + }; + int index; + for (index = 0; index < skipHeaderRows; index++) { + Assert.assertNull(parser.parse(body[index])); + } + final Map jsonMap = parser.parse(body[index]); + Assert.assertEquals( + "jsonMap", + ImmutableMap.of("column_1", "hello", "column_2", "world", "column_3", "foo"), + jsonMap + ); + } + + @Test + public void testWithHeaderRow() + { + final Parser parser = parserFactory.get(format, true, 0); + parser.startFileFromBeginning(); + final String[] body = new String[]{ + concat(format, "time", "value1", "value2"), + concat(format, "hello", "world", "foo") + }; + Assert.assertNull(parser.parse(body[0])); + final Map jsonMap = parser.parse(body[1]); + Assert.assertEquals( + "jsonMap", + ImmutableMap.of("time", "hello", "value1", "world", "value2", "foo"), + jsonMap + ); + } + + @Test + public void testWithHeaderRowOfEmptyColumns() + { + final Parser parser = parserFactory.get(format, true, 0); + parser.startFileFromBeginning(); + final String[] body = new String[]{ + concat(format, "time", "", "value2", ""), + concat(format, "hello", "world", "foo", "bar") + }; + Assert.assertNull(parser.parse(body[0])); + final Map jsonMap = parser.parse(body[1]); + Assert.assertEquals( + "jsonMap", + ImmutableMap.of("time", "hello", "column_2", "world", "value2", "foo", "column_4", "bar"), + jsonMap + ); + } + + @Test + public void testWithDifferentHeaderRows() + { + final Parser parser = parserFactory.get(format, true, 0); + parser.startFileFromBeginning(); + final String[] body = new String[]{ + concat(format, "time", "value1", "value2"), + concat(format, "hello", "world", "foo") + }; + Assert.assertNull(parser.parse(body[0])); + Map jsonMap = parser.parse(body[1]); + Assert.assertEquals( + "jsonMap", + ImmutableMap.of("time", "hello", "value1", "world", "value2", "foo"), + jsonMap + ); + + parser.startFileFromBeginning(); + final String[] body2 = new String[]{ + concat(format, "time", "value1", "value2", "value3"), + concat(format, "hello", "world", "foo", "bar") + }; + Assert.assertNull(parser.parse(body2[0])); + jsonMap = parser.parse(body2[1]); + Assert.assertEquals( + "jsonMap", + ImmutableMap.of("time", "hello", "value1", "world", "value2", "foo", "value3", "bar"), + jsonMap + ); + } + + @Test + public void testWithoutStartFileFromBeginning() + { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage( + "hasHeaderRow or maxSkipHeaderRows is not supported. Please check the indexTask supports these options." + ); + + final int skipHeaderRows = 2; + final Parser parser = parserFactory.get(format, false, skipHeaderRows); + final String[] body = new String[]{ + concat(format, "header", "line", "1"), + concat(format, "header", "line", "2"), + concat(format, "hello", "world", "foo") + }; + parser.parse(body[0]); + } + + private static class FlatTextFormatParserFactory + { + public Parser get(FlatTextFormat format) + { + return get(format, false, 0); + } + + public Parser get(FlatTextFormat format, boolean hasHeaderRow, int maxSkipHeaderRows) + { + switch (format) { + case CSV: + return new CSVParser(null, hasHeaderRow, maxSkipHeaderRows); + case DELIMITED: + return new DelimitedParser("\t", null, hasHeaderRow, maxSkipHeaderRows); + default: + throw new IAE("Unknown format[%s]", format); + } + } + + public Parser get(FlatTextFormat format, String header) + { + switch (format) { + case CSV: + return new CSVParser(null, header); + case DELIMITED: + return new DelimitedParser("\t", null, header); + default: + throw new IAE("Unknown format[%s]", format); + } + } + } + + private static String concat(FlatTextFormat format, String ... values) + { + return Arrays.stream(values).collect(Collectors.joining(format.getDefaultDelimiter())); + } +} diff --git a/pom.xml b/pom.xml index 317a4910cecb..eec437da751c 100644 --- a/pom.xml +++ b/pom.xml @@ -16,8 +16,7 @@ ~ limitations under the License. --> - + 4.0.0 @@ -27,7 +26,7 @@ druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT pom ${project.groupId}:${project.artifactId} @@ -75,6 +74,9 @@ 2.7.3 2.0.0 1.6.6 + + 1.10.77 @@ -187,9 +189,30 @@ com.amazonaws aws-java-sdk-ec2 - - 1.10.77 + ${aws.sdk.version} + + + javax.mail + mail + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + commons-codec + commons-codec + + + + + com.amazonaws + aws-java-sdk-s3 + ${aws.sdk.version} javax.mail @@ -604,15 +627,17 @@ + org.apache.httpcomponents httpclient - 4.5.3 + 4.5.1 org.apache.httpcomponents httpcore - 4.4.6 + 4.4.3 org.apache.hadoop diff --git a/processing/pom.xml b/processing/pom.xml index 9d95e413dc05..4a4ad802469f 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT diff --git a/processing/src/main/java/io/druid/guice/GuiceInjectors.java b/processing/src/main/java/io/druid/guice/GuiceInjectors.java index 6fbf5b01b393..f3bac2362ce4 100644 --- a/processing/src/main/java/io/druid/guice/GuiceInjectors.java +++ b/processing/src/main/java/io/druid/guice/GuiceInjectors.java @@ -49,6 +49,7 @@ public void configure(Binder binder) { binder.bind(DruidSecondaryModule.class); JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class); + JsonConfigProvider.bind(binder, "druid.modules", ModulesConfig.class); } } ); diff --git a/processing/src/main/java/io/druid/guice/ModulesConfig.java b/processing/src/main/java/io/druid/guice/ModulesConfig.java new file mode 100644 index 000000000000..5dbacfb85612 --- /dev/null +++ b/processing/src/main/java/io/druid/guice/ModulesConfig.java @@ -0,0 +1,49 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.guice; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Collections; +import java.util.List; + +public class ModulesConfig +{ + /** + * Canonical class names of modules, which should not be loaded despite they are founded in extensions from {@link + * io.druid.guice.ExtensionsConfig#loadList} or the standard list of modules loaded by some node type, e. g. {@code + * CliPeon}. + */ + @JsonProperty + private List excludeList = Collections.emptyList(); + + public List getExcludeList() + { + return excludeList; + } + + @Override + public String toString() + { + return "ModulesConfig{" + + "excludeList=" + excludeList + + '}'; + } +} diff --git a/processing/src/main/java/io/druid/query/DruidProcessingConfig.java b/processing/src/main/java/io/druid/query/DruidProcessingConfig.java index 5445e5a7324b..f6c7c9fdd9ec 100644 --- a/processing/src/main/java/io/druid/query/DruidProcessingConfig.java +++ b/processing/src/main/java/io/druid/query/DruidProcessingConfig.java @@ -25,6 +25,8 @@ public abstract class DruidProcessingConfig extends ExecutorServiceConfig implements ColumnConfig { + public static final int DEFAULT_NUM_MERGE_BUFFERS = -1; + @Config({"druid.computation.buffer.size", "${base_path}.buffer.sizeBytes"}) public int intermediateComputeSizeBytes() { @@ -39,17 +41,31 @@ public int poolCacheMaxCount() @Override @Config(value = "${base_path}.numThreads") - public int getNumThreads() + public int getNumThreadsConfigured() { - // default to leaving one core for background tasks - final int processors = Runtime.getRuntime().availableProcessors(); - return processors > 1 ? processors - 1 : processors; + return DEFAULT_NUM_THREADS; } - @Config("${base_path}.numMergeBuffers") public int getNumMergeBuffers() { - return Math.max(2, getNumThreads() / 4); + int numMergeBuffersConfigured = getNumMergeBuffersConfigured(); + if (numMergeBuffersConfigured != DEFAULT_NUM_MERGE_BUFFERS) { + return numMergeBuffersConfigured; + } else { + return Math.max(2, getNumThreads() / 4); + } + } + + /** + * Returns the number of merge buffers _explicitly_ configured, or -1 if it is not explicitly configured, that is not + * a valid number of buffers. To get the configured value or the default (valid) number, use {@link + * #getNumMergeBuffers()}. This method exists for ability to distinguish between the default value set when there is + * no explicit config, and an explicitly configured value. + */ + @Config("${base_path}.numMergeBuffers") + public int getNumMergeBuffersConfigured() + { + return DEFAULT_NUM_MERGE_BUFFERS; } @Override diff --git a/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java index dda375b100be..dc90592eb57b 100644 --- a/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java @@ -29,7 +29,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import io.druid.collections.StupidPool; +import io.druid.collections.NonBlockingPool; import io.druid.data.input.Row; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; @@ -60,13 +60,13 @@ public class GroupByMergedQueryRunner implements QueryRunner private final ListeningExecutorService exec; private final Supplier configSupplier; private final QueryWatcher queryWatcher; - private final StupidPool bufferPool; + private final NonBlockingPool bufferPool; public GroupByMergedQueryRunner( ExecutorService exec, Supplier configSupplier, QueryWatcher queryWatcher, - StupidPool bufferPool, + NonBlockingPool bufferPool, Iterable> queryables ) { diff --git a/processing/src/main/java/io/druid/query/Queries.java b/processing/src/main/java/io/druid/query/Queries.java index 956293c7a6ea..519ef2acc65c 100644 --- a/processing/src/main/java/io/druid/query/Queries.java +++ b/processing/src/main/java/io/druid/query/Queries.java @@ -25,6 +25,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -47,14 +48,35 @@ public static List decoratePostAggregators( return decorated; } + /** + * Like {@link #prepareAggregations(List, List, List)} but with otherOutputNames as an empty list. Deprecated + * because it makes it easy to forget to include dimensions, etc. in "otherOutputNames". + * + * @param aggFactories aggregator factories for this query + * @param postAggs post-aggregators for this query + * + * @return decorated post-aggregators + * + * @throws NullPointerException if aggFactories is null + * @throws IllegalArgumentException if there are any output name collisions or missing post-aggregator inputs + */ + @Deprecated + public static List prepareAggregations( + List aggFactories, + List postAggs + ) + { + return prepareAggregations(Collections.emptyList(), aggFactories, postAggs); + } + /** * Returns decorated post-aggregators, based on original un-decorated post-aggregators. In addition, this method * also verifies that there are no output name collisions, and that all of the post-aggregators' required input * fields are present. * * @param otherOutputNames names of fields that will appear in the same output namespace as aggregators and - * post-aggregators. For most built-in query types, this is either empty, or the list of - * dimension output names. + * post-aggregators, and are also assumed to be valid inputs to post-aggregators. For most + * built-in query types, this is either empty, or the list of dimension output names. * @param aggFactories aggregator factories for this query * @param postAggs post-aggregators for this query * diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java index db928b4b686f..a6dc784b9115 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java @@ -86,7 +86,13 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) private FloatColumnSelector getFloatColumnSelector(ColumnSelectorFactory metricFactory) { - return AggregatorUtil.getFloatColumnSelector(metricFactory, macroTable, fieldName, expression, Float.MIN_VALUE); + return AggregatorUtil.getFloatColumnSelector( + metricFactory, + macroTable, + fieldName, + expression, + Float.NEGATIVE_INFINITY + ); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java index 1ef0e12cd841..cd5963046081 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java @@ -87,7 +87,13 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) private FloatColumnSelector getFloatColumnSelector(ColumnSelectorFactory metricFactory) { - return AggregatorUtil.getFloatColumnSelector(metricFactory, macroTable, fieldName, expression, Float.MAX_VALUE); + return AggregatorUtil.getFloatColumnSelector( + metricFactory, + macroTable, + fieldName, + expression, + Float.POSITIVE_INFINITY + ); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/Histogram.java b/processing/src/main/java/io/druid/query/aggregation/Histogram.java index b77d2607f70d..3beda4ea7abe 100644 --- a/processing/src/main/java/io/druid/query/aggregation/Histogram.java +++ b/processing/src/main/java/io/druid/query/aggregation/Histogram.java @@ -42,8 +42,8 @@ public Histogram(float[] breaks) { this.breaks = breaks; this.bins = new long[this.breaks.length + 1]; this.count = 0; - this.min = Float.MAX_VALUE; - this.max = Float.MIN_VALUE; + this.min = Float.POSITIVE_INFINITY; + this.max = Float.NEGATIVE_INFINITY; } public Histogram(float[] breaks, long[] bins, float min, float max) { diff --git a/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java index 4a6dd9a85087..66b2a2fbd0e5 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java @@ -50,8 +50,8 @@ public void init(ByteBuffer buf, int position) final long[] bins = new long[breaks.length + 1]; mutationBuffer.asLongBuffer().put(bins); - mutationBuffer.putFloat(position + minOffset, Float.MAX_VALUE); - mutationBuffer.putFloat(position + maxOffset, Float.MIN_VALUE); + mutationBuffer.putFloat(position + minOffset, Float.POSITIVE_INFINITY); + mutationBuffer.putFloat(position + maxOffset, Float.NEGATIVE_INFINITY); } @Override diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java index 32da5b3b268e..97d75da8c219 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java @@ -27,8 +27,8 @@ import com.google.common.collect.Maps; import com.google.common.primitives.Ints; import com.google.inject.Inject; +import io.druid.collections.NonBlockingPool; import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.guice.annotations.Global; @@ -71,12 +71,12 @@ public class GroupByQueryEngine private static final int MISSING_VALUE = -1; private final Supplier config; - private final StupidPool intermediateResultsBufferPool; + private final NonBlockingPool intermediateResultsBufferPool; @Inject public GroupByQueryEngine( Supplier config, - @Global StupidPool intermediateResultsBufferPool + @Global NonBlockingPool intermediateResultsBufferPool ) { this.config = config; diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java index b9d0910d8dd5..2e9f33d1fed3 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java @@ -23,7 +23,7 @@ import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import io.druid.collections.StupidPool; +import io.druid.collections.NonBlockingPool; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; @@ -59,7 +59,7 @@ public class GroupByQueryHelper public static Pair> createIndexAccumulatorPair( final GroupByQuery query, final GroupByQueryConfig config, - StupidPool bufferPool, + NonBlockingPool bufferPool, final boolean combine ) { @@ -188,7 +188,7 @@ public Queue accumulate(Queue accumulated, T in) public static IncrementalIndex makeIncrementalIndex( GroupByQuery query, GroupByQueryConfig config, - StupidPool bufferPool, + NonBlockingPool bufferPool, Sequence rows, boolean combine ) diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java index 63394d06b25e..e6995b333a49 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java @@ -23,8 +23,8 @@ import com.google.common.base.Strings; import com.google.common.base.Suppliers; import com.google.common.collect.Maps; +import io.druid.collections.NonBlockingPool; import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.java.util.common.IAE; @@ -36,6 +36,7 @@ import io.druid.query.ColumnSelectorPlus; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.dimension.ColumnSelectorStrategyFactory; +import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.GroupByQuery; import io.druid.query.groupby.GroupByQueryConfig; import io.druid.query.groupby.epinephelinae.column.DictionaryBuildingStringGroupByColumnSelectorStrategy; @@ -87,7 +88,7 @@ private GroupByQueryEngineV2() public static Sequence process( final GroupByQuery query, final StorageAdapter storageAdapter, - final StupidPool intermediateResultsBufferPool, + final NonBlockingPool intermediateResultsBufferPool, final GroupByQueryConfig config ) { @@ -360,6 +361,8 @@ public Row apply(final Grouper.Entry entry) ); } + convertRowTypesToOutputTypes(query.getDimensions(), theMap); + // Add aggregations. for (int i = 0; i < entry.getValues().length; i++) { theMap.put(query.getAggregatorSpecs().get(i).getName(), entry.getValues()[i]); @@ -402,6 +405,34 @@ public void close() } } + private static void convertRowTypesToOutputTypes(List dimensionSpecs, Map rowMap) + { + for (DimensionSpec dimSpec : dimensionSpecs) { + final ValueType outputType = dimSpec.getOutputType(); + rowMap.compute( + dimSpec.getOutputName(), + (dimName, baseVal) -> { + switch (outputType) { + case STRING: + baseVal = baseVal == null ? "" : baseVal.toString(); + break; + case LONG: + baseVal = DimensionHandlerUtils.convertObjectToLong(baseVal); + baseVal = baseVal == null ? 0L : baseVal; + break; + case FLOAT: + baseVal = DimensionHandlerUtils.convertObjectToFloat(baseVal); + baseVal = baseVal == null ? 0.f : baseVal; + break; + default: + throw new IAE("Unsupported type: " + outputType); + } + return baseVal; + } + ); + } + } + private static class GroupByEngineKeySerde implements Grouper.KeySerde { private final int keySize; diff --git a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java index 0cf98c313503..180659999516 100644 --- a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java +++ b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV1.java @@ -29,7 +29,7 @@ import com.google.common.collect.Sets; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.inject.Inject; -import io.druid.collections.StupidPool; +import io.druid.collections.NonBlockingPool; import io.druid.data.input.Row; import io.druid.guice.annotations.Global; import io.druid.java.util.common.IAE; @@ -65,14 +65,14 @@ public class GroupByStrategyV1 implements GroupByStrategy private final Supplier configSupplier; private final GroupByQueryEngine engine; private final QueryWatcher queryWatcher; - private final StupidPool bufferPool; + private final NonBlockingPool bufferPool; @Inject public GroupByStrategyV1( Supplier configSupplier, GroupByQueryEngine engine, QueryWatcher queryWatcher, - @Global StupidPool bufferPool + @Global NonBlockingPool bufferPool ) { this.configSupplier = configSupplier; diff --git a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java index ff9a43032973..add5387e75bf 100644 --- a/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java +++ b/processing/src/main/java/io/druid/query/groupby/strategy/GroupByStrategyV2.java @@ -28,8 +28,8 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.inject.Inject; import io.druid.collections.BlockingPool; +import io.druid.collections.NonBlockingPool; import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.guice.annotations.Global; @@ -80,7 +80,7 @@ public class GroupByStrategyV2 implements GroupByStrategy private final DruidProcessingConfig processingConfig; private final Supplier configSupplier; - private final StupidPool bufferPool; + private final NonBlockingPool bufferPool; private final BlockingPool mergeBufferPool; private final ObjectMapper spillMapper; private final QueryWatcher queryWatcher; @@ -89,7 +89,7 @@ public class GroupByStrategyV2 implements GroupByStrategy public GroupByStrategyV2( DruidProcessingConfig processingConfig, Supplier configSupplier, - @Global StupidPool bufferPool, + @Global NonBlockingPool bufferPool, @Merging BlockingPool mergeBufferPool, @Smile ObjectMapper spillMapper, QueryWatcher queryWatcher diff --git a/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java b/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java index e826d2cd81b0..14ce9694b788 100644 --- a/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java @@ -19,7 +19,7 @@ package io.druid.query.topn; -import io.druid.collections.StupidPool; +import io.druid.collections.NonBlockingPool; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; import io.druid.query.ColumnSelectorPlus; @@ -41,12 +41,12 @@ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm bufferPool; + private final NonBlockingPool bufferPool; public AggregateTopNMetricFirstAlgorithm( Capabilities capabilities, TopNQuery query, - StupidPool bufferPool + NonBlockingPool bufferPool ) { this.capabilities = capabilities; diff --git a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java index 27fea230d68c..2605480fdcee 100644 --- a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java @@ -22,8 +22,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; +import io.druid.collections.NonBlockingPool; import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidPool; import io.druid.java.util.common.Pair; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.query.BaseQuery; @@ -185,13 +185,13 @@ private static void computeSpecializedScanAndAggregateImplementations() } private final TopNQuery query; - private final StupidPool bufferPool; + private final NonBlockingPool bufferPool; private static final int AGG_UNROLL_COUNT = 8; // Must be able to fit loop below public PooledTopNAlgorithm( Capabilities capabilities, TopNQuery query, - StupidPool bufferPool + NonBlockingPool bufferPool ) { super(capabilities); diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java index a81873865f8d..c1bb74717585 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryEngine.java @@ -22,7 +22,7 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Predicates; -import io.druid.collections.StupidPool; +import io.druid.collections.NonBlockingPool; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; @@ -51,9 +51,9 @@ public class TopNQueryEngine { private static final Logger log = new Logger(TopNQueryEngine.class); - private final StupidPool bufferPool; + private final NonBlockingPool bufferPool; - public TopNQueryEngine(StupidPool bufferPool) + public TopNQueryEngine(NonBlockingPool bufferPool) { this.bufferPool = bufferPool; } diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java index f28077bb6c29..cf8c3aab69b5 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java @@ -20,7 +20,7 @@ package io.druid.query.topn; import com.google.inject.Inject; -import io.druid.collections.StupidPool; +import io.druid.collections.NonBlockingPool; import io.druid.guice.annotations.Global; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Sequence; @@ -41,13 +41,13 @@ */ public class TopNQueryRunnerFactory implements QueryRunnerFactory, TopNQuery> { - private final StupidPool computationBufferPool; + private final NonBlockingPool computationBufferPool; private final TopNQueryQueryToolChest toolchest; private final QueryWatcher queryWatcher; @Inject public TopNQueryRunnerFactory( - @Global StupidPool computationBufferPool, + @Global NonBlockingPool computationBufferPool, TopNQueryQueryToolChest toolchest, QueryWatcher queryWatcher ) diff --git a/processing/src/main/java/io/druid/segment/CompressedPools.java b/processing/src/main/java/io/druid/segment/CompressedPools.java index c2453b920aaf..b140d33f6f41 100644 --- a/processing/src/main/java/io/druid/segment/CompressedPools.java +++ b/processing/src/main/java/io/druid/segment/CompressedPools.java @@ -21,6 +21,7 @@ import com.google.common.base.Supplier; import com.ning.compress.BufferRecycler; +import io.druid.collections.NonBlockingPool; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidPool; import io.druid.java.util.common.logger.Logger; @@ -36,7 +37,7 @@ public class CompressedPools private static final Logger log = new Logger(CompressedPools.class); public static final int BUFFER_SIZE = 0x10000; - private static final StupidPool bufferRecyclerPool = new StupidPool( + private static final NonBlockingPool bufferRecyclerPool = new StupidPool<>( "bufferRecyclerPool", new Supplier() { @@ -56,7 +57,7 @@ public static ResourceHolder getBufferRecycler() return bufferRecyclerPool.take(); } - private static final StupidPool outputBytesPool = new StupidPool( + private static final NonBlockingPool outputBytesPool = new StupidPool( "outputBytesPool", new Supplier() { @@ -76,7 +77,7 @@ public static ResourceHolder getOutputBytes() return outputBytesPool.take(); } - private static final StupidPool bigEndByteBufPool = new StupidPool( + private static final NonBlockingPool bigEndByteBufPool = new StupidPool( "bigEndByteBufPool", new Supplier() { @@ -91,7 +92,7 @@ public ByteBuffer get() } ); - private static final StupidPool littleEndByteBufPool = new StupidPool( + private static final NonBlockingPool littleEndByteBufPool = new StupidPool( "littleEndByteBufPool", new Supplier() { diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java index a7f5f75450c5..763f4edd9801 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java @@ -69,13 +69,13 @@ public Indexed getSortedIndexedValues() @Override public Float getMinValue() { - return Float.MIN_VALUE; + return Float.NEGATIVE_INFINITY; } @Override public Float getMaxValue() { - return Float.MAX_VALUE; + return Float.POSITIVE_INFINITY; } @Override diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index fac3004b6e0c..a71223ad7fa2 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -669,7 +669,7 @@ public ObjectColumnSelector makeObjectColumnSelector(String column) @Override public Class classOfObject() { - return Float.TYPE; + return Float.class; } @Override @@ -685,7 +685,7 @@ public Float get() @Override public Class classOfObject() { - return Long.TYPE; + return Long.class; } @Override 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 9e1c121abec8..54e4f8f0ea95 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -30,7 +30,7 @@ import com.google.common.collect.Maps; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; -import io.druid.collections.StupidPool; +import io.druid.collections.NonBlockingPool; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; @@ -377,7 +377,7 @@ public IncrementalIndex buildOnheap() ); } - public IncrementalIndex buildOffheap(final StupidPool bufferPool) + public IncrementalIndex buildOffheap(final NonBlockingPool bufferPool) { if (maxRowCount <= 0) { throw new IllegalArgumentException("Invalid max row count: " + maxRowCount); @@ -658,9 +658,9 @@ public Class getMetricClass(String metric) case COMPLEX: return ComplexMetrics.getSerdeForType(metricDesc.getType()).getObjectStrategy().getClazz(); case FLOAT: - return Float.TYPE; + return Float.class; case LONG: - return Long.TYPE; + return Long.class; case STRING: return String.class; } 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 3a19b4797bf1..8c4681806f70 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -547,7 +547,7 @@ public ObjectColumnSelector makeObjectColumnSelector(String column) @Override public Class classOfObject() { - return Long.TYPE; + return Long.class; } @Override 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 598d454a27c3..a6aa82925773 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -21,8 +21,8 @@ import com.google.common.base.Supplier; import com.google.common.collect.Maps; +import io.druid.collections.NonBlockingPool; import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidPool; import io.druid.data.input.InputRow; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; @@ -44,7 +44,7 @@ public class OffheapIncrementalIndex extends IncrementalIndex { private static final Logger log = new Logger(OffheapIncrementalIndex.class); - private final StupidPool bufferPool; + private final NonBlockingPool bufferPool; private final List> aggBuffers = new ArrayList<>(); private final List indexAndOffsets = new ArrayList<>(); @@ -72,7 +72,7 @@ public class OffheapIncrementalIndex extends IncrementalIndex boolean concurrentEventAdd, boolean sortFacts, int maxRowCount, - StupidPool bufferPool + NonBlockingPool bufferPool ) { super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions, concurrentEventAdd); diff --git a/processing/src/test/java/io/druid/query/TestQueryRunners.java b/processing/src/test/java/io/druid/query/TestQueryRunners.java index e4daec6ea846..a4f877a2f511 100644 --- a/processing/src/test/java/io/druid/query/TestQueryRunners.java +++ b/processing/src/test/java/io/druid/query/TestQueryRunners.java @@ -21,6 +21,7 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; +import io.druid.collections.NonBlockingPool; import io.druid.collections.StupidPool; import io.druid.query.search.SearchQueryQueryToolChest; import io.druid.query.search.SearchQueryRunnerFactory; @@ -41,7 +42,7 @@ */ public class TestQueryRunners { - public static final StupidPool pool = new StupidPool( + public static final NonBlockingPool pool = new StupidPool( "TestQueryRunners-bufferPool", new Supplier() { @@ -54,7 +55,7 @@ public ByteBuffer get() ); public static final TopNQueryConfig topNConfig = new TopNQueryConfig(); - public static StupidPool getPool() + public static NonBlockingPool getPool() { return pool; } diff --git a/processing/src/test/java/io/druid/query/aggregation/HistogramTest.java b/processing/src/test/java/io/druid/query/aggregation/HistogramTest.java index 1ac89cd857b9..78ec75fa7cf9 100644 --- a/processing/src/test/java/io/druid/query/aggregation/HistogramTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/HistogramTest.java @@ -45,6 +45,25 @@ public void testOffer() { Assert.assertEquals("histogram matches expected histogram", hExpected, h); } + /** + * This test differs from {@link #testOffer()} only in that it offers only negative values into Histogram. It's to + * expose the issue of using Float's MIN_VALUE that is actually positive as initial value for {@link Histogram#max}. + */ + @Test + public void testOfferOnlyNegative() { + final float[] values = {-0.3f, -.1f, -0.8f, -.7f, -.5f, -3f}; + final float[] breaks = {-1f, -0.5f, 0.0f, 0.5f, 1f}; + + Histogram hExpected = new Histogram(breaks, new long[]{1,3,2,0,0,0}, -3f, -0.1f); + + Histogram h = new Histogram(breaks); + for(float v : values) { + h.offer(v); + } + + Assert.assertEquals("histogram matches expected histogram", hExpected, h); + } + @Test public void testToFromBytes() { float[] breaks = {-1f, -0.5f, 0.0f, 0.5f, 1f}; diff --git a/processing/src/test/java/io/druid/query/aggregation/MetricSelectorUtils.java b/processing/src/test/java/io/druid/query/aggregation/MetricSelectorUtils.java index c838b61b12a8..f08de1e7e601 100644 --- a/processing/src/test/java/io/druid/query/aggregation/MetricSelectorUtils.java +++ b/processing/src/test/java/io/druid/query/aggregation/MetricSelectorUtils.java @@ -31,7 +31,7 @@ public static ObjectColumnSelector wrap(final FloatColumnSelector selecto @Override public Class classOfObject() { - return Float.TYPE; + return Float.class; } @Override diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java index 82180d295007..ea9bb4389230 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -26,7 +26,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.util.concurrent.MoreExecutors; -import io.druid.collections.BlockingPool; +import io.druid.collections.DefaultBlockingPool; +import io.druid.collections.NonBlockingPool; import io.druid.collections.ReferenceCountingResourceHolder; import io.druid.collections.StupidPool; import io.druid.data.input.Row; @@ -59,7 +60,7 @@ @RunWith(Parameterized.class) public class GroupByQueryMergeBufferTest { - private static class TestBlockingPool extends BlockingPool + private static class TestBlockingPool extends DefaultBlockingPool { private int minRemainBufferNum; @@ -136,7 +137,7 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( ) { final Supplier configSupplier = Suppliers.ofInstance(config); - final StupidPool bufferPool = new StupidPool<>( + final NonBlockingPool bufferPool = new StupidPool<>( "GroupByQueryEngine-bufferPool", new Supplier() { diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java index 11dd20b1507a..198c22df36ee 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java @@ -27,6 +27,8 @@ import com.google.common.collect.Lists; import com.google.common.util.concurrent.MoreExecutors; import io.druid.collections.BlockingPool; +import io.druid.collections.DefaultBlockingPool; +import io.druid.collections.NonBlockingPool; import io.druid.collections.ReferenceCountingResourceHolder; import io.druid.collections.StupidPool; import io.druid.data.input.Row; @@ -100,7 +102,7 @@ private static GroupByQueryRunnerFactory makeQueryRunnerFactory( ) { final Supplier configSupplier = Suppliers.ofInstance(config); - final StupidPool bufferPool = new StupidPool<>( + final NonBlockingPool bufferPool = new StupidPool<>( "GroupByQueryEngine-bufferPool", new Supplier() { @@ -138,7 +140,7 @@ public ByteBuffer get() ); } - private final static BlockingPool mergeBufferPool = new BlockingPool<>( + private final static BlockingPool mergeBufferPool = new DefaultBlockingPool<>( new Supplier() { @Override 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 867f62b657b0..fadd47198cf2 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -30,8 +30,11 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; import io.druid.collections.BlockingPool; +import io.druid.collections.DefaultBlockingPool; +import io.druid.collections.NonBlockingPool; import io.druid.collections.StupidPool; import io.druid.data.input.Row; import io.druid.java.util.common.IAE; @@ -46,6 +49,7 @@ import io.druid.js.JavaScriptConfig; import io.druid.query.BySegmentResultValue; import io.druid.query.BySegmentResultValueClass; +import io.druid.query.ChainedExecutionQueryRunner; import io.druid.query.DruidProcessingConfig; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; @@ -56,6 +60,7 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryToolChest; +import io.druid.query.QueryWatcher; import io.druid.query.ResourceLimitExceededException; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; @@ -325,7 +330,7 @@ public static GroupByQueryRunnerFactory makeQueryRunnerFactory( ) { final Supplier configSupplier = Suppliers.ofInstance(config); - final StupidPool bufferPool = new StupidPool<>( + final NonBlockingPool bufferPool = new StupidPool<>( "GroupByQueryEngine-bufferPool", new Supplier() { @@ -336,7 +341,7 @@ public ByteBuffer get() } } ); - final BlockingPool mergeBufferPool = new BlockingPool<>( + final BlockingPool mergeBufferPool = new DefaultBlockingPool<>( new Supplier() { @Override @@ -9127,4 +9132,58 @@ public void testSubqueryWithMultipleIntervalsInOuterQueryWithLimitPushDown() Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); TestHelper.assertExpectedObjects(expectedResults, results, ""); } + + @Test + public void testTypeConversionWithMergingChainedExecutionRunner() + { + if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { + expectedException.expect(UnsupportedOperationException.class); + expectedException.expectMessage("GroupBy v1 only supports dimensions with an outputType of STRING."); + } + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("quality", "alias"), + new ExtractionDimensionSpec("quality", "qualityLen", ValueType.LONG, StrlenExtractionFn.instance()) + )) + .setDimFilter(new SelectorDimFilter( + "quality", + "technology", + null + )) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx", "index") + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-01", "alias", "technology", "qualityLen", 10L, "rows", 2L, "idx", 156L), + GroupByQueryRunnerTestHelper.createExpectedRow("2011-04-02", "alias", "technology", "qualityLen", 10L, "rows", 2L, "idx", 194L) + ); + + ChainedExecutionQueryRunner ceqr = new ChainedExecutionQueryRunner( + MoreExecutors.sameThreadExecutor(), + new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + return; + } + }, + ImmutableList.>of(runner, runner) + ); + + QueryRunner mergingRunner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.>of(ceqr)); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, mergingRunner, query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } } diff --git a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java index bc043a3a905f..9b9abd278677 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java @@ -69,8 +69,8 @@ public static Iterable compressionStrategies() private final float values5[] = {123.16f, 1.12f, 62.00f, 462.12f, 517.71f, 56.54f, 971.32f, 824.22f, 472.12f, 625.26f}; private final float values6[] = {1000000f, 1000001f, 1000002f, 1000003f, 1000004f, 1000005f, 1000006f, 1000007f, 1000008f}; private final float values7[] = { - Float.MAX_VALUE, Float.MIN_VALUE, 12378.5734f, -12718243.7496f, -93653653.1f, 12743153.385534f, 21431.414538f, - 65487435436632.123f, -43734526234564.65f + Float.POSITIVE_INFINITY, Float.NEGATIVE_INFINITY, 12378.5734f, -12718243.7496f, -93653653.1f, 12743153.385534f, + 21431.414538f, 65487435436632.123f, -43734526234564.65f }; public CompressedFloatsSerdeTest( 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 afac0e2a6d3a..1b1856d3cba0 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java @@ -214,7 +214,7 @@ public void testNullDimensionTransform() throws IndexSizeExceededException Lists.newArrayList("string", "float", "long"), ImmutableMap.of( "string", Arrays.asList("A", null, ""), - "float", Arrays.asList(Float.MAX_VALUE, null, ""), + "float", Arrays.asList(Float.POSITIVE_INFINITY, null, ""), "long", Arrays.asList(Long.MIN_VALUE, null, "") ) ) @@ -223,7 +223,7 @@ public void testNullDimensionTransform() throws IndexSizeExceededException Row row = index.iterator().next(); Assert.assertEquals(Arrays.asList(new String[]{"", "", "A"}), row.getRaw("string")); - Assert.assertEquals(Arrays.asList(new String[]{"", "", String.valueOf(Float.MAX_VALUE)}), row.getRaw("float")); + Assert.assertEquals(Arrays.asList(new String[]{"", "", String.valueOf(Float.POSITIVE_INFINITY)}), row.getRaw("float")); Assert.assertEquals(Arrays.asList(new String[]{"", "", String.valueOf(Long.MIN_VALUE)}), row.getRaw("long")); } diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index 8449c3fd03e3..8ae10bd992c4 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -286,11 +286,11 @@ public void testConcurrentAddRead() final IncrementalIndex incrementalIndex = this.incrementalIndex.getConstructor( IncrementalIndexSchema.class, - Boolean.TYPE, - Boolean.TYPE, - Boolean.TYPE, - Boolean.TYPE, - Integer.TYPE + boolean.class, + boolean.class, + boolean.class, + boolean.class, + int.class ).newInstance( new IncrementalIndexSchema.Builder().withMetrics(factories).build(), true, diff --git a/server/pom.xml b/server/pom.xml index 099bd6a6e817..f45ae766c2f9 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -26,7 +26,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT diff --git a/server/src/main/java/io/druid/guice/DruidProcessingConfigModule.java b/server/src/main/java/io/druid/guice/DruidProcessingConfigModule.java new file mode 100644 index 000000000000..0c1daca0e066 --- /dev/null +++ b/server/src/main/java/io/druid/guice/DruidProcessingConfigModule.java @@ -0,0 +1,35 @@ +/* + * 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.common.collect.ImmutableMap; +import com.google.inject.Binder; +import com.google.inject.Module; +import io.druid.query.DruidProcessingConfig; + +public class DruidProcessingConfigModule implements Module +{ + + @Override + public void configure(Binder binder) + { + ConfigProvider.bind(binder, DruidProcessingConfig.class, ImmutableMap.of("base_path", "druid.processing")); + } +} diff --git a/server/src/main/java/io/druid/guice/DruidProcessingModule.java b/server/src/main/java/io/druid/guice/DruidProcessingModule.java index 79ede1b4fcdf..ac1b8a267fd8 100644 --- a/server/src/main/java/io/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/io/druid/guice/DruidProcessingModule.java @@ -19,16 +19,16 @@ package io.druid.guice; -import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.ProvisionException; - import io.druid.client.cache.CacheConfig; import io.druid.collections.BlockingPool; +import io.druid.collections.DefaultBlockingPool; +import io.druid.collections.NonBlockingPool; import io.druid.collections.StupidPool; import io.druid.common.utils.VMUtils; import io.druid.guice.annotations.BackgroundCaching; @@ -58,7 +58,6 @@ public class DruidProcessingModule implements Module @Override public void configure(Binder binder) { - ConfigProvider.bind(binder, DruidProcessingConfig.class, ImmutableMap.of("base_path", "druid.processing")); binder.bind(ExecutorServiceConfig.class).to(DruidProcessingConfig.class); MetricsModule.register(binder, ExecutorServiceMonitor.class); } @@ -105,7 +104,7 @@ public ExecutorService getProcessingExecutorService( @Provides @LazySingleton @Global - public StupidPool getIntermediateResultsPool(DruidProcessingConfig config) + public NonBlockingPool getIntermediateResultsPool(DruidProcessingConfig config) { verifyDirectMemory(config); return new StupidPool<>( @@ -122,7 +121,7 @@ public StupidPool getIntermediateResultsPool(DruidProcessingConfig c public BlockingPool getMergeBufferPool(DruidProcessingConfig config) { verifyDirectMemory(config); - return new BlockingPool<>( + return new DefaultBlockingPool<>( new OffheapBufferGenerator("result merging", config.intermediateComputeSizeBytes()), config.getNumMergeBuffers() ); diff --git a/server/src/main/java/io/druid/guice/QueryableModule.java b/server/src/main/java/io/druid/guice/QueryableModule.java index b35742da466f..f4e783edd982 100644 --- a/server/src/main/java/io/druid/guice/QueryableModule.java +++ b/server/src/main/java/io/druid/guice/QueryableModule.java @@ -22,9 +22,9 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; -import com.google.inject.util.Providers; import io.druid.initialization.DruidModule; -import io.druid.query.QuerySegmentWalker; +import io.druid.query.DefaultQueryRunnerFactoryConglomerate; +import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.server.log.ComposingRequestLoggerProvider; import io.druid.server.log.EmittingRequestLoggerProvider; import io.druid.server.log.FileRequestLoggerProvider; @@ -43,9 +43,12 @@ public class QueryableModule implements DruidModule @Override public void configure(Binder binder) { - binder.bind(QuerySegmentWalker.class).toProvider(Providers.of(null)); binder.bind(RequestLogger.class).toProvider(RequestLoggerProvider.class).in(ManageLifecycle.class); JsonConfigProvider.bind(binder, "druid.request.logging", RequestLoggerProvider.class); + + binder.bind(QueryRunnerFactoryConglomerate.class) + .to(DefaultQueryRunnerFactoryConglomerate.class) + .in(LazySingleton.class); } @Override diff --git a/server/src/main/java/io/druid/guice/RouterProcessingModule.java b/server/src/main/java/io/druid/guice/RouterProcessingModule.java new file mode 100644 index 000000000000..bc3ac38fd25d --- /dev/null +++ b/server/src/main/java/io/druid/guice/RouterProcessingModule.java @@ -0,0 +1,107 @@ +/* + * 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 com.google.inject.Provides; +import io.druid.client.cache.CacheConfig; +import io.druid.collections.BlockingPool; +import io.druid.collections.DummyBlockingPool; +import io.druid.collections.DummyNonBlockingPool; +import io.druid.collections.NonBlockingPool; +import io.druid.concurrent.Execs; +import io.druid.guice.annotations.BackgroundCaching; +import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.Merging; +import io.druid.guice.annotations.Processing; +import io.druid.java.util.common.concurrent.ExecutorServiceConfig; +import io.druid.java.util.common.logger.Logger; +import io.druid.query.DruidProcessingConfig; +import io.druid.query.ExecutorServiceMonitor; +import io.druid.server.metrics.MetricsModule; + +import java.nio.ByteBuffer; +import java.util.concurrent.ExecutorService; + +/** + * This module is used to fulfill dependency injection of query processing and caching resources: buffer pools and + * thread pools on Router Druid node type. Router needs to inject those resources, because it depends on + * {@link io.druid.query.QueryToolChest}s, and they couple query type aspects not related to processing and caching, + * which Router uses, and related to processing and caching, which Router doesn't use, but they inject the resources. + */ +public class RouterProcessingModule implements Module +{ + private static final Logger log = new Logger(RouterProcessingModule.class); + + @Override + public void configure(Binder binder) + { + binder.bind(ExecutorServiceConfig.class).to(DruidProcessingConfig.class); + MetricsModule.register(binder, ExecutorServiceMonitor.class); + } + + @Provides + @BackgroundCaching + @LazySingleton + public ExecutorService getBackgroundExecutorService(CacheConfig cacheConfig) + { + if (cacheConfig.getNumBackgroundThreads() > 0) { + log.error( + "numBackgroundThreads[%d] configured, that is ignored on Router", + cacheConfig.getNumBackgroundThreads() + ); + } + return Execs.dummy(); + } + + @Provides + @Processing + @ManageLifecycle + public ExecutorService getProcessingExecutorService(DruidProcessingConfig config) + { + if (config.getNumThreadsConfigured() != ExecutorServiceConfig.DEFAULT_NUM_THREADS) { + log.error("numThreads[%d] configured, that is ignored on Router", config.getNumThreadsConfigured()); + } + return Execs.dummy(); + } + + @Provides + @LazySingleton + @Global + public NonBlockingPool getIntermediateResultsPool() + { + return DummyNonBlockingPool.instance(); + } + + @Provides + @LazySingleton + @Merging + public BlockingPool getMergeBufferPool(DruidProcessingConfig config) + { + if (config.getNumMergeBuffersConfigured() != DruidProcessingConfig.DEFAULT_NUM_MERGE_BUFFERS) { + log.error( + "numMergeBuffers[%d] configured, that is ignored on Router", + config.getNumMergeBuffersConfigured() + ); + } + return DummyBlockingPool.instance(); + } +} diff --git a/server/src/main/java/io/druid/guice/StorageNodeModule.java b/server/src/main/java/io/druid/guice/StorageNodeModule.java index 65c4a9d7f125..d4d74add21a0 100644 --- a/server/src/main/java/io/druid/guice/StorageNodeModule.java +++ b/server/src/main/java/io/druid/guice/StorageNodeModule.java @@ -26,9 +26,7 @@ import com.google.inject.util.Providers; import io.druid.client.DruidServerConfig; import io.druid.guice.annotations.Self; -import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.DruidProcessingConfig; -import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.column.ColumnConfig; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.server.DruidNode; @@ -48,10 +46,6 @@ public void configure(Binder binder) binder.bind(NodeTypeConfig.class).toProvider(Providers.of(null)); binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class); - - binder.bind(QueryRunnerFactoryConglomerate.class) - .to(DefaultQueryRunnerFactoryConglomerate.class) - .in(LazySingleton.class); } @Provides diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index d525391ac4d5..81b23db3e31e 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -35,7 +35,7 @@ import io.druid.guice.AWSModule; import io.druid.guice.AnnouncerModule; import io.druid.guice.CoordinatorDiscoveryModule; -import io.druid.guice.DruidProcessingModule; +import io.druid.guice.DruidProcessingConfigModule; import io.druid.guice.DruidSecondaryModule; import io.druid.guice.ExpressionModule; import io.druid.guice.ExtensionsConfig; @@ -46,9 +46,8 @@ import io.druid.guice.LifecycleModule; import io.druid.guice.LocalDataStorageDruidModule; import io.druid.guice.MetadataConfigModule; +import io.druid.guice.ModulesConfig; import io.druid.guice.ParsersModule; -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; @@ -76,6 +75,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.ServiceLoader; @@ -90,17 +90,18 @@ public class Initialization private static final Logger log = new Logger(Initialization.class); private static final ConcurrentMap loadersMap = new ConcurrentHashMap<>(); - private final static Map extensionsMap = Maps.newHashMap(); + private final static Map extensionsMap = Maps.newHashMap(); /** - * @param clazz Module class - * @param + * @param clazz service class + * @param the service type * - * @return Returns the set of modules loaded. + * @return Returns a collection of implementations loaded. */ - public static Set getLoadedModules(Class clazz) + public static Collection getLoadedImplementations(Class clazz) { - Set retVal = extensionsMap.get(clazz); + @SuppressWarnings("unchecked") + Collection retVal = extensionsMap.get(clazz); if (retVal == null) { return Sets.newHashSet(); } @@ -108,7 +109,7 @@ public static Set getLoadedModules(Class clazz) } @VisibleForTesting - static void clearLoadedModules() + static void clearLoadedImplementations() { extensionsMap.clear(); } @@ -120,63 +121,82 @@ static Map getLoadersMap() } /** - * Look for extension modules for the given class from both classpath and extensions directory. A user should never - * put the same two extensions in classpath and extensions directory, if he/she does that, the one that is in the - * classpath will be loaded, the other will be ignored. + * Look for implementations for the given class from both classpath and extensions directory, using {@link + * java.util.ServiceLoader}. A user should never put the same two extensions in classpath and extensions directory, if + * he/she does that, the one that is in the classpath will be loaded, the other will be ignored. * - * @param config Extensions configuration - * @param clazz The class of extension module (e.g., DruidModule) + * @param config Extensions configuration + * @param serviceClass The class to look the implementations of (e.g., DruidModule) * - * @return A collection that contains distinct extension modules + * @return A collection that contains implementations (of distinct concrete classes) of the given class. The order of + * elements in the returned collection is not specified and not guaranteed to be the same for different calls to + * getFromExtensions(). */ - public synchronized static Collection getFromExtensions(ExtensionsConfig config, Class clazz) + public synchronized static Collection getFromExtensions(ExtensionsConfig config, Class serviceClass) { - final Set retVal = Sets.newHashSet(); - final Set loadedExtensionNames = Sets.newHashSet(); - - if (config.searchCurrentClassloader()) { - for (T module : ServiceLoader.load(clazz, Thread.currentThread().getContextClassLoader())) { - final String moduleName = module.getClass().getCanonicalName(); - if (moduleName == null) { - log.warn( - "Extension module [%s] was ignored because it doesn't have a canonical name, is it a local or anonymous class?", - module.getClass().getName() - ); - } else if (!loadedExtensionNames.contains(moduleName)) { - log.info("Adding classpath extension module [%s] for class [%s]", moduleName, clazz.getName()); - loadedExtensionNames.add(moduleName); - retVal.add(module); - } + Collection modulesToLoad = new ServiceLoadingFromExtensions<>(config, serviceClass).implsToLoad; + extensionsMap.put(serviceClass, modulesToLoad); + return modulesToLoad; + } + + private static class ServiceLoadingFromExtensions + { + private final ExtensionsConfig extensionsConfig; + private final Class serviceClass; + private final List implsToLoad = new ArrayList<>(); + private final Set implClassNamesToLoad = new HashSet<>(); + + private ServiceLoadingFromExtensions(ExtensionsConfig extensionsConfig, Class serviceClass) + { + this.extensionsConfig = extensionsConfig; + this.serviceClass = serviceClass; + if (extensionsConfig.searchCurrentClassloader()) { + addAllFromCurrentClassLoader(); } + addAllFromFileSystem(); } - for (File extension : getExtensionFilesToLoad(config)) { - log.info("Loading extension [%s] for class [%s]", extension.getName(), clazz.getName()); - try { - final URLClassLoader loader = getClassLoaderForExtension(extension); - for (T module : ServiceLoader.load(clazz, loader)) { - final String moduleName = module.getClass().getCanonicalName(); - if (moduleName == null) { - log.warn( - "Extension module [%s] was ignored because it doesn't have a canonical name, is it a local or anonymous class?", - module.getClass().getName() - ); - } else if (!loadedExtensionNames.contains(moduleName)) { - log.info("Adding local file system extension module [%s] for class [%s]", moduleName, clazz.getName()); - loadedExtensionNames.add(moduleName); - retVal.add(module); - } + private void addAllFromCurrentClassLoader() + { + ServiceLoader + .load(serviceClass, Thread.currentThread().getContextClassLoader()) + .forEach(impl -> tryAdd(impl, "classpath")); + } + + private void addAllFromFileSystem() + { + for (File extension : getExtensionFilesToLoad(extensionsConfig)) { + log.info("Loading extension [%s] for class [%s]", extension.getName(), serviceClass); + try { + final URLClassLoader loader = getClassLoaderForExtension(extension); + ServiceLoader.load(serviceClass, loader).forEach(impl -> tryAdd(impl, "local file system")); + } + catch (Exception e) { + throw Throwables.propagate(e); } - } - catch (Exception e) { - throw Throwables.propagate(e); } } - // update the map with currently loaded modules - extensionsMap.put(clazz, retVal); - - return retVal; + private void tryAdd(T serviceImpl, String extensionType) + { + final String serviceImplName = serviceImpl.getClass().getCanonicalName(); + if (serviceImplName == null) { + log.warn( + "Implementation [%s] was ignored because it doesn't have a canonical name, " + + "is it a local or anonymous class?", + serviceImpl.getClass().getName() + ); + } else if (!implClassNamesToLoad.contains(serviceImplName)) { + log.info( + "Adding implementation [%s] for class [%s] from %s extension", + serviceImplName, + serviceClass, + extensionType + ); + implClassNamesToLoad.add(serviceImplName); + implsToLoad.add(serviceImpl); + } + } } /** @@ -332,14 +352,12 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter new HttpClientModule("druid.broker.http", Client.class), new CuratorModule(), new AnnouncerModule(), - new DruidProcessingModule(), new AWSModule(), new MetricsModule(), new ServerModule(), + new DruidProcessingConfigModule(), new StorageNodeModule(), new JettyServerModule(), - new QueryableModule(), - new QueryRunnerFactoryModule(), new ExpressionModule(), new DiscoveryModule(), new ServerViewModule(), @@ -375,6 +393,7 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter private static class ModuleList { private final Injector baseInjector; + private final ModulesConfig modulesConfig; private final ObjectMapper jsonMapper; private final ObjectMapper smileMapper; private final List modules; @@ -382,6 +401,7 @@ private static class ModuleList public ModuleList(Injector baseInjector) { this.baseInjector = baseInjector; + this.modulesConfig = baseInjector.getInstance(ModulesConfig.class); this.jsonMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Json.class)); this.smileMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Smile.class)); this.modules = Lists.newArrayList(); @@ -395,12 +415,21 @@ private List getModules() public void addModule(Object input) { if (input instanceof DruidModule) { + if (!checkModuleClass(input.getClass())) { + return; + } baseInjector.injectMembers(input); modules.add(registerJacksonModules(((DruidModule) input))); } else if (input instanceof Module) { + if (!checkModuleClass(input.getClass())) { + return; + } baseInjector.injectMembers(input); modules.add((Module) input); } else if (input instanceof Class) { + if (!checkModuleClass((Class) input)) { + return; + } if (DruidModule.class.isAssignableFrom((Class) input)) { modules.add(registerJacksonModules(baseInjector.getInstance((Class) input))); } else if (Module.class.isAssignableFrom((Class) input)) { @@ -414,6 +443,16 @@ public void addModule(Object input) } } + private boolean checkModuleClass(Class moduleClass) + { + String moduleClassName = moduleClass.getCanonicalName(); + if (moduleClassName != null && modulesConfig.getExcludeList().contains(moduleClassName)) { + log.info("Not loading module [%s] because it is present in excludeList", moduleClassName); + return false; + } + return true; + } + public void addModules(Object... object) { for (Object o : object) { diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index b1f1013f47ef..90284b22e4c1 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -32,6 +32,7 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.DirectDruidClient; +import io.druid.guice.LazySingleton; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.ISE; @@ -84,6 +85,7 @@ /** */ +@LazySingleton @Path("/druid/v2/") public class QueryResource implements QueryCountStatsProvider { diff --git a/server/src/main/java/io/druid/server/StatusResource.java b/server/src/main/java/io/druid/server/StatusResource.java index edbd65b4fdb7..127c840be412 100644 --- a/server/src/main/java/io/druid/server/StatusResource.java +++ b/server/src/main/java/io/druid/server/StatusResource.java @@ -44,7 +44,7 @@ public class StatusResource @Produces(MediaType.APPLICATION_JSON) public Status doGet() { - return new Status(Initialization.getLoadedModules(DruidModule.class)); + return new Status(Initialization.getLoadedImplementations(DruidModule.class)); } public static class Status diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCluster.java b/server/src/main/java/io/druid/server/coordinator/DruidCluster.java index 20561bee8550..014c42ba91c4 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCluster.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCluster.java @@ -26,9 +26,11 @@ import io.druid.java.util.common.IAE; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; @@ -113,9 +115,13 @@ public MinMaxPriorityQueue getHistoricalsByTier(String tier) public Collection getAllServers() { - return historicals.values().stream() - .flatMap(Collection::stream) - .collect(() -> realtimes, Set::add, Set::addAll); + final int historicalSize = historicals.values().stream().mapToInt(Collection::size).sum(); + final int realtimeSize = realtimes.size(); + final List allServers = new ArrayList<>(historicalSize + realtimeSize); + + historicals.values().forEach(allServers::addAll); + allServers.addAll(realtimes); + return allServers; } public Iterable> getSortedHistoricalsByTier() diff --git a/server/src/test/java/io/druid/initialization/InitializationTest.java b/server/src/test/java/io/druid/initialization/InitializationTest.java index b5e238a0f040..80b0769c3c02 100644 --- a/server/src/test/java/io/druid/initialization/InitializationTest.java +++ b/server/src/test/java/io/druid/initialization/InitializationTest.java @@ -28,7 +28,6 @@ import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Key; - import io.druid.guice.ExtensionsConfig; import io.druid.guice.GuiceInjectors; import io.druid.guice.JsonConfigProvider; @@ -50,6 +49,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Comparator; +import java.util.HashSet; import java.util.List; import java.util.Set; @@ -62,11 +62,11 @@ public class InitializationTest @Test public void test01InitialModulesEmpty() throws Exception { - Initialization.clearLoadedModules(); + Initialization.clearLoadedImplementations(); Assert.assertEquals( "Initial set of loaded modules must be empty", 0, - Initialization.getLoadedModules(DruidModule.class).size() + Initialization.getLoadedImplementations(DruidModule.class).size() ); } @@ -95,7 +95,7 @@ public String apply(@Nullable DruidModule input) Assert.assertFalse( "modules does not contain TestDruidModule", - Collections2.transform(Initialization.getLoadedModules(DruidModule.class), fnClassName) + Collections2.transform(Initialization.getLoadedImplementations(DruidModule.class), fnClassName) .contains("io.druid.initialization.InitializationTest.TestDruidModule") ); @@ -179,13 +179,16 @@ public int compare(URL o1, URL o2) public void testGetLoadedModules() { - Set modules = Initialization.getLoadedModules(DruidModule.class); + Collection modules = Initialization.getLoadedImplementations(DruidModule.class); + HashSet moduleSet = new HashSet<>(modules); - Set loadedModules = Initialization.getLoadedModules(DruidModule.class); - Assert.assertEquals("Set from loaded modules #1 should be same!", modules, loadedModules); + Collection loadedModules = Initialization.getLoadedImplementations(DruidModule.class); + Assert.assertEquals("Set from loaded modules #1 should be same!", modules.size(), loadedModules.size()); + Assert.assertEquals("Set from loaded modules #1 should be same!", moduleSet, new HashSet<>(loadedModules)); - Set loadedModules2 = Initialization.getLoadedModules(DruidModule.class); - Assert.assertEquals("Set from loaded modules #2 should be same!", modules, loadedModules2); + Collection loadedModules2 = Initialization.getLoadedImplementations(DruidModule.class); + Assert.assertEquals("Set from loaded modules #2 should be same!", modules.size(), loadedModules2.size()); + Assert.assertEquals("Set from loaded modules #2 should be same!", moduleSet, new HashSet<>(loadedModules2)); } @Test diff --git a/server/src/test/java/io/druid/server/coordinator/DruidClusterTest.java b/server/src/test/java/io/druid/server/coordinator/DruidClusterTest.java index d91675f9e42f..9d001146c2dc 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidClusterTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidClusterTest.java @@ -38,6 +38,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; public class DruidClusterTest @@ -184,6 +185,9 @@ public void testGetAllServers() { cluster.add(newRealtime); cluster.add(newHistorical); + final Set expectedRealtimes = cluster.getRealtimes(); + final Map> expectedHistoricals = cluster.getHistoricals(); + final Collection allServers = cluster.getAllServers(); Assert.assertEquals(4, allServers.size()); Assert.assertTrue(allServers.containsAll(cluster.getRealtimes())); @@ -192,6 +196,9 @@ public void testGetAllServers() cluster.getHistoricals().values().stream().flatMap(Collection::stream).collect(Collectors.toList()) ) ); + + Assert.assertEquals(expectedHistoricals, cluster.getHistoricals()); + Assert.assertEquals(expectedRealtimes, cluster.getRealtimes()); } @Test diff --git a/server/src/test/java/io/druid/server/log/LoggingRequestLoggerProviderTest.java b/server/src/test/java/io/druid/server/log/LoggingRequestLoggerProviderTest.java index d1e920e69d09..ab1ce25f2c15 100644 --- a/server/src/test/java/io/druid/server/log/LoggingRequestLoggerProviderTest.java +++ b/server/src/test/java/io/druid/server/log/LoggingRequestLoggerProviderTest.java @@ -28,6 +28,8 @@ import io.druid.guice.GuiceInjectors; import io.druid.guice.JsonConfigProvider; import io.druid.guice.JsonConfigurator; +import io.druid.guice.ManageLifecycle; +import io.druid.guice.QueryableModule; import io.druid.initialization.Initialization; import org.junit.Assert; import org.junit.Test; @@ -73,11 +75,12 @@ private Injector makeInjector() return Initialization.makeInjectorWithModules( GuiceInjectors.makeStartupInjector(), ImmutableList.of( - new Module() + new QueryableModule() { @Override public void configure(Binder binder) { + binder.bind(RequestLogger.class).toProvider(RequestLoggerProvider.class).in(ManageLifecycle.class); binder.bind(Key.get(String.class, Names.named("serviceName"))).toInstance("some service"); binder.bind(Key.get(Integer.class, Names.named("servicePort"))).toInstance(0); JsonConfigProvider.bind(binder, propertyPrefix, RequestLoggerProvider.class); diff --git a/services/pom.xml b/services/pom.xml index 7c1dd2fee071..f3a78f769e72 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -25,7 +25,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT @@ -95,7 +95,7 @@ - + diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 17a0bdb3b58a..e4e748c44d5a 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -34,10 +34,13 @@ import io.druid.client.selector.ServerSelectorStrategy; import io.druid.client.selector.TierSelectorStrategy; import io.druid.guice.CacheModule; +import io.druid.guice.DruidProcessingModule; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; +import io.druid.guice.QueryRunnerFactoryModule; +import io.druid.guice.QueryableModule; import io.druid.java.util.common.logger.Logger; import io.druid.query.QuerySegmentWalker; import io.druid.query.RetryQueryRunnerConfig; @@ -75,6 +78,9 @@ public CliBroker() protected List getModules() { return ImmutableList.of( + new DruidProcessingModule(), + new QueryableModule(), + new QueryRunnerFactoryModule(), new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 89d92f175f1a..12271515341c 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -27,12 +27,15 @@ import io.druid.client.cache.CacheConfig; import io.druid.client.cache.CacheMonitor; import io.druid.guice.CacheModule; +import io.druid.guice.DruidProcessingModule; 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.guice.NodeTypeConfig; +import io.druid.guice.QueryRunnerFactoryModule; +import io.druid.guice.QueryableModule; import io.druid.java.util.common.logger.Logger; import io.druid.query.QuerySegmentWalker; import io.druid.query.lookup.LookupModule; @@ -69,6 +72,9 @@ public CliHistorical() protected List getModules() { return ImmutableList.of( + new DruidProcessingModule(), + new QueryableModule(), + new QueryRunnerFactoryModule(), new Module() { @Override @@ -86,7 +92,7 @@ public void configure(Binder binder) binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(ServerType.HISTORICAL)); binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class); - binder.bind(QueryCountStatsProvider.class).to(QueryResource.class).in(LazySingleton.class); + binder.bind(QueryCountStatsProvider.class).to(QueryResource.class); Jerseys.addResource(binder, QueryResource.class); Jerseys.addResource(binder, HistoricalResource.class); Jerseys.addResource(binder, SegmentListerResource.class); diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index d236ac593e75..6216b2d9e769 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -31,7 +31,6 @@ import com.google.inject.multibindings.MapBinder; import com.google.inject.name.Named; import com.google.inject.name.Names; - import io.airlift.airline.Arguments; import io.airlift.airline.Command; import io.airlift.airline.Option; @@ -39,6 +38,7 @@ import io.druid.client.coordinator.CoordinatorClient; import io.druid.guice.Binders; import io.druid.guice.CacheModule; +import io.druid.guice.DruidProcessingModule; import io.druid.guice.IndexingServiceFirehoseModule; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; @@ -47,6 +47,9 @@ import io.druid.guice.ManageLifecycle; import io.druid.guice.NodeTypeConfig; import io.druid.guice.PolyBind; +import io.druid.guice.QueryRunnerFactoryModule; +import io.druid.guice.QueryableModule; +import io.druid.guice.QueryablePeonModule; import io.druid.guice.annotations.Json; import io.druid.indexing.common.RetryPolicyConfig; import io.druid.indexing.common.RetryPolicyFactory; @@ -86,8 +89,6 @@ import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import io.druid.server.coordination.ServerType; import io.druid.server.http.SegmentListerResource; -import io.druid.server.metrics.QueryCountStatsProvider; -import io.druid.server.QueryResource; import io.druid.server.initialization.jetty.ChatHandlerServerModule; import io.druid.server.initialization.jetty.JettyServerInitializer; import io.druid.server.metrics.DataSourceTaskIdHolder; @@ -129,6 +130,9 @@ public CliPeon() protected List getModules() { return ImmutableList.of( + new DruidProcessingModule(), + new QueryableModule(), + new QueryRunnerFactoryModule(), new Module() { @Override @@ -204,10 +208,7 @@ public void configure(Binder binder) binder.bind(CoordinatorClient.class).in(LazySingleton.class); binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class); - binder.bind(QueryCountStatsProvider.class).to(QueryResource.class).in(LazySingleton.class); - Jerseys.addResource(binder, QueryResource.class); Jerseys.addResource(binder, SegmentListerResource.class); - LifecycleModule.register(binder, QueryResource.class); binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(ServerType.fromString(nodeType))); LifecycleModule.register(binder, Server.class); } @@ -265,6 +266,7 @@ public String getTaskIDFromTask(final Task task) return task.getId(); } }, + new QueryablePeonModule(), new IndexingServiceFirehoseModule(), new ChatHandlerServerModule(properties), new LookupModule() diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index f0b5cdf60636..2c9236511c48 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -24,8 +24,10 @@ import com.google.inject.Inject; import com.google.inject.Module; import com.google.inject.name.Names; - import io.airlift.airline.Command; +import io.druid.guice.DruidProcessingModule; +import io.druid.guice.QueryRunnerFactoryModule; +import io.druid.guice.QueryableModule; import io.druid.guice.RealtimeModule; import io.druid.java.util.common.logger.Logger; import io.druid.query.lookup.LookupModule; @@ -56,6 +58,9 @@ public CliRealtime() protected List getModules() { return ImmutableList.of( + new DruidProcessingModule(), + new QueryableModule(), + new QueryRunnerFactoryModule(), new RealtimeModule(), new Module() { diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index b6516fb4c66d..5075b65b1a72 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -29,7 +29,10 @@ import io.druid.client.DruidServer; import io.druid.client.InventoryView; import io.druid.client.ServerView; +import io.druid.guice.DruidProcessingModule; import io.druid.guice.LazySingleton; +import io.druid.guice.QueryRunnerFactoryModule; +import io.druid.guice.QueryableModule; import io.druid.guice.RealtimeModule; import io.druid.java.util.common.logger.Logger; import io.druid.query.lookup.LookupModule; @@ -68,6 +71,9 @@ public CliRealtimeExample() protected List getModules() { return ImmutableList.of( + new DruidProcessingModule(), + new QueryableModule(), + new QueryRunnerFactoryModule(), new RealtimeModule(), new Module() { diff --git a/services/src/main/java/io/druid/cli/CliRouter.java b/services/src/main/java/io/druid/cli/CliRouter.java index ae797509fc41..a926252ff044 100644 --- a/services/src/main/java/io/druid/cli/CliRouter.java +++ b/services/src/main/java/io/druid/cli/CliRouter.java @@ -33,6 +33,9 @@ import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; import io.druid.guice.ManageLifecycle; +import io.druid.guice.RouterProcessingModule; +import io.druid.guice.QueryRunnerFactoryModule; +import io.druid.guice.QueryableModule; import io.druid.guice.annotations.Self; import io.druid.guice.http.JettyHttpClientModule; import io.druid.java.util.common.logger.Logger; @@ -70,6 +73,9 @@ public CliRouter() protected List getModules() { return ImmutableList.of( + new RouterProcessingModule(), + new QueryableModule(), + new QueryRunnerFactoryModule(), new JettyHttpClientModule("druid.router.http", Router.class), new Module() { diff --git a/services/src/main/java/io/druid/cli/CreateTables.java b/services/src/main/java/io/druid/cli/CreateTables.java index aa9e3f5d98d6..abb5a7638acc 100644 --- a/services/src/main/java/io/druid/cli/CreateTables.java +++ b/services/src/main/java/io/druid/cli/CreateTables.java @@ -24,10 +24,12 @@ import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; - import io.airlift.airline.Command; import io.airlift.airline.Option; +import io.druid.guice.DruidProcessingModule; import io.druid.guice.JsonConfigProvider; +import io.druid.guice.QueryRunnerFactoryModule; +import io.druid.guice.QueryableModule; import io.druid.guice.annotations.Self; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.MetadataStorageConnector; @@ -66,6 +68,12 @@ public CreateTables() protected List getModules() { return ImmutableList.of( + // It's unknown why those modules are required in CreateTables, and if all of those modules are required or not. + // Maybe some of those modules could be removed. + // See https://github.com/druid-io/druid/pull/4429#discussion_r123602930 + new DruidProcessingModule(), + new QueryableModule(), + new QueryRunnerFactoryModule(), new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/DumpSegment.java b/services/src/main/java/io/druid/cli/DumpSegment.java index 58d7abf957db..d377e50ec4ed 100644 --- a/services/src/main/java/io/druid/cli/DumpSegment.java +++ b/services/src/main/java/io/druid/cli/DumpSegment.java @@ -41,6 +41,9 @@ import io.druid.collections.bitmap.ConciseBitmapFactory; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.bitmap.RoaringBitmapFactory; +import io.druid.guice.DruidProcessingModule; +import io.druid.guice.QueryRunnerFactoryModule; +import io.druid.guice.QueryableModule; import io.druid.guice.annotations.Json; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; @@ -425,6 +428,9 @@ private T withOutputStream(Function f) throws IOException protected List getModules() { return ImmutableList.of( + new DruidProcessingModule(), + new QueryableModule(), + new QueryRunnerFactoryModule(), new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/InsertSegment.java b/services/src/main/java/io/druid/cli/InsertSegment.java index f6b1850d2b6f..ad585af418ae 100644 --- a/services/src/main/java/io/druid/cli/InsertSegment.java +++ b/services/src/main/java/io/druid/cli/InsertSegment.java @@ -29,7 +29,10 @@ import com.google.inject.Module; import io.airlift.airline.Command; import io.airlift.airline.Option; +import io.druid.guice.DruidProcessingModule; import io.druid.guice.JsonConfigProvider; +import io.druid.guice.QueryRunnerFactoryModule; +import io.druid.guice.QueryableModule; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Self; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -71,6 +74,12 @@ public InsertSegment() protected List getModules() { return ImmutableList.of( + // It's unknown if those modules are required in InsertSegment. + // Maybe some of those modules could be removed. + // See https://github.com/druid-io/druid/pull/4429#discussion_r123603498 + new DruidProcessingModule(), + new QueryableModule(), + new QueryRunnerFactoryModule(), new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/ResetCluster.java b/services/src/main/java/io/druid/cli/ResetCluster.java index 4ae0dcf22ebe..76c14f59c61a 100644 --- a/services/src/main/java/io/druid/cli/ResetCluster.java +++ b/services/src/main/java/io/druid/cli/ResetCluster.java @@ -26,8 +26,11 @@ import com.google.inject.Module; import io.airlift.airline.Command; import io.airlift.airline.Option; +import io.druid.guice.DruidProcessingModule; import io.druid.guice.IndexingServiceTaskLogsModule; import io.druid.guice.JsonConfigProvider; +import io.druid.guice.QueryRunnerFactoryModule; +import io.druid.guice.QueryableModule; import io.druid.guice.annotations.Self; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.HadoopTask; @@ -76,6 +79,12 @@ public ResetCluster() protected List getModules() { return ImmutableList.of( + // It's unknown if those modules are required in ResetCluster. + // Maybe some of those modules could be removed. + // See https://github.com/druid-io/druid/pull/4429#discussion_r123603498 + new DruidProcessingModule(), + new QueryableModule(), + new QueryRunnerFactoryModule(), new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/ValidateSegments.java b/services/src/main/java/io/druid/cli/ValidateSegments.java index dcac614d921e..b021c201692f 100644 --- a/services/src/main/java/io/druid/cli/ValidateSegments.java +++ b/services/src/main/java/io/druid/cli/ValidateSegments.java @@ -27,6 +27,9 @@ import com.google.inject.name.Names; import io.airlift.airline.Arguments; import io.airlift.airline.Command; +import io.druid.guice.DruidProcessingModule; +import io.druid.guice.QueryRunnerFactoryModule; +import io.druid.guice.QueryableModule; import io.druid.java.util.common.IAE; import io.druid.java.util.common.logger.Logger; import io.druid.query.DruidProcessingConfig; @@ -72,6 +75,12 @@ public void run() { protected List getModules() { return ImmutableList.of( + // It's unknown if those modules are required in ValidateSegments. + // Maybe some of those modules could be removed. + // See https://github.com/druid-io/druid/pull/4429#discussion_r123603498 + new DruidProcessingModule(), + new QueryableModule(), + new QueryRunnerFactoryModule(), new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/Version.java b/services/src/main/java/io/druid/cli/Version.java index c06d8fb3125a..71da1eb525c3 100644 --- a/services/src/main/java/io/druid/cli/Version.java +++ b/services/src/main/java/io/druid/cli/Version.java @@ -33,6 +33,6 @@ public class Version implements Runnable @Override public void run() { - System.out.println(new StatusResource.Status(Initialization.getLoadedModules(DruidModule.class))); + System.out.println(new StatusResource.Status(Initialization.getLoadedImplementations(DruidModule.class))); } } diff --git a/services/src/main/java/io/druid/cli/validate/DruidJsonValidator.java b/services/src/main/java/io/druid/cli/validate/DruidJsonValidator.java index 21d05db7c508..99e7abaa2b2c 100644 --- a/services/src/main/java/io/druid/cli/validate/DruidJsonValidator.java +++ b/services/src/main/java/io/druid/cli/validate/DruidJsonValidator.java @@ -31,17 +31,19 @@ import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.name.Names; - import io.airlift.airline.Command; import io.airlift.airline.Option; import io.druid.cli.GuiceRunnable; import io.druid.data.input.InputRow; import io.druid.data.input.impl.StringInputRowParser; +import io.druid.guice.DruidProcessingModule; import io.druid.guice.ExtensionsConfig; import io.druid.guice.FirehoseModule; import io.druid.guice.IndexingServiceFirehoseModule; import io.druid.guice.LocalDataStorageDruidModule; import io.druid.guice.ParsersModule; +import io.druid.guice.QueryRunnerFactoryModule; +import io.druid.guice.QueryableModule; import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.IndexingHadoopModule; import io.druid.indexing.common.task.Task; @@ -94,6 +96,12 @@ public DruidJsonValidator() protected List getModules() { return ImmutableList.of( + // It's unknown if those modules are required in DruidJsonValidator. + // Maybe some of those modules could be removed. + // See https://github.com/druid-io/druid/pull/4429#discussion_r123603498 + new DruidProcessingModule(), + new QueryableModule(), + new QueryRunnerFactoryModule(), new com.google.inject.Module() { @Override diff --git a/services/src/main/java/io/druid/guice/QueryablePeonModule.java b/services/src/main/java/io/druid/guice/QueryablePeonModule.java new file mode 100644 index 000000000000..68b2dc3ad292 --- /dev/null +++ b/services/src/main/java/io/druid/guice/QueryablePeonModule.java @@ -0,0 +1,46 @@ +/* + * 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.fasterxml.jackson.databind.Module; +import com.google.inject.Binder; +import io.druid.initialization.DruidModule; +import io.druid.server.QueryResource; +import io.druid.server.metrics.QueryCountStatsProvider; + +import java.util.Collections; +import java.util.List; + +public class QueryablePeonModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + return Collections.emptyList(); + } + + @Override + public void configure(Binder binder) + { + binder.bind(QueryCountStatsProvider.class).to(QueryResource.class); + Jerseys.addResource(binder, QueryResource.class); + LifecycleModule.register(binder, QueryResource.class); + } +} diff --git a/sql/pom.xml b/sql/pom.xml index 2c1a5fc6e84b..77a2944fc340 100644 --- a/sql/pom.xml +++ b/sql/pom.xml @@ -18,8 +18,7 @@ ~ under the License. --> - + 4.0.0 druid-sql @@ -29,7 +28,7 @@ io.druid druid - 0.10.1-SNAPSHOT + 0.10.1-rc2-SNAPSHOT