From 465225047887f0ad4618172c04c98a64c8c042c9 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 19 Jan 2017 18:33:51 +0900 Subject: [PATCH 01/10] Fine-grained buffer management for group by queries --- .../collections/OrderedMergeSequence.java | 49 ++++++++++-- .../druid/common/guava/CombiningSequence.java | 60 ++++++++++---- .../common/guava/CombiningSequenceTest.java | 2 +- .../common/guava/ComplexSequenceTest.java | 2 +- .../java/util/common/guava/BaseSequence.java | 41 ++++++++++ .../util/common/guava/ConcatSequence.java | 79 ++++++++++++++----- .../guava/ExecutorExecutingSequence.java | 53 +++++++++++++ .../util/common/guava/FilteredSequence.java | 21 +++++ .../java/util/common/guava/LazySequence.java | 16 ++++ .../util/common/guava/LimitedSequence.java | 13 +++ .../util/common/guava/MappedSequence.java | 17 ++++ .../java/util/common/guava/MergeSequence.java | 24 +++++- .../common/guava/ResourceClosingSequence.java | 32 ++++++++ .../java/util/common/guava/Sequence.java | 55 +++++++++++-- .../java/util/common/guava/Sequences.java | 35 ++++++++ .../common/guava/YieldingSequenceBase.java | 15 ++++ .../util/common/guava/ConcatSequenceTest.java | 2 +- .../java/util/common/guava/TestSequence.java | 18 +++++ .../common/guava/UnsupportedSequence.java | 18 +++++ .../druid/query/CPUTimeMetricQueryRunner.java | 46 ++++++++++- .../query/MetricsEmittingQueryRunner.java | 66 +++++++++++++++- .../java/io/druid/query/RetryQueryRunner.java | 20 ++++- .../groupby/epinephelinae/BufferGrouper.java | 18 +++-- .../epinephelinae/ConcurrentGrouper.java | 62 +++++++++++---- .../GroupByMergingQueryRunnerV2.java | 6 +- .../epinephelinae/GroupByQueryEngineV2.java | 4 +- .../epinephelinae/GroupByRowProcessor.java | 47 +++++++---- .../query/groupby/epinephelinae/Grouper.java | 14 +++- .../epinephelinae/RowBasedGrouperHelper.java | 7 +- .../epinephelinae/SpillingGrouper.java | 14 ++-- .../spec/SpecificSegmentQueryRunner.java | 47 +++++++++++ .../segment/ReferenceCountingSequence.java | 10 +++ .../aggregation/AggregationTestHelper.java | 2 +- .../epinephelinae/BufferGrouperTest.java | 10 ++- .../spec/SpecificSegmentQueryRunnerTest.java | 17 +++- .../firehose/IngestSegmentFirehose.java | 2 +- .../java/io/druid/server/QueryResource.java | 2 +- .../coordination/ServerManagerTest.java | 10 +++ .../main/java/io/druid/cli/DumpSegment.java | 4 +- .../sql/calcite/planner/PlannerConfig.java | 1 + .../io/druid/sql/calcite/rel/QueryMaker.java | 8 +- .../druid/sql/calcite/util/CalciteTests.java | 4 +- 42 files changed, 845 insertions(+), 128 deletions(-) diff --git a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java index f5e8474356bc..43d366184792 100644 --- a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java +++ b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java @@ -20,9 +20,9 @@ package io.druid.collections; import com.google.common.base.Function; +import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Ordering; - import io.druid.java.util.common.guava.Accumulator; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.guava.Sequence; @@ -74,10 +74,44 @@ public OutType accumulate(OutType initValue, Accumulator a } } + @Override + public OutType accumulate( + Supplier initValue, Accumulator accumulator + ) + { + Yielder yielder = null; + try { + yielder = toYielder(initValue, YieldingAccumulators.fromAccumulator(accumulator)); + return yielder.get(); + } + finally { + CloseQuietly.close(yielder); + } + } + @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { - PriorityQueue> pQueue = new PriorityQueue>( + final PriorityQueue> pQueue = makePriorityQueue(); + final Yielder> oldDudeAtCrosswalk = makeOldDudeAtCrosswalk(); + + return makeYielder(pQueue, oldDudeAtCrosswalk, initValue, accumulator); + } + + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + final PriorityQueue> pQueue = makePriorityQueue(); + final Yielder> oldDudeAtCrosswalk = makeOldDudeAtCrosswalk(); + + return makeYielder(pQueue, oldDudeAtCrosswalk, initValue.get(), accumulator); + } + + private PriorityQueue> makePriorityQueue() + { + return new PriorityQueue>( 32, ordering.onResultOf( new Function, T>() @@ -90,16 +124,19 @@ public T apply(Yielder input) } ) ); + } - Yielder> oldDudeAtCrosswalk = sequences.toYielder( - null, + private Yielder> makeOldDudeAtCrosswalk() + { + return sequences.toYielder( + (Yielder) null, new YieldingAccumulator, Sequence>() { @Override public Yielder accumulate(Yielder accumulated, Sequence in) { final Yielder retVal = in.toYielder( - null, + (T) null, new YieldingAccumulator() { @Override @@ -128,8 +165,6 @@ public T accumulate(T accumulated, T in) } } ); - - return makeYielder(pQueue, oldDudeAtCrosswalk, initValue, accumulator); } private Yielder makeYielder( diff --git a/common/src/main/java/io/druid/common/guava/CombiningSequence.java b/common/src/main/java/io/druid/common/guava/CombiningSequence.java index e3ccc40453a4..757594de14d9 100644 --- a/common/src/main/java/io/druid/common/guava/CombiningSequence.java +++ b/common/src/main/java/io/druid/common/guava/CombiningSequence.java @@ -19,6 +19,8 @@ package io.druid.common.guava; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.collect.Ordering; import io.druid.java.util.common.guava.Accumulator; import io.druid.java.util.common.guava.Sequence; @@ -60,8 +62,11 @@ public CombiningSequence( @Override public OutType accumulate(OutType initValue, final Accumulator accumulator) { - final CombiningAccumulator combiningAccumulator = new CombiningAccumulator<>(initValue, accumulator); - T lastValue = baseSequence.accumulate(null, combiningAccumulator); + final CombiningAccumulator combiningAccumulator = new CombiningAccumulator<>( + Suppliers.ofInstance(initValue), + accumulator + ); + T lastValue = baseSequence.accumulate((T) null, combiningAccumulator); if (combiningAccumulator.accumulatedSomething()) { return accumulator.accumulate(combiningAccumulator.retVal, lastValue); } else { @@ -69,15 +74,37 @@ public OutType accumulate(OutType initValue, final Accumulator OutType accumulate( + Supplier initValue, Accumulator accumulator + ) + { + final CombiningAccumulator combiningAccumulator = new CombiningAccumulator<>(initValue, accumulator); + T lastValue = baseSequence.accumulate((T) null, combiningAccumulator); + if (combiningAccumulator.accumulatedSomething()) { + return accumulator.accumulate(combiningAccumulator.retVal, lastValue); + } else { + return initValue.get(); + } + } + @Override public Yielder toYielder(OutType initValue, final YieldingAccumulator accumulator) + { + return toYielder(Suppliers.ofInstance(initValue), accumulator); + } + + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) { final CombiningYieldingAccumulator combiningAccumulator = new CombiningYieldingAccumulator<>( ordering, mergeFn, accumulator ); - combiningAccumulator.setRetVal(initValue); - Yielder baseYielder = baseSequence.toYielder(null, combiningAccumulator); + combiningAccumulator.setInitValSupplier(initValue); + Yielder baseYielder = baseSequence.toYielder((T) null, combiningAccumulator); return makeYielder(baseYielder, combiningAccumulator, false); } @@ -92,23 +119,22 @@ private Yielder makeYielder( final OutType retVal; final boolean finalFinalValue; - if(!yielder.isDone()) { + if (!yielder.isDone()) { retVal = combiningAccumulator.getRetVal(); finalYielder = null; finalFinalValue = false; } else { - if(!finalValue && combiningAccumulator.accumulatedSomething()) { + if (!finalValue && combiningAccumulator.accumulatedSomething()) { combiningAccumulator.accumulateLastValue(); retVal = combiningAccumulator.getRetVal(); finalFinalValue = true; - if(!combiningAccumulator.yielded()) { + if (!combiningAccumulator.yielded()) { return Yielders.done(retVal, yielder); } else { finalYielder = Yielders.done(null, yielder); } - } - else { + } else { return Yielders.done(combiningAccumulator.getRetVal(), yielder); } } @@ -154,6 +180,7 @@ private static class CombiningYieldingAccumulator extends YieldingAc private final YieldingAccumulator accumulator; private OutType retVal; + private Supplier initValSupplier; private T lastMergedVal; private boolean accumulatedSomething = false; @@ -170,12 +197,12 @@ private static class CombiningYieldingAccumulator extends YieldingAc public OutType getRetVal() { - return retVal; + return accumulatedSomething ? retVal : initValSupplier.get(); } - public void setRetVal(OutType retVal) + public void setInitValSupplier(Supplier initValSupplier) { - this.retVal = retVal; + this.initValSupplier = initValSupplier; } public YieldingAccumulator getAccumulator() @@ -203,6 +230,7 @@ public T accumulate(T prevValue, T t) { if (!accumulatedSomething) { accumulatedSomething = true; + retVal = initValSupplier.get(); } if (prevValue == null) { @@ -222,7 +250,7 @@ public T accumulate(T prevValue, T t) void accumulateLastValue() { - retVal = accumulator.accumulate(retVal, lastMergedVal); + retVal = accumulator.accumulate(getRetVal(), lastMergedVal); } boolean accumulatedSomething() @@ -234,13 +262,14 @@ boolean accumulatedSomething() private class CombiningAccumulator implements Accumulator { private OutType retVal; + private Supplier initValSupplier; private final Accumulator accumulator; private volatile boolean accumulatedSomething = false; - CombiningAccumulator(OutType retVal, Accumulator accumulator) + CombiningAccumulator(Supplier initValSupplier, Accumulator accumulator) { - this.retVal = retVal; + this.initValSupplier = initValSupplier; this.accumulator = accumulator; } @@ -254,6 +283,7 @@ public T accumulate(T prevValue, T t) { if (!accumulatedSomething) { accumulatedSomething = true; + retVal = initValSupplier.get(); } if (prevValue == null) { diff --git a/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java b/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java index 974d18da4ff9..9793449f7ac1 100644 --- a/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java +++ b/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java @@ -261,7 +261,7 @@ public Pair apply( Assert.assertEquals(expected, merged); Yielder> yielder = seq.toYielder( - null, + (Pair) null, new YieldingAccumulator, Pair>() { int count = 0; diff --git a/common/src/test/java/io/druid/common/guava/ComplexSequenceTest.java b/common/src/test/java/io/druid/common/guava/ComplexSequenceTest.java index 0e7db26ffb8e..528920042461 100644 --- a/common/src/test/java/io/druid/common/guava/ComplexSequenceTest.java +++ b/common/src/test/java/io/druid/common/guava/ComplexSequenceTest.java @@ -54,7 +54,7 @@ private void check(String expected, Sequence complex) Assert.assertEquals(expected, combined.toString()); Yielder yielder = complex.toYielder( - null, + (Integer) null, new YieldingAccumulator() { @Override diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java index e4f49a1d45a0..67baf76104b9 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java @@ -19,6 +19,7 @@ package io.druid.java.util.common.guava; +import com.google.common.base.Supplier; import com.google.common.base.Throwables; import io.druid.java.util.common.logger.Logger; @@ -65,6 +66,21 @@ public BaseSequence( public OutType accumulate(OutType initValue, final Accumulator fn) { IterType iterator = maker.make(); + return accumulate(iterator, initValue, fn); + } + + @Override + public OutType accumulate(Supplier initValue, final Accumulator fn) + { + final IterType iterator = maker.make(); + // initValue.get() is called here to guarantee some kind of initialization for initValue is executed + // after making the iterator + final OutType retVal = initValue.get(); + return accumulate(iterator, retVal, fn); + } + + private OutType accumulate(IterType iterator, OutType initValue, final Accumulator fn) + { try { while (iterator.hasNext()) { initValue = fn.accumulate(initValue, iterator.next()); @@ -97,6 +113,31 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat } } + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + final IterType iterator = maker.make(); + + try { + // initValue.get() is called here to guarantee some kind of initialization for initValue is executed + // after making the iterator + return makeYielder(initValue.get(), accumulator, iterator); + } + catch (Exception e) { + // We caught an Exception instead of returning a really, real, live, real boy, errr, iterator + // So we better try to close our stuff, 'cause the exception is what is making it out of here. + try { + maker.cleanup(iterator); + } + catch (RuntimeException e1) { + log.error(e1, "Exception thrown when closing maker. Logging and ignoring."); + } + throw Throwables.propagate(e); + } + } + private Yielder makeYielder( OutType initValue, final YieldingAccumulator accumulator, diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java index 2c8f461f0738..0ed08a8b311d 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java @@ -19,6 +19,8 @@ package io.druid.java.util.common.guava; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import java.io.IOException; @@ -41,13 +43,30 @@ public OutType accumulate(OutType initValue, final Accumulator>() - { - @Override - public OutType accumulate(OutType accumulated, Sequence in) - { - return in.accumulate(accumulated, accumulator); - } - } + { + @Override + public OutType accumulate(OutType accumulated, Sequence in) + { + return in.accumulate(accumulated, accumulator); + } + } + ); + } + + @Override + public OutType accumulate( + final Supplier initValue, final Accumulator accumulator + ) + { + return baseSequences.accumulate( + initValue, new Accumulator>() + { + @Override + public OutType accumulate(OutType accumulated, Sequence in) + { + return in.accumulate(accumulated, accumulator); + } + } ); } @@ -56,9 +75,17 @@ public Yielder toYielder( final OutType initValue, final YieldingAccumulator accumulator ) + { + return toYielder(Suppliers.ofInstance(initValue), accumulator); + } + + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) { Yielder> yielderYielder = baseSequences.toYielder( - null, + (Sequence) null, new YieldingAccumulator, Sequence>() { @Override @@ -83,21 +110,20 @@ public Sequence accumulate(Sequence accumulated, Sequence in) public Yielder makeYielder( Yielder> yielderYielder, - OutType initValue, + Supplier initValSupplier, YieldingAccumulator accumulator ) { if (yielderYielder.isDone()) { - return Yielders.done(initValue, yielderYielder); - } - - while (!yielderYielder.isDone()) { - Yielder yielder = yielderYielder.get().toYielder(initValue, accumulator); + return Yielders.done(initValSupplier.get(), yielderYielder); + } else { + // pass the supplier to get the first Yielder + Yielder yielder = yielderYielder.get().toYielder(initValSupplier, accumulator); if (accumulator.yielded()) { return wrapYielder(yielder, yielderYielder, accumulator); } - initValue = yielder.get(); + OutType initVal = yielder.get(); try { yielder.close(); } @@ -106,9 +132,26 @@ public Yielder makeYielder( } yielderYielder = yielderYielder.next(null); - } - return Yielders.done(initValue, yielderYielder); + while (!yielderYielder.isDone()) { + yielder = yielderYielder.get().toYielder(initVal, accumulator); + if (accumulator.yielded()) { + return wrapYielder(yielder, yielderYielder, accumulator); + } + + initVal = yielder.get(); + try { + yielder.close(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + + yielderYielder = yielderYielder.next(null); + } + + return Yielders.done(initVal, yielderYielder); + } } private Yielder wrapYielder( @@ -126,7 +169,7 @@ private Yielder wrapYielder( throw Throwables.propagate(e); } - return makeYielder(yielderYielder.next(null), nextInit, accumulator); + return makeYielder(yielderYielder.next(null), Suppliers.ofInstance(nextInit), accumulator); } return new Yielder() diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/ExecutorExecutingSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/ExecutorExecutingSequence.java index d85d8f8a6f27..da6f6e4b84d6 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/ExecutorExecutingSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/ExecutorExecutingSequence.java @@ -19,6 +19,7 @@ package io.druid.java.util.common.guava; +import com.google.common.base.Supplier; import com.google.common.base.Throwables; import java.io.IOException; @@ -67,6 +68,32 @@ public OutType call() throws Exception } } + @Override + public OutType accumulate( + final Supplier initValue, final Accumulator accumulator + ) + { + Future future = exec.submit( + new Callable() + { + @Override + public OutType call() throws Exception + { + return sequence.accumulate(initValue, accumulator); + } + } + ); + try { + return future.get(); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + catch (ExecutionException e) { + throw Throwables.propagate(e); + } + } + @Override public Yielder toYielder(final OutType initValue, final YieldingAccumulator accumulator) { @@ -91,6 +118,32 @@ public Yielder call() throws Exception } } + @Override + public Yielder toYielder( + final Supplier initValue, final YieldingAccumulator accumulator + ) + { + Future> future = exec.submit( + new Callable>() + { + @Override + public Yielder call() throws Exception + { + return makeYielder(sequence.toYielder(initValue, accumulator)); + } + } + ); + try { + return future.get(); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + catch (ExecutionException e) { + throw Throwables.propagate(e); + } + } + private Yielder makeYielder(final Yielder yielder) { return new Yielder() diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/FilteredSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/FilteredSequence.java index 8fc52344403b..fe805f497d77 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/FilteredSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/FilteredSequence.java @@ -20,6 +20,7 @@ package io.druid.java.util.common.guava; import com.google.common.base.Predicate; +import com.google.common.base.Supplier; import java.io.IOException; @@ -45,6 +46,14 @@ public OutType accumulate(OutType initValue, Accumulator a return baseSequence.accumulate(initValue, new FilteringAccumulator<>(pred, accumulator)); } + @Override + public OutType accumulate( + Supplier initValue, Accumulator accumulator + ) + { + return baseSequence.accumulate(initValue, new FilteringAccumulator<>(pred, accumulator)); + } + @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { @@ -55,6 +64,18 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat return wrapYielder(baseSequence.toYielder(initValue, filteringAccumulator), filteringAccumulator); } + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + final FilteringYieldingAccumulator filteringAccumulator = new FilteringYieldingAccumulator<>( + pred, accumulator + ); + + return wrapYielder(baseSequence.toYielder(initValue, filteringAccumulator), filteringAccumulator); + } + private Yielder wrapYielder( final Yielder yielder, final FilteringYieldingAccumulator accumulator ) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/LazySequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/LazySequence.java index d832a6a95bb8..75b7d56a927c 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/LazySequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/LazySequence.java @@ -40,9 +40,25 @@ public OutType accumulate(OutType initValue, Accumulator a return provider.get().accumulate(initValue, accumulator); } + @Override + public OutType accumulate( + Supplier initValue, Accumulator accumulator + ) + { + return provider.get().accumulate(initValue, accumulator); + } + @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { return provider.get().toYielder(initValue, accumulator); } + + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + return provider.get().toYielder(initValue, accumulator); + } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java index bf412259b647..0df526526260 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java @@ -20,6 +20,7 @@ package io.druid.java.util.common.guava; import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; import java.io.IOException; @@ -54,6 +55,18 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat return new LimitedYielder<>(subYielder, limitedAccumulator); } + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + final LimitedYieldingAccumulator limitedAccumulator = new LimitedYieldingAccumulator<>( + accumulator + ); + final Yielder subYielder = baseSequence.toYielder(initValue, limitedAccumulator); + return new LimitedYielder<>(subYielder, limitedAccumulator); + } + private class LimitedYielder implements Yielder { private final Yielder subYielder; diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MappedSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/MappedSequence.java index 31d189b72312..f9c3a42e8a0e 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MappedSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MappedSequence.java @@ -20,6 +20,7 @@ package io.druid.java.util.common.guava; import com.google.common.base.Function; +import com.google.common.base.Supplier; /** */ @@ -43,9 +44,25 @@ public OutType accumulate(OutType initValue, Accumulator return baseSequence.accumulate(initValue, new MappingAccumulator<>(fn, accumulator)); } + @Override + public OutType accumulate( + Supplier initValue, Accumulator accumulator + ) + { + return baseSequence.accumulate(initValue, new MappingAccumulator<>(fn, accumulator)); + } + @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { return baseSequence.toYielder(initValue, new MappingYieldingAccumulator<>(fn, accumulator)); } + + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + return baseSequence.toYielder(initValue, new MappingYieldingAccumulator<>(fn, accumulator)); + } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java index d0597415e332..b248b264d01b 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java @@ -20,6 +20,7 @@ package io.druid.java.util.common.guava; import com.google.common.base.Function; +import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Ordering; @@ -44,6 +45,23 @@ public MergeSequence( @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) + { + final PriorityQueue> pQueue = makePriorityQueue(); + + return makeYielder(pQueue, initValue, accumulator); + } + + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + final PriorityQueue> pQueue = makePriorityQueue(); + + return makeYielder(pQueue, initValue.get(), accumulator); + } + + private PriorityQueue> makePriorityQueue() { PriorityQueue> pQueue = new PriorityQueue<>( 32, @@ -59,7 +77,7 @@ public T apply(Yielder input) ) ); - pQueue = baseSequences.accumulate( + return baseSequences.accumulate( pQueue, new Accumulator>, Sequence>() { @@ -67,7 +85,7 @@ public T apply(Yielder input) public PriorityQueue> accumulate(PriorityQueue> queue, Sequence in) { final Yielder yielder = in.toYielder( - null, + (T) null, new YieldingAccumulator() { @Override @@ -94,8 +112,6 @@ public T accumulate(T accumulated, T in) } } ); - - return makeYielder(pQueue, initValue, accumulator); } private Yielder makeYielder( diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/ResourceClosingSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/ResourceClosingSequence.java index 8c9ff8070f74..d299bd5aad1c 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/ResourceClosingSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/ResourceClosingSequence.java @@ -19,6 +19,8 @@ package io.druid.java.util.common.guava; +import com.google.common.base.Supplier; + import java.io.Closeable; /** @@ -45,6 +47,19 @@ public OutType accumulate(OutType initValue, Accumulator a } } + @Override + public OutType accumulate( + Supplier initValue, Accumulator accumulator + ) + { + try { + return baseSequence.accumulate(initValue, accumulator); + } + finally { + CloseQuietly.close(closeable); + } + } + @Override public Yielder toYielder( OutType initValue, YieldingAccumulator accumulator @@ -61,4 +76,21 @@ public Yielder toYielder( return new ResourceClosingYielder<>(baseYielder, closeable); } + + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + final Yielder baseYielder; + try { + baseYielder = baseSequence.toYielder(initValue, accumulator); + } + catch (RuntimeException e) { + CloseQuietly.close(closeable); + throw e; + } + + return new ResourceClosingYielder<>(baseYielder, closeable); + } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java index 4b287e937bdd..477720d6e3d7 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java @@ -19,21 +19,64 @@ package io.druid.java.util.common.guava; +import com.google.common.base.Supplier; + /** - * A Sequence represents an iterable sequence of elements. Unlike normal Iterators however, it doesn't expose + * A Sequence represents an iterable sequence of elements. Unlike normal Iterators however, it doesn't expose * a way for you to extract values from it, instead you provide it with a worker (an Accumulator) and that defines * what happens with the data. * - * This inversion of control is in place to allow the Sequence to do resource management. It can enforce that close() - * methods get called and other resources get cleaned up whenever processing is complete. Without this inversion + * This inversion of control is in place to allow the Sequence to do resource management. It can enforce that close() + * methods get called and other resources get cleaned up whenever processing is complete. Without this inversion * it is very easy to unintentionally leak resources when iterating over something that is backed by a resource. * * Sequences also expose {#see com.metamx.common.guava.Yielder} Yielder objects which allow you to implement a - * continuation over the Sequence. Yielder do not offer the same guarantees of automagic resource management + * continuation over the Sequence. Yielder do not offer the same guarantees of automatic resource management * as the accumulate method, but they are Closeable and will do the proper cleanup when close() is called on them. */ public interface Sequence { - public OutType accumulate(OutType initValue, Accumulator accumulator); - public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator); + /** + * Accumulate this sequence using the given accumulator. + * + * @param initValue the initial value to pass along to start the accumulation. + * @param accumulator the accumulator which is responsible for accumulating input values. + * @param the type of accumulated value. + * @return + */ + OutType accumulate(OutType initValue, Accumulator accumulator); + + /** + * Accumulate this sequence using the given accumulator. + * The {@code initValSupplier} provides an way for lazy evaluation of the initial value. + * + * @param initValSupplier the supplier which returns an initial value to pass along to start the accumulation. + * @param accumulator the accumulator which is responsible for accumulating input values. + * @param the type of accumulated value. + * @return + */ + OutType accumulate(Supplier initValSupplier, Accumulator accumulator); + + /** + * Return an Yielder for accumulated sequence. + * + * @param initValue the initial value to pass along to start the accumulation. + * @param accumulator the accumulator which is responsible for accumulating input values. + * @param the type of accumulated value. + * @see Yielder + * @return an Yielder for accumulated sequence. + */ + Yielder toYielder(OutType initValue, YieldingAccumulator accumulator); + + /** + * Return an Yielder for accumulated sequence. + * The {@code initValSupplier} provides an way for lazy evaluation of the initial value. + * + * @param initValSupplier the supplier which returns an initial value to pass along to start the accumulation. + * @param accumulator the accumulator which is responsible for accumulating input values. + * @param the type of accumulated value. + * @see Yielder + * @return an Yielder for accumulated sequence. + */ + Yielder toYielder(Supplier initValSupplier, YieldingAccumulator accumulator); } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java index 942b51d2ef4c..caa470bbf439 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java @@ -21,6 +21,7 @@ import com.google.common.base.Function; import com.google.common.base.Predicate; +import com.google.common.base.Supplier; import com.google.common.collect.Lists; import java.io.Closeable; @@ -95,11 +96,29 @@ public OutType accumulate(OutType initValue, Accumulator a return out; } + @Override + public OutType accumulate( + Supplier initValue, Accumulator accumulator + ) + { + final OutType out = seq.accumulate(initValue, accumulator); + exec.execute(effect); + return out; + } + @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { return new ExecuteWhenDoneYielder<>(seq.toYielder(initValue, accumulator), effect, exec); } + + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + return new ExecuteWhenDoneYielder<>(seq.toYielder(initValue, accumulator), effect, exec); + } }; } @@ -124,10 +143,26 @@ public OutType accumulate(OutType initValue, Accumulator OutType accumulate( + Supplier initValue, Accumulator accumulator + ) + { + return initValue.get(); + } + @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { return Yielders.done(initValue, null); } + + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + return Yielders.done(initValue.get(), null); + } } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java b/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java index c3150c0a818c..3e9174d58362 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java @@ -19,6 +19,8 @@ package io.druid.java.util.common.guava; +import com.google.common.base.Supplier; + /** * A Sequence that is based entirely on the Yielder implementation. *

@@ -38,4 +40,17 @@ public OutType accumulate(OutType initValue, Accumulator a CloseQuietly.close(yielder); } } + + @Override + public OutType accumulate(Supplier initValue, Accumulator accumulator) + { + Yielder yielder = toYielder(initValue, YieldingAccumulators.fromAccumulator(accumulator)); + + try { + return yielder.get(); + } + finally { + CloseQuietly.close(yielder); + } + } } diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/ConcatSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/ConcatSequenceTest.java index bf2eb3761568..b2af7bcae22b 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/ConcatSequenceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/ConcatSequenceTest.java @@ -256,7 +256,7 @@ public void remove() ); Yielder yielder = seq.toYielder( - null, + (Integer) null, new YieldingAccumulator() { @Override diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java b/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java index b77a750a98ae..67481f90f7b6 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java @@ -19,6 +19,8 @@ package io.druid.java.util.common.guava; +import com.google.common.base.Supplier; + import java.util.Arrays; import java.util.Iterator; import java.util.concurrent.atomic.AtomicBoolean; @@ -65,12 +67,28 @@ public OutType accumulate(OutType initValue, Accumulator a return base.accumulate(initValue, accumulator); } + @Override + public OutType accumulate( + Supplier initValue, Accumulator accumulator + ) + { + return base.accumulate(initValue, accumulator); + } + @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { return base.toYielder(initValue, accumulator); } + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + return base.toYielder(initValue, accumulator); + } + public boolean isClosed() { return closed.get(); diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/UnsupportedSequence.java b/java-util/src/test/java/io/druid/java/util/common/guava/UnsupportedSequence.java index f410fbefdc1e..c56f367c244d 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/UnsupportedSequence.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/UnsupportedSequence.java @@ -19,6 +19,8 @@ package io.druid.java.util.common.guava; +import com.google.common.base.Supplier; + /** */ public class UnsupportedSequence implements Sequence @@ -31,6 +33,14 @@ public OutType accumulate( throw new UnsupportedOperationException(); } + @Override + public OutType accumulate( + Supplier initValue, Accumulator accumulator + ) + { + throw new UnsupportedOperationException(); + } + @Override public Yielder toYielder( OutType initValue, YieldingAccumulator accumulator @@ -38,4 +48,12 @@ public Yielder toYielder( { throw new UnsupportedOperationException(); } + + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + throw new UnsupportedOperationException(); + } } diff --git a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java index a2366365641e..045e27857149 100644 --- a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java +++ b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java @@ -22,6 +22,7 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.common.utils.VMUtils; @@ -86,12 +87,53 @@ public OutType accumulate(OutType initValue, Accumulator a } @Override - public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) + public OutType accumulate( + Supplier initValue, Accumulator accumulator + ) + { + final long start = VMUtils.getCurrentThreadCpuTime(); + try { + return baseSequence.accumulate(initValue, accumulator); + } + finally { + cpuTimeAccumulator.addAndGet(VMUtils.getCurrentThreadCpuTime() - start); + } + } + + @Override + public Yielder toYielder(final OutType initValue, final YieldingAccumulator accumulator) + { + return toYielder(new Supplier>() + { + @Override + public Yielder get() + { + return baseSequence.toYielder(initValue, accumulator); + } + }); + } + + @Override + public Yielder toYielder( + final Supplier initValue, final YieldingAccumulator accumulator + ) + { + return toYielder(new Supplier>() + { + @Override + public Yielder get() + { + return baseSequence.toYielder(initValue, accumulator); + } + }); + } + + private Yielder toYielder(Supplier> supplier) { final long start = VMUtils.getCurrentThreadCpuTime(); final Yielder delegateYielder; try { - delegateYielder = baseSequence.toYielder(initValue, accumulator); + delegateYielder = supplier.get(); } finally { cpuTimeAccumulator.addAndGet(VMUtils.getCurrentThreadCpuTime() - start); diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index a7558bbfa413..ba361d796ea2 100644 --- a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java @@ -20,6 +20,7 @@ package io.druid.query; import com.google.common.base.Function; +import com.google.common.base.Supplier; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.java.util.common.guava.Accumulator; @@ -93,13 +94,40 @@ public Sequence run(final Query query, final Map responseC return new Sequence() { @Override - public OutType accumulate(OutType outType, Accumulator accumulator) + public OutType accumulate(final OutType outType, final Accumulator accumulator) + { + return accumulate(new Supplier() + { + @Override + public OutType get() + { + return queryRunner.run(query, responseContext).accumulate(outType, accumulator); + } + }); + } + + @Override + public OutType accumulate( + final Supplier initValue, final Accumulator accumulator + ) + { + return accumulate(new Supplier() + { + @Override + public OutType get() + { + return queryRunner.run(query, responseContext).accumulate(initValue, accumulator); + } + }); + } + + private OutType accumulate(Supplier retValSupplier) { OutType retVal; long startTime = System.currentTimeMillis(); try { - retVal = queryRunner.run(query, responseContext).accumulate(outType, accumulator); + retVal = retValSupplier.get(); } catch (RuntimeException e) { builder.setDimension(DruidMetrics.STATUS, "failed"); @@ -123,13 +151,43 @@ public OutType accumulate(OutType outType, Accumulator acc } @Override - public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) + public Yielder toYielder( + final OutType initValue, + final YieldingAccumulator accumulator + ) + { + return toYielder(new Supplier>() + { + @Override + public Yielder get() + { + return queryRunner.run(query, responseContext).toYielder(initValue, accumulator); + } + }); + } + + @Override + public Yielder toYielder( + final Supplier initValue, final YieldingAccumulator accumulator + ) + { + return toYielder(new Supplier>() + { + @Override + public Yielder get() + { + return queryRunner.run(query, responseContext).toYielder(initValue, accumulator); + } + }); + } + + private Yielder toYielder(Supplier> supplier) { Yielder retVal; long startTime = System.currentTimeMillis(); try { - retVal = queryRunner.run(query, responseContext).toYielder(initValue, accumulator); + retVal = supplier.get(); } catch (RuntimeException e) { builder.setDimension(DruidMetrics.STATUS, "failed"); diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java index 57a5cde5124b..35e31e268c96 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -72,6 +73,19 @@ public Sequence run(final Query query, final Map context) public Yielder toYielder( OutType initValue, YieldingAccumulator accumulator ) + { + return makeSequence().toYielder(initValue, accumulator); + } + + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + return makeSequence().toYielder(initValue, accumulator); + } + + private Sequence makeSequence() { List missingSegments = getMissingSegments(context); @@ -100,12 +114,10 @@ public Yielder toYielder( return new MergeSequence<>( query.getResultOrdering(), - Sequences.simple(listOfSequences)).toYielder( - initValue, accumulator - ); + Sequences.simple(listOfSequences)); } else { - return Iterables.getOnlyElement(listOfSequences).toYielder(initValue, accumulator); + return Iterables.getOnlyElement(listOfSequences); } } }; diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java index a2dcd4ed815b..e356867e3941 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java @@ -19,6 +19,7 @@ package io.druid.query.groupby.epinephelinae; +import com.google.common.base.Supplier; import com.google.common.primitives.Ints; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; @@ -64,17 +65,19 @@ public class BufferGrouper implements Grouper private static final float DEFAULT_MAX_LOAD_FACTOR = 0.7f; private static final int HASH_SIZE = Ints.BYTES; - private final ByteBuffer buffer; + private final Supplier bufferSupplier; private final KeySerde keySerde; private final int keySize; private final BufferAggregator[] aggregators; private final int[] aggregatorOffsets; private final int initialBuckets; private final int bucketSize; - private final int tableArenaSize; private final int bufferGrouperMaxSize; // Integer.MAX_VALUE in production, only used for unit tests private final float maxLoadFactor; + private ByteBuffer buffer; + private int tableArenaSize = -1; + // Buffer pointing to the current table (it moves around as the table grows) private ByteBuffer tableBuffer; @@ -91,7 +94,7 @@ public class BufferGrouper implements Grouper private int maxSize; public BufferGrouper( - final ByteBuffer buffer, + final Supplier bufferSupplier, final KeySerde keySerde, final ColumnSelectorFactory columnSelectorFactory, final AggregatorFactory[] aggregatorFactories, @@ -100,7 +103,7 @@ public BufferGrouper( final int initialBuckets ) { - this.buffer = buffer; + this.bufferSupplier = bufferSupplier; this.keySerde = keySerde; this.keySize = keySerde.keySize(); this.aggregators = new BufferAggregator[aggregatorFactories.length]; @@ -121,8 +124,13 @@ public BufferGrouper( } this.bucketSize = offset; - this.tableArenaSize = (buffer.capacity() / (bucketSize + Ints.BYTES)) * bucketSize; + } + @Override + public void init() + { + this.buffer = bufferSupplier.get(); + this.tableArenaSize = (buffer.capacity() / (bucketSize + Ints.BYTES)) * bucketSize; reset(); } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java index 5072b5d5be97..d086155f8234 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java @@ -21,6 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import io.druid.java.util.common.ISE; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.ColumnSelectorFactory; @@ -34,7 +36,7 @@ /** * Grouper based around a set of underlying {@link SpillingGrouper} instances. Thread-safe. - * + *

* The passed-in buffer is cut up into concurrencyHint slices, and each slice is passed to a different underlying * grouper. Access to each slice is separately synchronized. As long as the result set fits in memory, keys are * partitioned between buffers based on their hash, and multiple threads can write into the same buffer. When @@ -50,8 +52,19 @@ public class ConcurrentGrouper implements Grouper private volatile boolean closed = false; private final Comparator keyObjComparator; + private final Supplier bufferSupplier; + private final ColumnSelectorFactory columnSelectorFactory; + private final AggregatorFactory[] aggregatorFactories; + private final int bufferGrouperMaxSize; + private final float bufferGrouperMaxLoadFactor; + private final int bufferGrouperInitialBuckets; + private final LimitedTemporaryStorage temporaryStorage; + private final ObjectMapper spillMapper; + private final int concurrencyHint; + private final KeySerdeFactory keySerdeFactory; + public ConcurrentGrouper( - final ByteBuffer buffer, + final Supplier bufferSupplier, final KeySerdeFactory keySerdeFactory, final ColumnSelectorFactory columnSelectorFactory, final AggregatorFactory[] aggregatorFactories, @@ -75,29 +88,44 @@ protected SpillingGrouper initialValue() } }; + this.bufferSupplier = bufferSupplier; + this.columnSelectorFactory = columnSelectorFactory; + this.aggregatorFactories = aggregatorFactories; + this.bufferGrouperMaxSize = bufferGrouperMaxSize; + this.bufferGrouperMaxLoadFactor = bufferGrouperMaxLoadFactor; + this.bufferGrouperInitialBuckets = bufferGrouperInitialBuckets; + this.temporaryStorage = temporaryStorage; + this.spillMapper = spillMapper; + this.concurrencyHint = concurrencyHint; + this.keySerdeFactory = keySerdeFactory; + this.keyObjComparator = keySerdeFactory.objectComparator(); + } + + @Override + public void init() + { + final ByteBuffer buffer = bufferSupplier.get(); final int sliceSize = (buffer.capacity() / concurrencyHint); for (int i = 0; i < concurrencyHint; i++) { final ByteBuffer slice = buffer.duplicate(); slice.position(sliceSize * i); slice.limit(slice.position() + sliceSize); - groupers.add( - new SpillingGrouper<>( - slice.slice(), - keySerdeFactory, - columnSelectorFactory, - aggregatorFactories, - bufferGrouperMaxSize, - bufferGrouperMaxLoadFactor, - bufferGrouperInitialBuckets, - temporaryStorage, - spillMapper, - false - ) + final SpillingGrouper grouper = new SpillingGrouper<>( + Suppliers.ofInstance(slice.slice()), + keySerdeFactory, + columnSelectorFactory, + aggregatorFactories, + bufferGrouperMaxSize, + bufferGrouperMaxLoadFactor, + bufferGrouperInitialBuckets, + temporaryStorage, + spillMapper, + false ); + grouper.init(); + groupers.add(grouper); } - - this.keyObjComparator = keySerdeFactory.objectComparator(); } @Override diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java index 885186852d63..ca3dad127c4a 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Predicates; +import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -69,7 +70,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -public class GroupByMergingQueryRunnerV2 implements QueryRunner +public class GroupByMergingQueryRunnerV2 implements QueryRunner { private static final Logger log = new Logger(GroupByMergingQueryRunnerV2.class); private static final String CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION = "mergeRunnersUsingChainedExecution"; @@ -177,7 +178,7 @@ public CloseableGrouperIterator make() query, false, config, - mergeBufferHolder.get(), + Suppliers.ofInstance(mergeBufferHolder.get()), concurrencyHint, temporaryStorage, spillMapper, @@ -185,6 +186,7 @@ public CloseableGrouperIterator make() ); final Grouper grouper = pair.lhs; final Accumulator, Row> accumulator = pair.rhs; + grouper.init(); final ReferenceCountingResourceHolder> grouperHolder = ReferenceCountingResourceHolder.fromCloseable(grouper); 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 5c17f5bbc81d..4a6974dcd2a2 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 @@ -21,6 +21,7 @@ import com.google.common.base.Function; import com.google.common.base.Strings; +import com.google.common.base.Suppliers; import com.google.common.collect.Maps; import com.google.common.primitives.Ints; import io.druid.collections.ResourceHolder; @@ -402,7 +403,7 @@ public Row next() } final Grouper grouper = new BufferGrouper<>( - buffer, + Suppliers.ofInstance(buffer), keySerde, cursor, query.getAggregatorSpecs() @@ -411,6 +412,7 @@ public Row next() querySpecificConfig.getBufferGrouperMaxLoadFactor(), querySpecificConfig.getBufferGrouperInitialBuckets() ); + grouper.init(); outer: while (!cursor.isDone()) { diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java index c1c467923414..5c37209b571f 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Predicate; +import com.google.common.base.Supplier; import com.google.common.collect.Lists; import io.druid.collections.BlockingPool; import io.druid.collections.ReferenceCountingResourceHolder; @@ -112,6 +113,7 @@ public boolean apply(Row input) for (Interval queryInterval : queryIntervals) { if (queryInterval.contains(rowTime)) { inInterval = true; + break; } } if (!inInterval) { @@ -139,23 +141,32 @@ public CloseableGrouperIterator make() closeOnFailure.add(temporaryStorage); - final ReferenceCountingResourceHolder mergeBufferHolder; - try { - // This will potentially block if there are no merge buffers left in the pool. - if (timeout <= 0 || (mergeBufferHolder = mergeBufferPool.take(timeout)) == null) { - throw new QueryInterruptedException(new TimeoutException()); - } - closeOnFailure.add(mergeBufferHolder); - } - catch (InterruptedException e) { - throw new QueryInterruptedException(e); - } + final SettableSupplier> bufferHolderSupplier = new SettableSupplier<>(); Pair, Accumulator, Row>> pair = RowBasedGrouperHelper.createGrouperAccumulatorPair( query, true, querySpecificConfig, - mergeBufferHolder.get(), + new Supplier() + { + @Override + public ByteBuffer get() + { + final ReferenceCountingResourceHolder mergeBufferHolder; + try { + if (timeout <= 0 || (mergeBufferHolder = mergeBufferPool.take(timeout)) == null) { + throw new QueryInterruptedException(new TimeoutException()); + } + bufferHolderSupplier.set(mergeBufferHolder); + closeOnFailure.add(mergeBufferHolder); + + return mergeBufferHolder.get(); + } + catch (InterruptedException e) { + throw new QueryInterruptedException(e); + } + } + }, -1, temporaryStorage, spillMapper, @@ -165,7 +176,15 @@ public CloseableGrouperIterator make() final Accumulator, Row> accumulator = pair.rhs; closeOnFailure.add(grouper); - final Grouper retVal = filteredSequence.accumulate(grouper, accumulator); + final Grouper retVal = filteredSequence.accumulate(new Supplier>() + { + @Override + public Grouper get() + { + grouper.init(); + return grouper; + } + }, accumulator); if (retVal != grouper) { throw new ResourceLimitExceededException("Grouping resources exhausted"); } @@ -179,7 +198,7 @@ public CloseableGrouperIterator make() public void close() throws IOException { grouper.close(); - mergeBufferHolder.close(); + CloseQuietly.close(bufferHolderSupplier.get()); CloseQuietly.close(temporaryStorage); } } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java index 922fb61c3b18..3680a11e42d2 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java @@ -32,7 +32,7 @@ * Groupers aggregate metrics from rows that they typically get from a ColumnSelectorFactory, under * grouping keys that some outside driver is passing in. They can also iterate over the grouped * rows after the aggregation is done. - * + *

* They work sort of like a map of KeyType to aggregated values, except they don't support * random lookups. * @@ -40,6 +40,12 @@ */ public interface Grouper extends Closeable { + /** + * Initialize the grouper. + * This method needs to be called before calling {@link #aggregate(Object)} and {@link #aggregate(Object, int)}. + */ + void init(); + /** * Aggregate the current row with the provided key. Some implementations are thread-safe and * some are not. @@ -74,11 +80,11 @@ public interface Grouper extends Closeable /** * Iterate through entries. If a comparator is provided, do a sorted iteration. - * + *

* Once this method is called, writes are no longer safe. After you are done with the iterator returned by this * method, you should either call {@link #close()} (if you are done with the Grouper), {@link #reset()} (if you * want to reuse it), or {@link #iterator(boolean)} again if you want another iterator. - * + *

* If "sorted" is true then the iterator will return sorted results. It will use KeyType's natural ordering on * deserialized objects, and will use the {@link KeySerde#comparator()} on serialized objects. Woe be unto you * if these comparators are not equivalent. @@ -188,7 +194,7 @@ interface KeySerde /** * Serialize a key. This will be called by the {@link #aggregate(Comparable)} method. The buffer will not * be retained after the aggregate method returns, so reusing buffers is OK. - * + *

* This method may return null, which indicates that some internal resource limit has been reached and * no more keys can be generated. In this situation you can call {@link #reset()} and try again, although * beware the caveats on that method. diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index dbb3228b2cac..5135b4b741bb 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -25,6 +25,7 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import com.google.common.base.Supplier; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.primitives.Chars; @@ -62,7 +63,7 @@ public static Pair, Accumulator, Row>> final GroupByQuery query, final boolean isInputRaw, final GroupByQueryConfig config, - final ByteBuffer buffer, + final Supplier bufferSupplier, final int concurrencyHint, final LimitedTemporaryStorage temporaryStorage, final ObjectMapper spillMapper, @@ -88,7 +89,7 @@ public static Pair, Accumulator, Row>> final Grouper grouper; if (concurrencyHint == -1) { grouper = new SpillingGrouper<>( - buffer, + bufferSupplier, keySerdeFactory, columnSelectorFactory, aggregatorFactories, @@ -101,7 +102,7 @@ public static Pair, Accumulator, Row>> ); } else { grouper = new ConcurrentGrouper<>( - buffer, + bufferSupplier, keySerdeFactory, columnSelectorFactory, aggregatorFactories, diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java index addda75d464c..f2561d7b20e9 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java @@ -23,11 +23,11 @@ import com.fasterxml.jackson.databind.MappingIterator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; +import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import io.druid.java.util.common.guava.CloseQuietly; -import io.druid.java.util.common.logger.Logger; import io.druid.query.QueryInterruptedException; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.ColumnSelectorFactory; @@ -51,8 +51,6 @@ */ public class SpillingGrouper implements Grouper { - private static final Logger log = new Logger(SpillingGrouper.class); - private final BufferGrouper grouper; private final KeySerde keySerde; private final LimitedTemporaryStorage temporaryStorage; @@ -66,7 +64,7 @@ public class SpillingGrouper implements Grouper private boolean spillingAllowed = false; public SpillingGrouper( - final ByteBuffer buffer, + final Supplier bufferSupplier, final KeySerdeFactory keySerdeFactory, final ColumnSelectorFactory columnSelectorFactory, final AggregatorFactory[] aggregatorFactories, @@ -81,7 +79,7 @@ public SpillingGrouper( this.keySerde = keySerdeFactory.factorize(); this.keyObjComparator = keySerdeFactory.objectComparator(); this.grouper = new BufferGrouper<>( - buffer, + bufferSupplier, keySerde, columnSelectorFactory, aggregatorFactories, @@ -95,6 +93,12 @@ public SpillingGrouper( this.spillingAllowed = spillingAllowed; } + @Override + public void init() + { + grouper.init(); + } + @Override public boolean aggregate(KeyType key, int keyHash) { diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java index 1683b3de9114..699ba7874293 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -19,6 +19,7 @@ package io.druid.query.spec; +import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import io.druid.java.util.common.guava.Accumulator; @@ -96,6 +97,29 @@ public OutType call() throws Exception ); } + @Override + public OutType accumulate( + final Supplier initValue, final Accumulator accumulator + ) + { + return doItNamed( + new Callable() + { + @Override + public OutType call() throws Exception + { + try { + return baseSequence.accumulate(initValue, accumulator); + } + catch (SegmentMissingException e) { + appendMissingSegment(responseContext); + return initValue.get(); + } + } + } + ); + } + @Override public Yielder toYielder( final OutType initValue, @@ -120,6 +144,29 @@ public Yielder call() throws Exception ); } + @Override + public Yielder toYielder( + final Supplier initValue, final YieldingAccumulator accumulator + ) + { + return doItNamed( + new Callable>() + { + @Override + public Yielder call() throws Exception + { + try { + return makeYielder(baseSequence.toYielder(initValue, accumulator)); + } + catch (SegmentMissingException e) { + appendMissingSegment(responseContext); + return Yielders.done(initValue.get(), null); + } + } + } + ); + } + private Yielder makeYielder(final Yielder yielder) { return new Yielder() diff --git a/processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java b/processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java index dd2e5b1d357d..74823061aeda 100644 --- a/processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java +++ b/processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java @@ -19,6 +19,7 @@ package io.druid.segment; +import com.google.common.base.Supplier; import io.druid.java.util.common.guava.ResourceClosingYielder; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Yielder; @@ -48,4 +49,13 @@ public Yielder toYielder( final Closeable closeable = segment.increment(); return new ResourceClosingYielder(baseSequence.toYielder(initValue, accumulator), closeable); } + + @Override + public Yielder toYielder( + Supplier initValue, YieldingAccumulator accumulator + ) + { + final Closeable closeable = segment.increment(); + return new ResourceClosingYielder(baseSequence.toYielder(initValue, accumulator), closeable); + } } diff --git a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java index fd1dbf351069..2dfcb94f45d0 100644 --- a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java @@ -524,7 +524,7 @@ public Sequence run(Query query, Map map) try { Sequence resultSeq = baseRunner.run(query, Maps.newHashMap()); final Yielder yielder = resultSeq.toYielder( - null, + (Object) null, new YieldingAccumulator() { @Override diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java index 06d20cedc1f7..97e9ae3e6587 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/BufferGrouperTest.java @@ -19,6 +19,7 @@ package io.druid.query.groupby.epinephelinae; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -42,7 +43,7 @@ public void testSimple() { final TestColumnSelectorFactory columnSelectorFactory = GrouperTestUtil.newColumnSelectorFactory(); final Grouper grouper = new BufferGrouper<>( - ByteBuffer.allocate(1000), + Suppliers.ofInstance(ByteBuffer.allocate(1000)), GrouperTestUtil.intKeySerde(), columnSelectorFactory, new AggregatorFactory[]{ @@ -53,6 +54,7 @@ public void testSimple() 0, 0 ); + grouper.init(); columnSelectorFactory.setRow(new MapBasedRow(0, ImmutableMap.of("value", 10L))); grouper.aggregate(12); @@ -148,8 +150,8 @@ private static BufferGrouper makeGrouper( int initialBuckets ) { - return new BufferGrouper<>( - ByteBuffer.allocate(bufferSize), + final BufferGrouper grouper = new BufferGrouper<>( + Suppliers.ofInstance(ByteBuffer.allocate(bufferSize)), GrouperTestUtil.intKeySerde(), columnSelectorFactory, new AggregatorFactory[]{ @@ -160,5 +162,7 @@ private static BufferGrouper makeGrouper( 0.75f, initialBuckets ); + grouper.init(); + return grouper; } } diff --git a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java index 7c85101bba83..ac55da67317f 100644 --- a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java @@ -20,6 +20,7 @@ package io.druid.query.spec; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -72,6 +73,14 @@ public Sequence run(Query query, Map responseContext) { return new Sequence() { + @Override + public Object accumulate( + Supplier initValue, Accumulator accumulator + ) + { + throw new SegmentMissingException("FAILSAUCE"); + } + @Override public Object accumulate(Object initValue, Accumulator accumulator) { @@ -85,6 +94,12 @@ public Yielder toYielder( { throw new SegmentMissingException("FAILSAUCE"); } + + @Override + public Yielder toYielder(Supplier initValue, YieldingAccumulator accumulator) + { + throw new SegmentMissingException("FAILSAUCE"); + } }; } @@ -114,7 +129,7 @@ public Yielder toYielder( responseContext = Maps.newHashMap(); results = queryRunner.run(query, responseContext); results.toYielder( - null, new YieldingAccumulator() + (Object) null, new YieldingAccumulator() { final List lists = Lists.newArrayList(); @Override diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/IngestSegmentFirehose.java b/server/src/main/java/io/druid/segment/realtime/firehose/IngestSegmentFirehose.java index 30e7932d2dec..dfa163b8be34 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/IngestSegmentFirehose.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/IngestSegmentFirehose.java @@ -172,7 +172,7 @@ public void remove() ) ); rowYielder = rows.toYielder( - null, + (InputRow) null, new YieldingAccumulator() { @Override diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index cee47b3def78..fe7abba08c0b 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -227,7 +227,7 @@ public Response doPost( } final Yielder yielder = results.toYielder( - null, + (Object) null, new YieldingAccumulator() { @Override diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index b24ed2ba4287..82dbbece5c38 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -711,6 +713,14 @@ private BlockingSequence( public Yielder toYielder( final OutType initValue, final YieldingAccumulator accumulator ) + { + return toYielder(Suppliers.ofInstance(initValue), accumulator); + } + + @Override + public Yielder toYielder( + final Supplier initValue, final YieldingAccumulator accumulator + ) { notifyLatch.countDown(); diff --git a/services/src/main/java/io/druid/cli/DumpSegment.java b/services/src/main/java/io/druid/cli/DumpSegment.java index 7f1d9f781fff..9ef500fc2c92 100644 --- a/services/src/main/java/io/druid/cli/DumpSegment.java +++ b/services/src/main/java/io/druid/cli/DumpSegment.java @@ -479,7 +479,7 @@ private static Sequence executeQuery(final Injector injector, final Query private static void evaluateSequenceForSideEffects(final Sequence sequence) { sequence.accumulate( - null, + (Object) null, new Accumulator() { @Override @@ -487,6 +487,8 @@ public Object accumulate(Object accumulated, T in) { return null; } + + public void close() {} } ); } diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerConfig.java b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerConfig.java index 4e57b679bfe0..ef8e1858b6b1 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerConfig.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerConfig.java @@ -63,6 +63,7 @@ public int getMaxTopNLimit() return maxTopNLimit; } + // TODO remove this public int getMaxQueryCount() { return maxTopNLimit; diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java index a403c4269dda..5a7bece0fa47 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java @@ -177,7 +177,7 @@ private void executeSelect( final AtomicBoolean gotResult = new AtomicBoolean(); query.run(walker, Maps.newHashMap()).accumulate( - null, + (Object) null, new Accumulator>() { @Override @@ -236,7 +236,7 @@ private void executeTimeseries( Hook.QUERY_PLAN.run(query); query.run(walker, Maps.newHashMap()).accumulate( - null, + (Object) null, new Accumulator>() { @Override @@ -274,7 +274,7 @@ private void executeTopN( Hook.QUERY_PLAN.run(query); query.run(walker, Maps.newHashMap()).accumulate( - null, + (Object) null, new Accumulator>() { @Override @@ -310,7 +310,7 @@ private void executeGroupBy( Hook.QUERY_PLAN.run(query); query.run(walker, Maps.newHashMap()).accumulate( - null, + (Object) null, new Accumulator() { @Override diff --git a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java index be9d0d8dd6f8..157d1e870a1a 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java @@ -226,8 +226,8 @@ public int intermediateComputeSizeBytes() @Override public int getNumMergeBuffers() { - // Need 3 buffers for CalciteQueryTest.testDoubleNestedGroupby. - return 3; + // Need 2 buffers for CalciteQueryTest.testDoubleNestedGroupby. + return 2; } } ) From 1ed6ce2285b2cbf03812f5abe3c33869450d39bc Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 19 Jan 2017 19:30:06 +0900 Subject: [PATCH 02/10] Remove maxQueryCount from GroupByRules --- .../sql/calcite/planner/PlannerConfig.java | 1 - .../druid/sql/calcite/rule/GroupByRules.java | 17 ++++------------- .../druid/sql/calcite/CalciteQueryTest.java | 19 ------------------- 3 files changed, 4 insertions(+), 33 deletions(-) diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerConfig.java b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerConfig.java index ef8e1858b6b1..4e57b679bfe0 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerConfig.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerConfig.java @@ -63,7 +63,6 @@ public int getMaxTopNLimit() return maxTopNLimit; } - // TODO remove this public int getMaxQueryCount() { return maxTopNLimit; diff --git a/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java b/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java index 7cb63918a8b0..1407aea98316 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java +++ b/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java @@ -176,8 +176,7 @@ public void onMatch(RelOptRuleCall call) null, null, aggregate, - plannerConfig.isUseApproximateCountDistinct(), - plannerConfig.getMaxQueryCount() + plannerConfig.isUseApproximateCountDistinct() ); if (newDruidRel != null) { call.transformTo(newDruidRel); @@ -215,8 +214,7 @@ public void onMatch(RelOptRuleCall call) null, project, aggregate, - plannerConfig.isUseApproximateCountDistinct(), - plannerConfig.getMaxQueryCount() + plannerConfig.isUseApproximateCountDistinct() ); if (newDruidRel != null) { call.transformTo(newDruidRel); @@ -256,8 +254,7 @@ public void onMatch(RelOptRuleCall call) filter, project, aggregate, - plannerConfig.isUseApproximateCountDistinct(), - plannerConfig.getMaxQueryCount() + plannerConfig.isUseApproximateCountDistinct() ); if (newDruidRel != null) { call.transformTo(newDruidRel); @@ -367,8 +364,7 @@ private static DruidRel applyAggregate( final Filter filter0, final Project project0, final Aggregate aggregate, - final boolean approximateCountDistinct, - final int maxQueryCount + final boolean approximateCountDistinct ) { Preconditions.checkState(canApplyAggregate(druidRel, filter0, project0, aggregate), "Cannot applyAggregate."); @@ -473,11 +469,6 @@ private static DruidRel applyAggregate( rowOrder ); - // Check maxQueryCount. - if (maxQueryCount > 0 && retVal.getQueryCount() > maxQueryCount) { - return null; - } - return retVal; } else { // groupBy on a base dataSource. diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java index cbfdd78c1615..8ba69b68a9ee 100644 --- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java @@ -1589,25 +1589,6 @@ public void testDoubleNestedGroupBy() throws Exception ); } - @Test - public void testDoubleNestedGroupByForbiddenByConfig() throws Exception - { - assertQueryIsUnplannable( - PLANNER_CONFIG_SINGLE_NESTING_ONLY, - "SELECT SUM(cnt), COUNT(*) FROM (\n" - + " SELECT dim2, SUM(t1.cnt) cnt FROM (\n" - + " SELECT\n" - + " dim1,\n" - + " dim2,\n" - + " COUNT(*) cnt\n" - + " FROM druid.foo\n" - + " GROUP BY dim1, dim2\n" - + " ) t1\n" - + " GROUP BY dim2\n" - + ") t2" - ); - } - @Test public void testExactCountDistinctUsingSubquery() throws Exception { From 6d3f53388b0e292b36d9e279628a650efd931923 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 19 Jan 2017 19:51:04 +0900 Subject: [PATCH 03/10] Fix code style --- .../collections/OrderedMergeSequence.java | 23 +++++-------- .../guava/ExecutorExecutingSequence.java | 7 ++-- .../java/util/common/guava/Sequence.java | 34 +++++++++++-------- .../java/util/common/guava/Sequences.java | 2 +- .../druid/query/CPUTimeMetricQueryRunner.java | 5 ++- .../java/io/druid/query/RetryQueryRunner.java | 6 ++-- 6 files changed, 42 insertions(+), 35 deletions(-) diff --git a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java index 43d366184792..9585952909b0 100644 --- a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java +++ b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java @@ -37,13 +37,13 @@ /** * An OrderedMergeIterator is an iterator that merges together multiple sorted iterators. It is written assuming * that the input Iterators are provided in order. That is, it places an extra restriction in the input iterators. - * + *

* Normally a merge operation could operate with the actual input iterators in any order as long as the actual values * in the iterators are sorted. This requires that not only the individual values be sorted, but that the iterators * be provided in the order of the first element of each iterator. - * + *

* If this doesn't make sense, check out OrderedMergeSequenceTest.testScrewsUpOnOutOfOrderBeginningOfList() - * + *

* It places this extra restriction on the input data in order to implement an optimization that allows it to * remain as lazy as possible in the face of a common case where the iterators are just appended one after the other. */ @@ -156,8 +156,7 @@ public T accumulate(T accumulated, T in) throw Throwables.propagate(e); } return null; - } - else { + } else { yield(); } @@ -179,19 +178,16 @@ private Yielder makeYielder( Yielder yielder; if (oldDudeAtCrosswalk.isDone()) { yielder = pQueue.remove(); - } - else if (pQueue.isEmpty()) { + } else if (pQueue.isEmpty()) { yielder = oldDudeAtCrosswalk.get(); oldDudeAtCrosswalk = oldDudeAtCrosswalk.next(null); - } - else { + } else { Yielder queueYielder = pQueue.peek(); Yielder iterYielder = oldDudeAtCrosswalk.get(); if (ordering.compare(queueYielder.get(), iterYielder.get()) <= 0) { yielder = pQueue.remove(); - } - else { + } else { yielder = oldDudeAtCrosswalk.get(); oldDudeAtCrosswalk = oldDudeAtCrosswalk.next(null); } @@ -206,8 +202,7 @@ else if (pQueue.isEmpty()) { catch (IOException e) { throw Throwables.propagate(e); } - } - else { + } else { pQueue.add(yielder); } } @@ -242,7 +237,7 @@ public boolean isDone() @Override public void close() throws IOException { - while(!pQueue.isEmpty()) { + while (!pQueue.isEmpty()) { pQueue.remove().close(); } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/ExecutorExecutingSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/ExecutorExecutingSequence.java index da6f6e4b84d6..cd4d75e3b874 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/ExecutorExecutingSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/ExecutorExecutingSequence.java @@ -95,7 +95,10 @@ public OutType call() throws Exception } @Override - public Yielder toYielder(final OutType initValue, final YieldingAccumulator accumulator) + public Yielder toYielder( + final OutType initValue, + final YieldingAccumulator accumulator + ) { Future> future = exec.submit( new Callable>() @@ -120,7 +123,7 @@ public Yielder call() throws Exception @Override public Yielder toYielder( - final Supplier initValue, final YieldingAccumulator accumulator + final Supplier initValue, final YieldingAccumulator accumulator ) { Future> future = exec.submit( diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java index 477720d6e3d7..2ec1456b5095 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java @@ -25,11 +25,11 @@ * A Sequence represents an iterable sequence of elements. Unlike normal Iterators however, it doesn't expose * a way for you to extract values from it, instead you provide it with a worker (an Accumulator) and that defines * what happens with the data. - * + *

* This inversion of control is in place to allow the Sequence to do resource management. It can enforce that close() * methods get called and other resources get cleaned up whenever processing is complete. Without this inversion * it is very easy to unintentionally leak resources when iterating over something that is backed by a resource. - * + *

* Sequences also expose {#see com.metamx.common.guava.Yielder} Yielder objects which allow you to implement a * continuation over the Sequence. Yielder do not offer the same guarantees of automatic resource management * as the accumulate method, but they are Closeable and will do the proper cleanup when close() is called on them. @@ -39,10 +39,11 @@ public interface Sequence /** * Accumulate this sequence using the given accumulator. * - * @param initValue the initial value to pass along to start the accumulation. + * @param initValue the initial value to pass along to start the accumulation. * @param accumulator the accumulator which is responsible for accumulating input values. - * @param the type of accumulated value. - * @return + * @param the type of accumulated value. + * + * @return accumulated value. */ OutType accumulate(OutType initValue, Accumulator accumulator); @@ -51,20 +52,23 @@ public interface Sequence * The {@code initValSupplier} provides an way for lazy evaluation of the initial value. * * @param initValSupplier the supplier which returns an initial value to pass along to start the accumulation. - * @param accumulator the accumulator which is responsible for accumulating input values. - * @param the type of accumulated value. - * @return + * @param accumulator the accumulator which is responsible for accumulating input values. + * @param the type of accumulated value. + * + * @return accumulated value. */ OutType accumulate(Supplier initValSupplier, Accumulator accumulator); /** * Return an Yielder for accumulated sequence. * - * @param initValue the initial value to pass along to start the accumulation. + * @param initValue the initial value to pass along to start the accumulation. * @param accumulator the accumulator which is responsible for accumulating input values. - * @param the type of accumulated value. - * @see Yielder + * @param the type of accumulated value. + * * @return an Yielder for accumulated sequence. + * + * @see Yielder */ Yielder toYielder(OutType initValue, YieldingAccumulator accumulator); @@ -73,10 +77,12 @@ public interface Sequence * The {@code initValSupplier} provides an way for lazy evaluation of the initial value. * * @param initValSupplier the supplier which returns an initial value to pass along to start the accumulation. - * @param accumulator the accumulator which is responsible for accumulating input values. - * @param the type of accumulated value. - * @see Yielder + * @param accumulator the accumulator which is responsible for accumulating input values. + * @param the type of accumulated value. + * * @return an Yielder for accumulated sequence. + * + * @see Yielder */ Yielder toYielder(Supplier initValSupplier, YieldingAccumulator accumulator); } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java index caa470bbf439..8923efeef293 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java @@ -84,7 +84,7 @@ public static Sequence withBaggage(final Sequence seq, Closeable bagga return new ResourceClosingSequence<>(seq, baggage); } - public static Sequence withEffect(final Sequence seq, final Runnable effect, final Executor exec) + public static Sequence withEffect(final Sequence seq, final Runnable effect, final Executor exec) { return new Sequence() { diff --git a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java index 045e27857149..5337dd10e354 100644 --- a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java +++ b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java @@ -101,7 +101,10 @@ public OutType accumulate( } @Override - public Yielder toYielder(final OutType initValue, final YieldingAccumulator accumulator) + public Yielder toYielder( + final OutType initValue, + final YieldingAccumulator accumulator + ) { return toYielder(new Supplier>() { diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java index 35e31e268c96..2359e640acdd 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -114,9 +114,9 @@ private Sequence makeSequence() return new MergeSequence<>( query.getResultOrdering(), - Sequences.simple(listOfSequences)); - } - else { + Sequences.simple(listOfSequences) + ); + } else { return Iterables.getOnlyElement(listOfSequences); } } From 7af3b23401f91ca17eb8d02e8a6e1250062e1c6a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 24 Jan 2017 10:07:22 +0900 Subject: [PATCH 04/10] Merge master --- api/pom.xml | 4 + .../druid/benchmark/query/SqlBenchmark.java | 53 +- .../collections/OrderedMergeSequence.java | 20 +- .../druid/common/guava/CombiningSequence.java | 11 +- .../common/guava/CombiningSequenceTest.java | 5 +- distribution/pom.xml | 2 + docs/content/configuration/broker.md | 7 +- docs/content/configuration/coordinator.md | 1 + .../extensions-contrib/scan-query.md | 157 ++++ docs/content/development/extensions.md | 1 + docs/content/querying/sql.md | 116 ++- extensions-contrib/scan-query/pom.xml | 58 ++ .../java/io/druid/query/scan/ScanQuery.java | 390 ++++++++++ .../query/scan/ScanQueryDruidModule.java | 53 ++ .../io/druid/query/scan/ScanQueryEngine.java | 244 ++++++ .../query/scan/ScanQueryLimitRowIterator.java | 98 +++ .../query/scan/ScanQueryQueryToolChest.java | 114 +++ .../query/scan/ScanQueryRunnerFactory.java | 128 +++ .../io/druid/query/scan/ScanResultValue.java | 115 +++ .../io.druid.initialization.DruidModule | 1 + .../query/scan/MultiSegmentScanQueryTest.java | 242 ++++++ .../druid/query/scan/ScanQueryRunnerTest.java | 605 ++++++++++++++ .../druid/query/scan/ScanQuerySpecTest.java | 75 ++ .../sqlserver-metadata-storage/pom.xml | 2 +- extensions-core/histogram/pom.xml | 13 + .../ApproximateHistogramDruidModule.java | 7 + .../histogram/QuantilePostAggregator.java | 26 + .../histogram/sql/QuantileSqlAggregator.java | 163 ++++ .../sql/QuantileSqlAggregatorTest.java | 134 ++++ .../indexer/ITKafkaIndexingServiceTest.java | 40 +- .../java/util/common/guava/BaseSequence.java | 72 +- .../util/common/guava/ConcatSequence.java | 19 +- .../common/guava/ExecuteWhenDoneYielder.java | 25 +- .../guava/ExecutorExecutingSequence.java | 197 ----- .../util/common/guava/LimitedSequence.java | 10 +- .../java/util/common/guava/MergeSequence.java | 5 +- .../common/guava/ResourceClosingSequence.java | 96 --- .../util/common/guava/SequenceWrapper.java | 73 ++ .../java/util/common/guava/Sequences.java | 68 +- .../util/common/guava/WrappingSequence.java | 119 +++ .../util/common/guava/WrappingYielder.java | 103 +++ .../java/util/common/guava/Yielders.java | 31 +- .../common/guava/YieldingSequenceBase.java | 10 +- .../util/common/guava/BaseSequenceTest.java | 4 +- .../guava/ExecutorExecutingSequenceTest.java | 173 ---- .../java/util/common/guava/TestSequence.java | 30 +- .../common/guava/WithEffectSequenceTest.java | 109 +++ ...nceTest.java => WrappingSequenceTest.java} | 43 +- pom.xml | 33 + .../druid/query/CPUTimeMetricQueryRunner.java | 118 +-- .../druid/query/GroupByMergedQueryRunner.java | 3 +- .../query/MetricsEmittingQueryRunner.java | 180 +---- .../io/druid/query/QueryRunnerHelper.java | 3 +- .../ReferenceCountingSegmentQueryRunner.java | 4 +- .../epinephelinae/GroupByQueryEngineV2.java | 3 +- .../epinephelinae/GroupByRowProcessor.java | 21 +- .../groupby/strategy/GroupByStrategyV1.java | 5 +- .../druid/query/select/SelectQueryEngine.java | 52 +- .../spec/SpecificSegmentQueryRunner.java | 131 +--- .../segment/ReferenceCountingSequence.java | 61 -- .../firehose/IngestSegmentFirehose.java | 16 +- .../java/io/druid/server/QueryResource.java | 84 +- .../server/coordinator/DruidCoordinator.java | 1 + .../coordinator/DruidCoordinatorConfig.java | 6 + .../server/coordinator/LoadQueuePeon.java | 206 ++--- .../coordinator/ReplicationThrottler.java | 24 +- .../helper/DruidCoordinatorRuleRunner.java | 1 - .../server/coordinator/rules/LoadRule.java | 26 +- .../druid/client/CachingQueryRunnerTest.java | 36 +- .../DruidCoordinatorConfigTest.java | 3 + .../DruidCoordinatorRuleRunnerTest.java | 3 +- .../coordinator/DruidCoordinatorTest.java | 5 +- .../server/coordinator/LoadQueuePeonTest.java | 8 +- .../TestDruidCoordinatorConfig.java | 10 +- .../DruidCoordinatorSegmentKillerTest.java | 3 +- .../coordinator/rules/LoadRuleTest.java | 1 - .../main/java/io/druid/cli/DumpSegment.java | 2 - .../sql/avatica/AvaticaServerConfig.java | 48 +- .../sql/avatica/DruidAvaticaHandler.java | 11 +- .../io/druid/sql/avatica/DruidConnection.java | 42 +- .../java/io/druid/sql/avatica/DruidMeta.java | 592 ++++++++++++++ .../io/druid/sql/avatica/DruidStatement.java | 282 +++++++ .../ApproxCountDistinctSqlAggregator.java | 122 +++ .../calcite/aggregation/SqlAggregator.java | 61 ++ .../filtration/CombineAndSimplifyBounds.java | 42 +- .../druid/sql/calcite/planner/Calcites.java | 296 +++++-- .../calcite/planner/DruidOperatorTable.java | 90 +++ .../sql/calcite/planner/DruidPlannerImpl.java | 75 -- .../sql/calcite/planner/PlannerFactory.java | 81 ++ .../sql/calcite/planner/PlannerResult.java | 52 +- .../io/druid/sql/calcite/planner/Rules.java | 79 +- .../sql/calcite/rel/DruidConvention.java | 19 +- .../sql/calcite/rel/DruidNestedGroupBy.java | 22 +- .../sql/calcite/rel/DruidQueryBuilder.java | 33 +- .../druid/sql/calcite/rel/DruidQueryRel.java | 29 +- .../io/druid/sql/calcite/rel/DruidRel.java | 32 +- .../druid/sql/calcite/rel/DruidSemiJoin.java | 151 ++-- .../io/druid/sql/calcite/rel/QueryMaker.java | 319 ++++---- ...rRule.java => DruidRelToBindableRule.java} | 13 +- .../sql/calcite/rule/DruidRelToDruidRule.java | 53 +- .../sql/calcite/rule/DruidSemiJoinRule.java | 124 ++- .../druid/sql/calcite/rule/GroupByRules.java | 104 +-- .../sql/calcite/{ => schema}/DruidSchema.java | 48 +- .../sql/calcite/schema/InformationSchema.java | 303 +++++++ .../druid/sql/calcite/table/DruidTable.java | 7 +- .../druid/sql/calcite/table/RowSignature.java | 25 +- .../java/io/druid/sql/guice/SqlBindings.java | 19 +- .../java/io/druid/sql/guice/SqlModule.java | 24 +- .../java/io/druid/sql/http/SqlResource.java | 149 ++-- .../sql/avatica/DruidAvaticaHandlerTest.java | 159 +++- .../druid/sql/avatica/DruidStatementTest.java | 180 +++++ .../druid/sql/calcite/CalciteQueryTest.java | 736 +++++++++++++----- .../sql/calcite/http/SqlResourceTest.java | 41 +- .../sql/calcite/planner/CalcitesTest.java | 40 + .../calcite/{ => schema}/DruidSchemaTest.java | 35 +- .../druid/sql/calcite/util/CalciteTests.java | 314 ++++---- .../druid/sql/calcite/util/QueryLogHook.java | 106 +++ .../SpecificSegmentsQuerySegmentWalker.java | 1 + 118 files changed, 7504 insertions(+), 2576 deletions(-) create mode 100644 docs/content/development/extensions-contrib/scan-query.md create mode 100644 extensions-contrib/scan-query/pom.xml create mode 100644 extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQuery.java create mode 100644 extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryDruidModule.java create mode 100644 extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryEngine.java create mode 100644 extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryLimitRowIterator.java create mode 100644 extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java create mode 100644 extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryRunnerFactory.java create mode 100644 extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanResultValue.java create mode 100644 extensions-contrib/scan-query/src/main/resources/META-INF/services/io.druid.initialization.DruidModule create mode 100644 extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java create mode 100644 extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java create mode 100644 extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQuerySpecTest.java create mode 100644 extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java create mode 100644 extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java delete mode 100644 java-util/src/main/java/io/druid/java/util/common/guava/ExecutorExecutingSequence.java delete mode 100644 java-util/src/main/java/io/druid/java/util/common/guava/ResourceClosingSequence.java create mode 100644 java-util/src/main/java/io/druid/java/util/common/guava/SequenceWrapper.java create mode 100644 java-util/src/main/java/io/druid/java/util/common/guava/WrappingSequence.java create mode 100644 java-util/src/main/java/io/druid/java/util/common/guava/WrappingYielder.java delete mode 100644 java-util/src/test/java/io/druid/java/util/common/guava/ExecutorExecutingSequenceTest.java create mode 100644 java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java rename java-util/src/test/java/io/druid/java/util/common/guava/{ResourceClosingSequenceTest.java => WrappingSequenceTest.java} (55%) delete mode 100644 processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java rename java-util/src/main/java/io/druid/java/util/common/guava/SimpleSequence.java => sql/src/main/java/io/druid/sql/avatica/AvaticaServerConfig.java (57%) rename java-util/src/main/java/io/druid/java/util/common/guava/Fns.java => sql/src/main/java/io/druid/sql/avatica/DruidConnection.java (57%) create mode 100644 sql/src/main/java/io/druid/sql/avatica/DruidMeta.java create mode 100644 sql/src/main/java/io/druid/sql/avatica/DruidStatement.java create mode 100644 sql/src/main/java/io/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java create mode 100644 sql/src/main/java/io/druid/sql/calcite/aggregation/SqlAggregator.java create mode 100644 sql/src/main/java/io/druid/sql/calcite/planner/DruidOperatorTable.java delete mode 100644 sql/src/main/java/io/druid/sql/calcite/planner/DruidPlannerImpl.java create mode 100644 sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java rename java-util/src/main/java/io/druid/java/util/common/guava/ResourceClosingYielder.java => sql/src/main/java/io/druid/sql/calcite/planner/PlannerResult.java (50%) rename sql/src/main/java/io/druid/sql/calcite/rule/{DruidBindableConverterRule.java => DruidRelToBindableRule.java} (77%) rename java-util/src/main/java/io/druid/java/util/common/guava/IteratorWithBaggage.java => sql/src/main/java/io/druid/sql/calcite/rule/DruidRelToDruidRule.java (53%) rename sql/src/main/java/io/druid/sql/calcite/{ => schema}/DruidSchema.java (91%) create mode 100644 sql/src/main/java/io/druid/sql/calcite/schema/InformationSchema.java rename java-util/src/main/java/io/druid/java/util/common/guava/YieldSign.java => sql/src/main/java/io/druid/sql/guice/SqlBindings.java (62%) create mode 100644 sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java create mode 100644 sql/src/test/java/io/druid/sql/calcite/planner/CalcitesTest.java rename sql/src/test/java/io/druid/sql/calcite/{ => schema}/DruidSchemaTest.java (82%) create mode 100644 sql/src/test/java/io/druid/sql/calcite/util/QueryLogHook.java diff --git a/api/pom.xml b/api/pom.xml index 8dbd653c9fae..9029a9317a0b 100644 --- a/api/pom.xml +++ b/api/pom.xml @@ -83,6 +83,10 @@ commons-io commons-io + + com.google.code.findbugs + jsr305 + diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java index e9ea84fa8bd0..4637abc7fbbe 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java @@ -34,6 +34,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; +import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.TableDataSource; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -41,18 +42,25 @@ import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.GroupByQuery; +import io.druid.segment.IndexBuilder; +import io.druid.segment.QueryableIndex; +import io.druid.segment.TestHelper; import io.druid.segment.column.ValueType; import io.druid.segment.serde.ComplexMetrics; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerConfig; +import io.druid.sql.calcite.planner.PlannerFactory; +import io.druid.sql.calcite.planner.PlannerResult; import io.druid.sql.calcite.rel.QueryMaker; import io.druid.sql.calcite.table.DruidTable; import io.druid.sql.calcite.util.CalciteTests; import io.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; -import org.apache.calcite.jdbc.CalciteConnection; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.LinearShardSpec; import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.Table; import org.apache.calcite.schema.impl.AbstractSchema; +import org.apache.calcite.tools.Planner; import org.apache.commons.io.FileUtils; import org.joda.time.Interval; import org.openjdk.jmh.annotations.Benchmark; @@ -71,8 +79,6 @@ import org.openjdk.jmh.infra.Blackhole; import java.io.File; -import java.sql.ResultSet; -import java.sql.ResultSetMetaData; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -96,7 +102,7 @@ public class SqlBenchmark private File tmpDir; private SpecificSegmentsQuerySegmentWalker walker; - private CalciteConnection calciteConnection; + private PlannerFactory plannerFactory; private GroupByQuery groupByQuery; private String sqlQuery; @@ -130,7 +136,23 @@ public void setup() throws Exception log.info("%,d/%,d rows generated.", rows.size(), rowsPerSegment); final PlannerConfig plannerConfig = new PlannerConfig(); - walker = CalciteTests.createWalker(tmpDir, rows); + final QueryRunnerFactoryConglomerate conglomerate = CalciteTests.queryRunnerFactoryConglomerate(); + final QueryableIndex index = IndexBuilder.create() + .tmpDir(new File(tmpDir, "1")) + .indexMerger(TestHelper.getTestIndexMergerV9()) + .rows(rows) + .buildMMappedIndex(); + + this.walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add( + DataSegment.builder() + .dataSource("foo") + .interval(index.getDataInterval()) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .build(), + index + ); + final Map tableMap = ImmutableMap.of( "foo", new DruidTable( @@ -152,7 +174,11 @@ protected Map getTableMap() return tableMap; } }; - calciteConnection = Calcites.jdbc(druidSchema, plannerConfig); + plannerFactory = new PlannerFactory( + Calcites.createRootSchema(druidSchema), + CalciteTests.createOperatorTable(), + plannerConfig + ); groupByQuery = GroupByQuery .builder() .setDataSource("foo") @@ -160,7 +186,7 @@ protected Map getTableMap() .setDimensions( Arrays.asList( new DefaultDimensionSpec("dimZipf", "d0"), - new DefaultDimensionSpec("dimSequential", "d1") + new DefaultDimensionSpec("dimSequential", "d1") ) ) .setAggregatorSpecs(Arrays.asList(new CountAggregatorFactory("c"))) @@ -204,15 +230,12 @@ public void queryNative(Blackhole blackhole) throws Exception @Benchmark @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void querySql(Blackhole blackhole) throws Exception + public void queryPlanner(Blackhole blackhole) throws Exception { - final ResultSet resultSet = calciteConnection.createStatement().executeQuery(sqlQuery); - final ResultSetMetaData metaData = resultSet.getMetaData(); - - while (resultSet.next()) { - for (int i = 0; i < metaData.getColumnCount(); i++) { - blackhole.consume(resultSet.getObject(i + 1)); - } + try (final Planner planner = plannerFactory.createPlanner()) { + final PlannerResult plannerResult = Calcites.plan(planner, sqlQuery); + final ArrayList results = Sequences.toList(plannerResult.run(), Lists.newArrayList()); + blackhole.consume(results); } } } diff --git a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java index 9585952909b0..9c88be318ed0 100644 --- a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java +++ b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java @@ -21,8 +21,10 @@ import com.google.common.base.Function; import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import com.google.common.collect.Ordering; +import com.google.common.io.Closer; import io.druid.java.util.common.guava.Accumulator; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.guava.Sequence; @@ -64,14 +66,7 @@ public OrderedMergeSequence( @Override public OutType accumulate(OutType initValue, Accumulator accumulator) { - Yielder yielder = null; - try { - yielder = toYielder(initValue, YieldingAccumulators.fromAccumulator(accumulator)); - return yielder.get(); - } - finally { - CloseQuietly.close(yielder); - } + return accumulate(Suppliers.ofInstance(initValue), accumulator); } @Override @@ -92,10 +87,7 @@ public OutType accumulate( @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { - final PriorityQueue> pQueue = makePriorityQueue(); - final Yielder> oldDudeAtCrosswalk = makeOldDudeAtCrosswalk(); - - return makeYielder(pQueue, oldDudeAtCrosswalk, initValue, accumulator); + return toYielder(Suppliers.ofInstance(initValue), accumulator); } @Override @@ -237,9 +229,11 @@ public boolean isDone() @Override public void close() throws IOException { + Closer closer = Closer.create(); while (!pQueue.isEmpty()) { - pQueue.remove().close(); + closer.register(pQueue.remove()); } + closer.close(); } }; } diff --git a/common/src/main/java/io/druid/common/guava/CombiningSequence.java b/common/src/main/java/io/druid/common/guava/CombiningSequence.java index 757594de14d9..4bf95d8b8932 100644 --- a/common/src/main/java/io/druid/common/guava/CombiningSequence.java +++ b/common/src/main/java/io/druid/common/guava/CombiningSequence.java @@ -62,16 +62,7 @@ public CombiningSequence( @Override public OutType accumulate(OutType initValue, final Accumulator accumulator) { - final CombiningAccumulator combiningAccumulator = new CombiningAccumulator<>( - Suppliers.ofInstance(initValue), - accumulator - ); - T lastValue = baseSequence.accumulate((T) null, combiningAccumulator); - if (combiningAccumulator.accumulatedSomething()) { - return accumulator.accumulate(combiningAccumulator.retVal, lastValue); - } else { - return initValue; - } + return accumulate(Suppliers.ofInstance(initValue), accumulator); } @Override diff --git a/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java b/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java index 9793449f7ac1..bf46b38f2868 100644 --- a/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java +++ b/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java @@ -24,15 +24,12 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; - import io.druid.java.util.common.Pair; -import io.druid.java.util.common.guava.ResourceClosingSequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Yielder; import io.druid.java.util.common.guava.YieldingAccumulator; import io.druid.java.util.common.guava.nary.BinaryFn; - import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -232,7 +229,7 @@ public void close() throws IOException Sequence> seq = Sequences.limit( new CombiningSequence<>( - new ResourceClosingSequence<>(Sequences.simple(pairs), closeable), + Sequences.withBaggage(Sequences.simple(pairs), closeable), Ordering.natural().onResultOf(Pair.lhsFn()), new BinaryFn, Pair, Pair>() { diff --git a/distribution/pom.xml b/distribution/pom.xml index 8908e0332a4d..044728d62911 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -104,6 +104,8 @@ io.druid.extensions:mysql-metadata-storage -c io.druid.extensions:postgresql-metadata-storage + -c + io.druid.extensions.contrib:scan-query ${druid.distribution.pulldeps.opts} diff --git a/docs/content/configuration/broker.md b/docs/content/configuration/broker.md index a70083473622..4da79a9912ee 100644 --- a/docs/content/configuration/broker.md +++ b/docs/content/configuration/broker.md @@ -92,8 +92,11 @@ The broker's [built-in SQL server](../querying/sql.html) can be configured throu |Property|Description|Default| |--------|-----------|-------| |`druid.sql.enable`|Whether to enable SQL at all, including background metadata fetching. If false, this overrides all other SQL-related properties and disables SQL metadata, serving, and planning completely.|false| -|`druid.sql.server.enableAvatica`|Whether to enable an Avatica server at `/druid/v2/sql/avatica/`.|true| -|`druid.sql.server.enableJsonOverHttp`|Whether to enable a simple JSON over HTTP route at `/druid/v2/sql/`.|true| +|`druid.sql.avatica.enable`|Whether to enable an Avatica server at `/druid/v2/sql/avatica/`.|true| +|`druid.sql.avatica.connectionIdleTimeout`|Avatica client connection idle timeout.|PT30M| +|`druid.sql.avatica.maxConnections`|Maximum number of open connections for the Avatica server. These are not HTTP connections, but are logical client connections that may span multiple HTTP connections.|25| +|`druid.sql.avatica.maxStatementsPerConnection`|Maximum number of simultaneous open statements per Avatica client connection.|4| +|`druid.sql.http.enable`|Whether to enable a simple JSON over HTTP route at `/druid/v2/sql/`.|true| #### SQL Planner Configuration diff --git a/docs/content/configuration/coordinator.md b/docs/content/configuration/coordinator.md index 5ed2c2c70867..b4e4f0c8f79e 100644 --- a/docs/content/configuration/coordinator.md +++ b/docs/content/configuration/coordinator.md @@ -33,6 +33,7 @@ The coordinator node uses several of the global configs in [Configuration](../co |`druid.coordinator.kill.durationToRetain`| Do not kill segments in last `durationToRetain`, must be greater or equal to 0. Only applies and MUST be specified if kill is turned on. Note that default value is invalid.|PT-1S (-1 seconds)| |`druid.coordinator.kill.maxSegments`|Kill at most n segments per kill task submission, must be greater than 0. Only applies and MUST be specified if kill is turned on. Note that default value is invalid.|0| |`druid.coordinator.balancer.strategy`|Specify the type of balancing strategy that the coordinator should use to distribute segments among the historicals. Use `diskNormalized` to distribute segments among nodes so that the disks fill up uniformly and use `random` to randomly pick nodes to distribute segments.|`cost`| +|`druid.coordinator.loadqueuepeon.repeatDelay`|The start and repeat delay for the loadqueuepeon , which manages the load and drop of segments.|PT0.050S (50 ms)| ### Metadata Retrieval diff --git a/docs/content/development/extensions-contrib/scan-query.md b/docs/content/development/extensions-contrib/scan-query.md new file mode 100644 index 000000000000..3eef162b254e --- /dev/null +++ b/docs/content/development/extensions-contrib/scan-query.md @@ -0,0 +1,157 @@ +--- +layout: doc_page +--- + +# Scan query +Scan query returns raw Druid rows in streaming mode. + +```json + { + "queryType": "scan", + "dataSource": "wikipedia", + "resultFormat": "list", + "columns":[], + "intervals": [ + "2013-01-01/2013-01-02" + ], + "batchSize":20480, + "limit":5 + } +``` + +There are several main parts to a scan query: + +|property|description|required?| +|--------|-----------|---------| +|queryType|This String should always be "scan"; this is the first thing Druid looks at to figure out how to interpret the query|yes| +|dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../querying/datasource.html) for more information.|yes| +|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| +|resultFormat|How result represented, list or compactedList or valueVector. Currently only `list` and `compactedList` are supported. Default is `list`|no| +|filter|See [Filters](../querying/filters.html)|no| +|columns|A String array of dimensions and metrics to scan. If left empty, all dimensions and metrics are returned.|no| +|batchSize|How many rows buffered before return to client. Default is `20480`|no| +|limit|How many rows to return. If not specified, all rows will be returned.|no| +|context|An additional JSON Object which can be used to specify certain flags.|no| + +The format of the result when resultFormat equals to `list`: + +```json + [{ + "segmentId" : "wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9", + "columns" : [ + "timestamp", + "robot", + "namespace", + "anonymous", + "unpatrolled", + "page", + "language", + "newpage", + "user", + "count", + "added", + "delta", + "variation", + "deleted" + ], + "events" : [ { + "timestamp" : "2013-01-01T00:00:00.000Z", + "robot" : "1", + "namespace" : "article", + "anonymous" : "0", + "unpatrolled" : "0", + "page" : "11._korpus_(NOVJ)", + "language" : "sl", + "newpage" : "0", + "user" : "EmausBot", + "count" : 1.0, + "added" : 39.0, + "delta" : 39.0, + "variation" : 39.0, + "deleted" : 0.0 + }, { + "timestamp" : "2013-01-01T00:00:00.000Z", + "robot" : "0", + "namespace" : "article", + "anonymous" : "0", + "unpatrolled" : "0", + "page" : "112_U.S._580", + "language" : "en", + "newpage" : "1", + "user" : "MZMcBride", + "count" : 1.0, + "added" : 70.0, + "delta" : 70.0, + "variation" : 70.0, + "deleted" : 0.0 + }, { + "timestamp" : "2013-01-01T00:00:00.000Z", + "robot" : "0", + "namespace" : "article", + "anonymous" : "0", + "unpatrolled" : "0", + "page" : "113_U.S._243", + "language" : "en", + "newpage" : "1", + "user" : "MZMcBride", + "count" : 1.0, + "added" : 77.0, + "delta" : 77.0, + "variation" : 77.0, + "deleted" : 0.0 + }, { + "timestamp" : "2013-01-01T00:00:00.000Z", + "robot" : "0", + "namespace" : "article", + "anonymous" : "0", + "unpatrolled" : "0", + "page" : "113_U.S._73", + "language" : "en", + "newpage" : "1", + "user" : "MZMcBride", + "count" : 1.0, + "added" : 70.0, + "delta" : 70.0, + "variation" : 70.0, + "deleted" : 0.0 + }, { + "timestamp" : "2013-01-01T00:00:00.000Z", + "robot" : "0", + "namespace" : "article", + "anonymous" : "0", + "unpatrolled" : "0", + "page" : "113_U.S._756", + "language" : "en", + "newpage" : "1", + "user" : "MZMcBride", + "count" : 1.0, + "added" : 68.0, + "delta" : 68.0, + "variation" : 68.0, + "deleted" : 0.0 + } ] +} ] +``` + +The format of the result when resultFormat equals to `compactedList`: + +```json + [{ + "segmentId" : "wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9", + "columns" : [ + "timestamp", "robot", "namespace", "anonymous", "unpatrolled", "page", "language", "newpage", "user", "count", "added", "delta", "variation", "deleted" + ], + "events" : [ + ["2013-01-01T00:00:00.000Z", "1", "article", "0", "0", "11._korpus_(NOVJ)", "sl", "0", "EmausBot", 1.0, 39.0, 39.0, 39.0, 0.0], + ["2013-01-01T00:00:00.000Z", "0", "article", "0", "0", "112_U.S._580", "en", "1", "MZMcBride", 1.0, 70.0, 70.0, 70.0, 0.0], + ["2013-01-01T00:00:00.000Z", "0", "article", "0", "0", "113_U.S._243", "en", "1", "MZMcBride", 1.0, 77.0, 77.0, 77.0, 0.0], + ["2013-01-01T00:00:00.000Z", "0", "article", "0", "0", "113_U.S._73", "en", "1", "MZMcBride", 1.0, 70.0, 70.0, 70.0, 0.0], + ["2013-01-01T00:00:00.000Z", "0", "article", "0", "0", "113_U.S._756", "en", "1", "MZMcBride", 1.0, 68.0, 68.0, 68.0, 0.0] + ] +} ] +``` + +The biggest difference between select query and scan query is that, scan query doesn't retain all rows in memory before rows can be returned to client. +It will cause memory pressure if too many rows required by select query. +Scan query doesn't have this issue. +Scan query can return all rows without issuing another pagination query, which is extremely useful when query against historical or realtime node directly. \ No newline at end of file diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md index 5813ccd78dcb..6a2e996ffd44 100644 --- a/docs/content/development/extensions.md +++ b/docs/content/development/extensions.md @@ -65,6 +65,7 @@ All of these community extensions can be downloaded using *pull-deps* with the c |graphite-emitter|Graphite metrics emitter|[link](../development/extensions-contrib/graphite.html)| |statsd-emitter|StatsD metrics emitter|[link](../development/extensions-contrib/statsd.html)| |druid-thrift-extensions|Support thrift ingestion |[link](../development/extensions-contrib/thrift.html)| +|scan-query|Scan query|[link](../development/extensions-contrib/scan-query.html)| ## Promoting Community Extension to Core Extension diff --git a/docs/content/querying/sql.md b/docs/content/querying/sql.md index 1e56b58fe856..b9b94ecf3321 100644 --- a/docs/content/querying/sql.md +++ b/docs/content/querying/sql.md @@ -12,7 +12,10 @@ subject to change. Druid includes a native SQL layer with an [Apache Calcite](https://calcite.apache.org/)-based parser and planner. All parsing and planning takes place on the Broker, where SQL is converted to native Druid queries. Those native Druid -queries are then passed down to data nodes. Each Druid dataSource appears as a table in the "druid" schema. +queries are then passed down to data nodes. Each Druid datasource appears as a table in the "druid" schema. Datasource +and column names are both case-sensitive and can optionally be quoted using double quotes. Literal strings should be +quoted with single quotes, like `'foo'`. Literal strings with Unicode escapes can be written like `U&'fo\00F6'`, where +character codes in hex are prefixed by a backslash. Add "EXPLAIN PLAN FOR" to the beginning of any query to see how Druid will plan that query. @@ -29,7 +32,7 @@ Example code: ```java Connection connection = DriverManager.getConnection("jdbc:avatica:remote:url=http://localhost:8082/druid/v2/sql/avatica/"); -ResultSet resultSet = connection.createStatement().executeQuery("SELECT COUNT(*) AS cnt FROM druid.foo"); +ResultSet resultSet = connection.createStatement().executeQuery("SELECT COUNT(*) AS cnt FROM data_source"); while (resultSet.next()) { // Do something } @@ -46,17 +49,17 @@ is: ```json { - "query" : "SELECT COUNT(*) FROM druid.ds WHERE foo = ?" + "query" : "SELECT COUNT(*) FROM data_source WHERE foo = 'bar'" } ``` You can use _curl_ to send these queries from the command-line: ```bash -curl -XPOST -H'Content-Type: application/json' http://BROKER:8082/druid/v2/sql/ -d '{"query":"SELECT COUNT(*) FROM druid.ds"}' +curl -XPOST -H'Content-Type: application/json' http://BROKER:8082/druid/v2/sql/ -d '{"query":"SELECT COUNT(*) FROM data_source"}' ``` -Metadata is not available over the HTTP API. +Metadata is only available over the HTTP API by querying the "INFORMATION_SCHEMA" tables (see below). ### Metadata @@ -65,28 +68,28 @@ on broker startup and also periodically in the background through [SegmentMetadata queries](../querying/segmentmetadataquery.html). Background metadata refreshing is triggered by segments entering and exiting the cluster, and can also be throttled through configuration. -This cached metadata is queryable through the "metadata.COLUMNS" and "metadata.TABLES" tables. When -`druid.sql.planner.useFallback` is disabled (the default), only full scans of this table are possible. For example, to -retrieve column metadata, use the query: +This cached metadata is queryable through "INFORMATION_SCHEMA" tables. For example, to retrieve metadata for the Druid +datasource "foo", use the query: ```sql -SELECT * FROM metadata.COLUMNS +SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE SCHEMA_NAME = 'druid' AND TABLE_NAME = 'foo' ``` -If `druid.sql.planner.useFallback` is enabled, full SQL is possible on metadata tables. However, useFallback is not -recommended in production since it can generate unscalable query plans. The JDBC driver allows accessing -table and column metadata through `connection.getMetaData()` even if useFallback is off. +See the [INFORMATION_SCHEMA tables](#information_schema-tables) section below for details on the available metadata. + +You can also access table and column metadata through JDBC using `connection.getMetaData()`. ### Approximate queries The following SQL queries and features may be executed using approximate algorithms: -- `COUNT(DISTINCT col)` aggregations use [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf), a -fast approximate distinct counting algorithm. If you need exact distinct counts, you can instead use -`SELECT COUNT(*) FROM (SELECT DISTINCT col FROM druid.foo)`, which will use a slower and more resource intensive exact +- `COUNT(DISTINCT col)` and `APPROX_COUNT_DISTINCT(col)` aggregations use +[HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf), a fast approximate distinct counting +algorithm. If you need exact distinct counts, you can instead use +`SELECT COUNT(*) FROM (SELECT DISTINCT col FROM data_source)`, which will use a slower and more resource intensive exact algorithm. - TopN-style queries with a single grouping column, like -`SELECT col1, SUM(col2) FROM druid.foo GROUP BY col1 ORDER BY SUM(col2) DESC LIMIT 100`, by default will be executed +`SELECT col1, SUM(col2) FROM data_source GROUP BY col1 ORDER BY SUM(col2) DESC LIMIT 100`, by default will be executed as [TopN queries](topnquery.html), which use an approximate algorithm. To disable this behavior, and use exact algorithms for topN-style queries, set [druid.sql.planner.useApproximateTopN](../configuration/broker.html#sql-planner-configuration) to "false". @@ -95,9 +98,9 @@ algorithms for topN-style queries, set Druid's SQL language supports a number of time operations, including: -- `FLOOR(__time TO )` for grouping or filtering on time buckets, like `SELECT FLOOR(__time TO MONTH), SUM(cnt) FROM druid.foo GROUP BY FLOOR(__time TO MONTH)` -- `EXTRACT( FROM __time)` for grouping or filtering on time parts, like `SELECT EXTRACT(HOUR FROM __time), SUM(cnt) FROM druid.foo GROUP BY EXTRACT(HOUR FROM __time)` -- Comparisons to `TIMESTAMP '

+ * Required: dataSource(), intervals() must be called before build() + *

+ * Usage example: + *


+   *   ScanQuery query = new ScanQueryBuilder()
+   *                                  .dataSource("Example")
+   *                                  .interval("2010/2013")
+   *                                  .build();
+   * 
+ * + * @see io.druid.query.scan.ScanQuery + */ + public static class ScanQueryBuilder + { + private DataSource dataSource; + private QuerySegmentSpec querySegmentSpec; + private Map context; + private String resultFormat; + private int batchSize; + private int limit; + private DimFilter dimFilter; + private List columns; + + public ScanQueryBuilder() + { + dataSource = null; + querySegmentSpec = null; + context = null; + resultFormat = null; + batchSize = 0; + limit = 0; + dimFilter = null; + columns = Lists.newArrayList(); + } + + public ScanQuery build() + { + return new ScanQuery( + dataSource, + querySegmentSpec, + resultFormat, + batchSize, + limit, + dimFilter, + columns, + context + ); + } + + public ScanQueryBuilder copy(ScanQueryBuilder builder) + { + return new ScanQueryBuilder() + .dataSource(builder.dataSource) + .intervals(builder.querySegmentSpec) + .context(builder.context); + } + + public ScanQueryBuilder dataSource(String ds) + { + dataSource = new TableDataSource(ds); + return this; + } + + public ScanQueryBuilder dataSource(DataSource ds) + { + dataSource = ds; + return this; + } + + public ScanQueryBuilder intervals(QuerySegmentSpec q) + { + querySegmentSpec = q; + return this; + } + + public ScanQueryBuilder intervals(String s) + { + querySegmentSpec = new LegacySegmentSpec(s); + return this; + } + + public ScanQueryBuilder intervals(List l) + { + querySegmentSpec = new LegacySegmentSpec(l); + return this; + } + + public ScanQueryBuilder context(Map c) + { + context = c; + return this; + } + + public ScanQueryBuilder resultFormat(String r) + { + resultFormat = r; + return this; + } + + public ScanQueryBuilder batchSize(int b) + { + batchSize = b; + return this; + } + + public ScanQueryBuilder limit(int l) + { + limit = l; + return this; + } + + public ScanQueryBuilder filters(String dimensionName, String value) + { + dimFilter = new SelectorDimFilter(dimensionName, value, null); + return this; + } + + public ScanQueryBuilder filters(String dimensionName, String value, String... values) + { + dimFilter = new InDimFilter(dimensionName, Lists.asList(value, values), null); + return this; + } + + public ScanQueryBuilder filters(DimFilter f) + { + dimFilter = f; + return this; + } + + public ScanQueryBuilder columns(List c) + { + columns = c; + return this; + } + + public ScanQueryBuilder columns(String... c) + { + columns = Arrays.asList(c); + return this; + } + } + + public static ScanQueryBuilder newScanQueryBuilder() + { + return new ScanQueryBuilder(); + } +} diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryDruidModule.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryDruidModule.java new file mode 100644 index 000000000000..f33ea997b268 --- /dev/null +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryDruidModule.java @@ -0,0 +1,53 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.druid.query.scan; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.inject.Binder; +import io.druid.guice.DruidBinders; +import io.druid.guice.LazySingleton; +import io.druid.initialization.DruidModule; + +import java.util.Arrays; +import java.util.List; + +public class ScanQueryDruidModule implements DruidModule { + public void configure(Binder binder) { + DruidBinders.queryToolChestBinder(binder) + .addBinding(ScanQuery.class) + .to(ScanQueryQueryToolChest.class) + .in(LazySingleton.class); + + DruidBinders.queryRunnerFactoryBinder(binder) + .addBinding(ScanQuery.class) + .to(ScanQueryRunnerFactory.class) + .in(LazySingleton.class); + } + + public List getJacksonModules() { + return Arrays.asList( + new SimpleModule("ScanQueryDruidModule") + .registerSubtypes( + new NamedType(ScanQuery.class, ScanQuery.SCAN) + ) + ); + } +} diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryEngine.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryEngine.java new file mode 100644 index 000000000000..5b04bb19a8d6 --- /dev/null +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryEngine.java @@ -0,0 +1,244 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.druid.query.scan; + +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import io.druid.granularity.QueryGranularities; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.guava.BaseSequence; +import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Sequences; +import io.druid.query.ColumnSelectorPlus; +import io.druid.query.QueryInterruptedException; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.filter.Filter; +import io.druid.query.select.SelectQueryEngine; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionHandlerUtils; +import io.druid.segment.LongColumnSelector; +import io.druid.segment.ObjectColumnSelector; +import io.druid.segment.Segment; +import io.druid.segment.StorageAdapter; +import io.druid.segment.VirtualColumns; +import io.druid.segment.column.Column; +import io.druid.segment.filter.Filters; +import org.joda.time.Interval; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeoutException; + +public class ScanQueryEngine +{ + private static final SelectQueryEngine.SelectStrategyFactory STRATEGY_FACTORY = new SelectQueryEngine.SelectStrategyFactory(); + public Sequence process( + final ScanQuery query, + final Segment segment, + final Map responseContext + ) + { + if (responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) != null) { + int count = (int) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT); + if (count >= query.getLimit()) { + return Sequences.empty(); + } + } + final Long timeoutAt = (long) responseContext.get(ScanQueryRunnerFactory.CTX_TIMEOUT_AT); + final long start = System.currentTimeMillis(); + final StorageAdapter adapter = segment.asStorageAdapter(); + + if (adapter == null) { + throw new ISE( + "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + ); + } + + List allDims = Lists.newLinkedList(adapter.getAvailableDimensions()); + List allMetrics = Lists.newLinkedList(adapter.getAvailableMetrics()); + final List allColumns = Lists.newLinkedList(); + if (query.getColumns() != null && !query.getColumns().isEmpty()) { + if (!query.getColumns().contains(ScanResultValue.timestampKey)) { + allColumns.add(ScanResultValue.timestampKey); + } + allColumns.addAll(query.getColumns()); + allDims.retainAll(query.getColumns()); + allMetrics.retainAll(query.getColumns()); + } + else { + if (!allDims.contains(ScanResultValue.timestampKey)) { + allColumns.add(ScanResultValue.timestampKey); + } + allColumns.addAll(allDims); + allColumns.addAll(allMetrics); + } + final List dims = DefaultDimensionSpec.toSpec(allDims); + final List metrics = allMetrics; + + final List intervals = query.getQuerySegmentSpec().getIntervals(); + Preconditions.checkArgument(intervals.size() == 1, "Can only handle a single interval, got[%s]", intervals); + + final String segmentId = segment.getIdentifier(); + + final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimensionsFilter())); + + if (responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) == null) { + responseContext.put(ScanQueryRunnerFactory.CTX_COUNT, 0); + } + final int limit = query.getLimit() - (int) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT); + return Sequences.concat( + Sequences.map( + adapter.makeCursors( + filter, + intervals.get(0), + VirtualColumns.EMPTY, + QueryGranularities.ALL, + query.isDescending() + ), + new Function>() + { + @Override + public Sequence apply(final Cursor cursor) + { + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + final LongColumnSelector timestampColumnSelector = cursor.makeLongColumnSelector(Column.TIME_COLUMN_NAME); + + final List> selectorPlusList = Arrays.asList( + DimensionHandlerUtils.createColumnSelectorPluses( + STRATEGY_FACTORY, + Lists.newArrayList(dims), + cursor + ) + ); + + final Map metSelectors = Maps.newHashMap(); + for (String metric : metrics) { + final ObjectColumnSelector metricSelector = cursor.makeObjectColumnSelector(metric); + metSelectors.put(metric, metricSelector); + } + final int batchSize = query.getBatchSize(); + return new Iterator() + { + private int offset = 0; + + @Override + public boolean hasNext() + { + return !cursor.isDone() && offset < limit; + } + + @Override + public ScanResultValue next() + { + if (System.currentTimeMillis() >= timeoutAt) { + throw new QueryInterruptedException(new TimeoutException()); + } + int lastOffset = offset; + Object events = null; + String resultFormat = query.getResultFormat(); + if (ScanQuery.RESULT_FORMAT_VALUE_VECTOR.equals(resultFormat)) { + throw new UnsupportedOperationException("valueVector is not supported now"); + } else if (ScanQuery.RESULT_FORMAT_COMPACTED_LIST.equals(resultFormat)) { + events = rowsToCompactedList(); + } else { + events = rowsToList(); + } + responseContext.put( + ScanQueryRunnerFactory.CTX_COUNT, + (int) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) + (offset - lastOffset) + ); + responseContext.put( + ScanQueryRunnerFactory.CTX_TIMEOUT_AT, + timeoutAt - (System.currentTimeMillis() - start) + ); + return new ScanResultValue(segmentId, allColumns, events); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + + private Object rowsToCompactedList() + { + return Lists.transform( + (List>) rowsToList(), + new Function, Object>() + { + @Override + public Object apply(Map input) + { + List eventValues = Lists.newArrayListWithExpectedSize(allColumns.size()); + for (String expectedColumn : allColumns) { + eventValues.add(input.get(expectedColumn)); + } + return eventValues; + } + } + ); + } + + private Object rowsToList() + { + List> events = Lists.newArrayListWithCapacity(batchSize); + for (int i = 0; !cursor.isDone() + && i < batchSize + && offset < limit; cursor.advance(), i++, offset++) { + final Map theEvent = SelectQueryEngine.singleEvent( + ScanResultValue.timestampKey, + timestampColumnSelector, + selectorPlusList, + metSelectors + ); + events.add(theEvent); + } + return events; + } + + private Object rowsToValueVector() + { + // only support list now, we can support ValueVector or Arrow in future + return rowsToList(); + } + }; + } + + @Override + public void cleanup(Iterator iterFromMake) + { + } + } + ); + } + } + ) + ); + } +} diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryLimitRowIterator.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryLimitRowIterator.java new file mode 100644 index 000000000000..caa406a14c09 --- /dev/null +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryLimitRowIterator.java @@ -0,0 +1,98 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.druid.query.scan; + +import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Yielder; +import io.druid.java.util.common.guava.YieldingAccumulator; +import io.druid.java.util.common.parsers.CloseableIterator; +import io.druid.query.QueryRunner; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public class ScanQueryLimitRowIterator implements CloseableIterator +{ + private Yielder yielder; + private String resultFormat; + private int limit = 0; + private int count = 0; + + public ScanQueryLimitRowIterator( + QueryRunner baseRunner, ScanQuery query, + Map responseContext + ) + { + resultFormat = query.getResultFormat(); + limit = query.getLimit(); + Sequence baseSequence = baseRunner.run(query, responseContext); + yielder = baseSequence.toYielder( + (ScanResultValue) null, + new YieldingAccumulator() + { + @Override + public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue in) + { + yield(); + return in; + } + } + ); + } + + @Override + public boolean hasNext() + { + return !yielder.isDone() && count < limit; + } + + @Override + public ScanResultValue next() + { + ScanResultValue batch = yielder.get(); + if (ScanQuery.RESULT_FORMAT_COMPACTED_LIST.equals(resultFormat) || + ScanQuery.RESULT_FORMAT_LIST.equals(resultFormat)) { + List events = (List) batch.getEvents(); + if (events.size() <= limit - count) { + count += events.size(); + yielder = yielder.next(null); + return batch; + } else { + // last batch + int left = limit - count; + count = limit; + return new ScanResultValue(batch.getSegmentId(), batch.getColumns(), events.subList(0, left)); + } + } + throw new UnsupportedOperationException(ScanQuery.RESULT_FORMAT_VALUE_VECTOR + " is not supported yet"); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException + { + yielder.close(); + } +} diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java new file mode 100644 index 000000000000..222efda0a8e1 --- /dev/null +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryQueryToolChest.java @@ -0,0 +1,114 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.druid.query.scan; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Function; +import com.google.common.base.Functions; +import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.java.util.common.guava.BaseSequence; +import io.druid.java.util.common.guava.CloseQuietly; +import io.druid.java.util.common.guava.Sequence; +import io.druid.query.DruidMetrics; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChest; +import io.druid.query.aggregation.MetricManipulationFn; + +import java.util.Map; + +public class ScanQueryQueryToolChest extends QueryToolChest +{ + private static final TypeReference TYPE_REFERENCE = new TypeReference() + { + }; + + @Override + public QueryRunner mergeResults(final QueryRunner runner) + { + return new QueryRunner() + { + @Override + public Sequence run( + final Query query, final Map responseContext + ) + { + ScanQuery scanQuery = (ScanQuery) query; + if (scanQuery.getLimit() == Integer.MAX_VALUE) { + return runner.run(query, responseContext); + } + return new BaseSequence<>( + new BaseSequence.IteratorMaker() + { + @Override + public ScanQueryLimitRowIterator make() + { + return new ScanQueryLimitRowIterator(runner, (ScanQuery) query, responseContext); + } + + @Override + public void cleanup(ScanQueryLimitRowIterator iterFromMake) + { + CloseQuietly.close(iterFromMake); + } + } + ); + } + }; + } + + @Override + public ServiceMetricEvent.Builder makeMetricBuilder(ScanQuery query) + { + return DruidMetrics.makePartialQueryTimeMetric(query); + } + + @Override + public Function makePreComputeManipulatorFn( + ScanQuery query, MetricManipulationFn fn + ) + { + return Functions.identity(); + } + + @Override + public TypeReference getResultTypeReference() + { + return TYPE_REFERENCE; + } + + @Override + public QueryRunner preMergeQueryDecoration(final QueryRunner runner) + { + return new QueryRunner() + { + @Override + public Sequence run( + Query query, Map responseContext + ) + { + ScanQuery scanQuery = (ScanQuery) query; + if (scanQuery.getDimensionsFilter() != null) { + scanQuery = scanQuery.withDimFilter(scanQuery.getDimensionsFilter().optimize()); + } + return runner.run(scanQuery, responseContext); + } + }; + } +} diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryRunnerFactory.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryRunnerFactory.java new file mode 100644 index 000000000000..6b1244c5ff8a --- /dev/null +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryRunnerFactory.java @@ -0,0 +1,128 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.druid.query.scan; + +import com.google.common.base.Function; +import com.google.inject.Inject; +import io.druid.common.utils.JodaUtils; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Sequences; +import io.druid.query.Query; +import io.druid.query.QueryContextKeys; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import io.druid.segment.Segment; + +import java.util.Map; +import java.util.concurrent.ExecutorService; + +public class ScanQueryRunnerFactory implements QueryRunnerFactory +{ + public static final String CTX_TIMEOUT_AT = "timeoutAt"; + public static final String CTX_COUNT = "count"; + private final ScanQueryQueryToolChest toolChest; + private final ScanQueryEngine engine; + + @Inject + public ScanQueryRunnerFactory( + ScanQueryQueryToolChest toolChest, + ScanQueryEngine engine + ) + { + this.toolChest = toolChest; + this.engine = engine; + } + + @Override + public QueryRunner createRunner(Segment segment) + { + return new ScanQueryRunner(engine, segment); + } + + @Override + public QueryRunner mergeRunners( + ExecutorService queryExecutor, + final Iterable> queryRunners + ) + { + // in single thread and in jetty thread instead of processing thread + return new QueryRunner() + { + @Override + public Sequence run( + final Query query, final Map responseContext + ) + { + final Number queryTimeout = query.getContextValue(QueryContextKeys.TIMEOUT, null); + final long timeoutAt = queryTimeout == null + ? JodaUtils.MAX_INSTANT : System.currentTimeMillis() + queryTimeout.longValue(); + responseContext.put(CTX_TIMEOUT_AT, timeoutAt); + return Sequences.concat( + Sequences.map( + Sequences.simple(queryRunners), + new Function, Sequence>() + { + @Override + public Sequence apply(final QueryRunner input) + { + return input.run(query, responseContext); + } + } + ) + ); + } + }; + } + + @Override + public QueryToolChest getToolchest() + { + return toolChest; + } + + private class ScanQueryRunner implements QueryRunner + { + private final ScanQueryEngine engine; + private final Segment segment; + + public ScanQueryRunner(ScanQueryEngine engine, Segment segment) + { + this.engine = engine; + this.segment = segment; + } + + @Override + public Sequence run( + Query query, Map responseContext + ) + { + if (!(query instanceof ScanQuery)) { + throw new ISE("Got a [%s] which isn't a %s", query.getClass(), ScanQuery.class); + } + + // it happens in unit tests + if (responseContext.get(CTX_TIMEOUT_AT) == null) { + responseContext.put(CTX_TIMEOUT_AT, JodaUtils.MAX_INSTANT); + }; + return engine.process((ScanQuery) query, segment, responseContext); + } + } +} diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanResultValue.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanResultValue.java new file mode 100644 index 000000000000..76d8bf5dba97 --- /dev/null +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanResultValue.java @@ -0,0 +1,115 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package io.druid.query.scan; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +public class ScanResultValue implements Comparable +{ + public static final String timestampKey = "timestamp"; + + private final String segmentId; + private final List columns; + private final Object events; + + @JsonCreator + public ScanResultValue( + @JsonProperty("segmentId") String segmentId, + @JsonProperty("columns") List columns, + @JsonProperty("events") Object events + ) + { + this.segmentId = segmentId; + this.columns = columns; + this.events = events; + } + + @JsonProperty + public String getSegmentId() + { + return segmentId; + } + + @JsonProperty + public List getColumns() + { + return columns; + } + + @JsonProperty + public Object getEvents() + { + return events; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + ScanResultValue that = (ScanResultValue) o; + + if (segmentId != null ? !segmentId.equals(that.segmentId) : that.segmentId != null) { + return false; + } + if (columns != null ? !columns.equals(that.columns) : that.columns != null) { + return false; + } + return events != null ? events.equals(that.events) : that.events == null; + } + + @Override + public int hashCode() + { + int result = segmentId != null ? segmentId.hashCode() : 0; + result = 31 * result + (columns != null ? columns.hashCode() : 0); + result = 31 * result + (events != null ? events.hashCode() : 0); + return result; + } + + @Override + public String toString() + { + return "ScanResultValue{" + + "segmentId='" + segmentId + '\'' + + ", columns=" + columns + + ", events=" + events + + '}'; + } + + @Override + public int compareTo(ScanResultValue that) + { + if (that == null) { + return 1; + } + if (segmentId != null && that.segmentId != null) { + return segmentId.compareTo(that.segmentId); + } + return segmentId != null ? 1 : 0; + } +} diff --git a/extensions-contrib/scan-query/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-contrib/scan-query/src/main/resources/META-INF/services/io.druid.initialization.DruidModule new file mode 100644 index 000000000000..1459501bf4a8 --- /dev/null +++ b/extensions-contrib/scan-query/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1 @@ +io.druid.query.scan.ScanQueryDruidModule \ No newline at end of file diff --git a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java new file mode 100644 index 000000000000..f2fbf0f5f41e --- /dev/null +++ b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/MultiSegmentScanQueryTest.java @@ -0,0 +1,242 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.scan; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.io.CharSource; +import com.google.common.util.concurrent.MoreExecutors; +import io.druid.granularity.QueryGranularities; +import io.druid.java.util.common.guava.MergeSequence; +import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Sequences; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.TableDataSource; +import io.druid.query.select.SelectQueryRunnerTest; +import io.druid.segment.IncrementalIndexSegment; +import io.druid.segment.Segment; +import io.druid.segment.TestIndex; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.incremental.OnheapIncrementalIndex; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.NoneShardSpec; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang.StringUtils; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + */ +@RunWith(Parameterized.class) +public class MultiSegmentScanQueryTest +{ + private static final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest(); + + private static final QueryRunnerFactory factory = new ScanQueryRunnerFactory( + toolChest, + new ScanQueryEngine() + ); + + // time modified version of druid.sample.tsv + public static final String[] V_0112 = { + "2011-01-12T00:00:00.000Z spot automotive preferred apreferred 100.000000", + "2011-01-12T01:00:00.000Z spot business preferred bpreferred 100.000000", + "2011-01-12T02:00:00.000Z spot entertainment preferred epreferred 100.000000", + "2011-01-12T03:00:00.000Z spot health preferred hpreferred 100.000000", + "2011-01-12T04:00:00.000Z spot mezzanine preferred mpreferred 100.000000", + "2011-01-12T05:00:00.000Z spot news preferred npreferred 100.000000", + "2011-01-12T06:00:00.000Z spot premium preferred ppreferred 100.000000", + "2011-01-12T07:00:00.000Z spot technology preferred tpreferred 100.000000", + "2011-01-12T08:00:00.000Z spot travel preferred tpreferred 100.000000", + "2011-01-12T09:00:00.000Z total_market mezzanine preferred mpreferred 1000.000000", + "2011-01-12T10:00:00.000Z total_market premium preferred ppreferred 1000.000000", + "2011-01-12T11:00:00.000Z upfront mezzanine preferred mpreferred 800.000000 value", + "2011-01-12T12:00:00.000Z upfront premium preferred ppreferred 800.000000 value", + "2011-01-12T13:00:00.000Z upfront premium preferred ppreferred2 800.000000 value" + }; + public static final String[] V_0113 = { + "2011-01-13T00:00:00.000Z spot automotive preferred apreferred 94.874713", + "2011-01-13T01:00:00.000Z spot business preferred bpreferred 103.629399", + "2011-01-13T02:00:00.000Z spot entertainment preferred epreferred 110.087299", + "2011-01-13T03:00:00.000Z spot health preferred hpreferred 114.947403", + "2011-01-13T04:00:00.000Z spot mezzanine preferred mpreferred 104.465767", + "2011-01-13T05:00:00.000Z spot news preferred npreferred 102.851683", + "2011-01-13T06:00:00.000Z spot premium preferred ppreferred 108.863011", + "2011-01-13T07:00:00.000Z spot technology preferred tpreferred 111.356672", + "2011-01-13T08:00:00.000Z spot travel preferred tpreferred 106.236928", + "2011-01-13T09:00:00.000Z total_market mezzanine preferred mpreferred 1040.945505", + "2011-01-13T10:00:00.000Z total_market premium preferred ppreferred 1689.012875", + "2011-01-13T11:00:00.000Z upfront mezzanine preferred mpreferred 826.060182 value", + "2011-01-13T12:00:00.000Z upfront premium preferred ppreferred 1564.617729 value" + }; + + private static Segment segment0; + private static Segment segment1; + + @BeforeClass + public static void setup() throws IOException + { + CharSource v_0112 = CharSource.wrap(StringUtils.join(V_0112, "\n")); + CharSource v_0113 = CharSource.wrap(StringUtils.join(V_0113, "\n")); + + IncrementalIndex index0 = TestIndex.loadIncrementalIndex(newIndex("2011-01-12T00:00:00.000Z"), v_0112); + IncrementalIndex index1 = TestIndex.loadIncrementalIndex(newIndex("2011-01-13T00:00:00.000Z"), v_0113); + + segment0 = new IncrementalIndexSegment(index0, makeIdentifier(index0, "v1")); + segment1 = new IncrementalIndexSegment(index1, makeIdentifier(index1, "v1")); + } + + private static String makeIdentifier(IncrementalIndex index, String version) + { + return makeIdentifier(index.getInterval(), version); + } + + private static String makeIdentifier(Interval interval, String version) + { + return DataSegment.makeDataSegmentIdentifier( + QueryRunnerTestHelper.dataSource, + interval.getStart(), + interval.getEnd(), + version, + NoneShardSpec.instance() + ); + } + + private static IncrementalIndex newIndex(String minTimeStamp) + { + return newIndex(minTimeStamp, 10000); + } + + private static IncrementalIndex newIndex(String minTimeStamp, int maxRowCount) + { + final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() + .withMinTimestamp(new DateTime(minTimeStamp).getMillis()) + .withQueryGranularity(QueryGranularities.HOUR) + .withMetrics(TestIndex.METRIC_AGGS) + .build(); + return new OnheapIncrementalIndex(schema, true, maxRowCount); + } + + @AfterClass + public static void clear() + { + IOUtils.closeQuietly(segment0); + IOUtils.closeQuietly(segment1); + } + + @Parameterized.Parameters(name = "limit={0},batchSize={1}") + public static Iterable constructorFeeder() throws IOException + { + return QueryRunnerTestHelper.cartesian(Arrays.asList(0, 1, 3, 7, 10, 20, 1000), Arrays.asList(0, 1, 3, 6, 7, 10, 123, 2000)); + } + + private final int limit; + private final int batchSize; + + public MultiSegmentScanQueryTest(int limit, int batchSize) + { + this.limit = limit; + this.batchSize = batchSize; + } + + private ScanQuery.ScanQueryBuilder newBuilder() + { + return ScanQuery.newScanQueryBuilder() + .dataSource(new TableDataSource(QueryRunnerTestHelper.dataSource)) + .intervals(SelectQueryRunnerTest.I_0112_0114) + .batchSize(batchSize) + .columns(Arrays.asList()) + .limit(limit); + } + + @Test + public void testMergeRunnersWithLimit() + { + ScanQuery query = newBuilder().build(); + List results = Sequences.toList( + factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.of( + factory.createRunner(segment0), + factory.createRunner(segment1) + )).run(query, new HashMap()), + Lists.newArrayList() + ); + int totalCount = 0; + for (ScanResultValue result : results) { + System.out.println(((List) result.getEvents()).size()); + totalCount += ((List) result.getEvents()).size(); + } + Assert.assertEquals( + totalCount, + limit != 0 ? Math.min(limit, V_0112.length + V_0113.length) : V_0112.length + V_0113.length + ); + } + + @Test + public void testMergeResultsWithLimit() + { + QueryRunner runner = toolChest.mergeResults( + new QueryRunner() { + @Override + public Sequence run( + Query query, Map responseContext + ) + { + // simulate results back from 2 historicals + List> sequences = Lists.newArrayListWithExpectedSize(2); + sequences.add(factory.createRunner(segment0).run(query, new HashMap())); + sequences.add(factory.createRunner(segment1).run(query, new HashMap())); + return new MergeSequence<>( + query.getResultOrdering(), + Sequences.simple(sequences) + ); + } + } + ); + ScanQuery query = newBuilder().build(); + List results = Sequences.toList( + runner.run(query, new HashMap()), + Lists.newArrayList() + ); + int totalCount = 0; + for (ScanResultValue result : results) { + totalCount += ((List) result.getEvents()).size(); + } + Assert.assertEquals( + totalCount, + limit != 0 ? Math.min(limit, V_0112.length + V_0113.length) : V_0112.length + V_0113.length + ); + } +} diff --git a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java new file mode 100644 index 000000000000..e5277256c5e2 --- /dev/null +++ b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQueryRunnerTest.java @@ -0,0 +1,605 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.scan; + +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.ObjectArrays; +import com.google.common.collect.Sets; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.guava.Sequences; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.TableDataSource; +import io.druid.query.extraction.MapLookupExtractor; +import io.druid.query.filter.AndDimFilter; +import io.druid.query.filter.DimFilter; +import io.druid.query.filter.SelectorDimFilter; +import io.druid.query.lookup.LookupExtractionFn; +import io.druid.query.spec.LegacySegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + */ +@RunWith(Parameterized.class) +public class ScanQueryRunnerTest +{ + // copied from druid.sample.tsv + public static final String[] V_0112 = { + "2011-01-12T00:00:00.000Z spot automotive preferred apreferred 100.000000", + "2011-01-12T00:00:00.000Z spot business preferred bpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot entertainment preferred epreferred 100.000000", + "2011-01-12T00:00:00.000Z spot health preferred hpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot mezzanine preferred mpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot news preferred npreferred 100.000000", + "2011-01-12T00:00:00.000Z spot premium preferred ppreferred 100.000000", + "2011-01-12T00:00:00.000Z spot technology preferred tpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot travel preferred tpreferred 100.000000", + "2011-01-12T00:00:00.000Z total_market mezzanine preferred mpreferred 1000.000000", + "2011-01-12T00:00:00.000Z total_market premium preferred ppreferred 1000.000000", + "2011-01-12T00:00:00.000Z upfront mezzanine preferred mpreferred 800.000000 value", + "2011-01-12T00:00:00.000Z upfront premium preferred ppreferred 800.000000 value" + }; + public static final String[] V_0113 = { + "2011-01-13T00:00:00.000Z spot automotive preferred apreferred 94.874713", + "2011-01-13T00:00:00.000Z spot business preferred bpreferred 103.629399", + "2011-01-13T00:00:00.000Z spot entertainment preferred epreferred 110.087299", + "2011-01-13T00:00:00.000Z spot health preferred hpreferred 114.947403", + "2011-01-13T00:00:00.000Z spot mezzanine preferred mpreferred 104.465767", + "2011-01-13T00:00:00.000Z spot news preferred npreferred 102.851683", + "2011-01-13T00:00:00.000Z spot premium preferred ppreferred 108.863011", + "2011-01-13T00:00:00.000Z spot technology preferred tpreferred 111.356672", + "2011-01-13T00:00:00.000Z spot travel preferred tpreferred 106.236928", + "2011-01-13T00:00:00.000Z total_market mezzanine preferred mpreferred 1040.945505", + "2011-01-13T00:00:00.000Z total_market premium preferred ppreferred 1689.012875", + "2011-01-13T00:00:00.000Z upfront mezzanine preferred mpreferred 826.060182 value", + "2011-01-13T00:00:00.000Z upfront premium preferred ppreferred 1564.617729 value" + }; + + public static final QuerySegmentSpec I_0112_0114 = new LegacySegmentSpec( + new Interval("2011-01-12/2011-01-14") + ); + public static final String[] V_0112_0114 = ObjectArrays.concat(V_0112, V_0113, String.class); + + private static final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest(); + + @Parameterized.Parameters(name = "{0}") + public static Iterable constructorFeeder() throws IOException + { + return QueryRunnerTestHelper.cartesian( + QueryRunnerTestHelper.makeQueryRunners( + new ScanQueryRunnerFactory( + toolChest, + new ScanQueryEngine() + ) + ) + ); + } + + private final QueryRunner runner; + + public ScanQueryRunnerTest(QueryRunner runner) + { + this.runner = runner; + } + + private ScanQuery.ScanQueryBuilder newTestQuery() + { + return ScanQuery.newScanQueryBuilder() + .dataSource(new TableDataSource(QueryRunnerTestHelper.dataSource)) + .columns(Arrays.asList()) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .limit(3); + } + + @Test + public void testFullOnSelect() + { + List columns = Lists.newArrayList( + ScanResultValue.timestampKey, + "market", + "quality", + "placement", + "placementish", + "partial_null_column", + "null_column", + "index", + "indexMin", + "indexMaxPlusTen", + "quality_uniques" + ); + ScanQuery query = newTestQuery() + .intervals(I_0112_0114) + .build(); + + HashMap context = new HashMap(); + Iterable results = Sequences.toList( + runner.run(query, context), + Lists.newArrayList() + ); + + List expectedResults = toExpected( + toFullEvents(V_0112_0114), + columns, + 0, + 3 + ); + verify(expectedResults, populateNullColumnAtLastForQueryableIndexCase(results, "null_column")); + } + + @Test + public void testFullOnSelectAsCompactedList() + { + final List columns = Lists.newArrayList( + ScanResultValue.timestampKey, + "market", + "quality", + "placement", + "placementish", + "partial_null_column", + "null_column", + "index", + "indexMin", + "indexMaxPlusTen", + "quality_uniques" + ); + ScanQuery query = newTestQuery() + .intervals(I_0112_0114) + .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .build(); + + HashMap context = new HashMap(); + Iterable results = Sequences.toList( + runner.run(query, context), + Lists.newArrayList() + ); + + List expectedResults = toExpected( + toFullEvents(V_0112_0114), + columns, + 0, + 3 + ); + verify(expectedResults, populateNullColumnAtLastForQueryableIndexCase(compactedListToRow(results), "null_column")); + } + + @Test + public void testSelectWithDimsAndMets() + { + ScanQuery query = newTestQuery() + .intervals(I_0112_0114) + .columns(QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.indexMetric) + .build(); + + HashMap context = new HashMap(); + Iterable results = Sequences.toList( + runner.run(query, context), + Lists.newArrayList() + ); + + List expectedResults = toExpected( + toEvents( + new String[]{ + ScanResultValue.timestampKey + ":TIME", + QueryRunnerTestHelper.marketDimension + ":STRING", + null, + null, + null, + QueryRunnerTestHelper.indexMetric + ":FLOAT" + }, + V_0112_0114 + ), + Lists.newArrayList(ScanResultValue.timestampKey, "market", "index"), + 0, + 3 + ); + verify(expectedResults, results); + } + + @Test + public void testSelectWithDimsAndMetsAsCompactedList() + { + ScanQuery query = newTestQuery() + .intervals(I_0112_0114) + .columns(QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.indexMetric) + .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .build(); + + HashMap context = new HashMap(); + Iterable results = Sequences.toList( + runner.run(query, context), + Lists.newArrayList() + ); + + List expectedResults = toExpected( + toEvents( + new String[]{ + ScanResultValue.timestampKey + ":TIME", + QueryRunnerTestHelper.marketDimension + ":STRING", + null, + null, + null, + QueryRunnerTestHelper.indexMetric + ":FLOAT" + }, + V_0112_0114 + ), + Lists.newArrayList(ScanResultValue.timestampKey, "market", "index"), + 0, + 3 + ); + verify(expectedResults, compactedListToRow(results)); + } + + @Test + public void testFullOnSelectWithFilterAndLimit() + { + // limits + for (int limit : new int[]{3, 1, 5, 7, 0}) { + ScanQuery query = newTestQuery() + .intervals(I_0112_0114) + .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) + .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) + .limit(limit) + .build(); + + HashMap context = new HashMap(); + Iterable results = Sequences.toList( + runner.run(query, context), + Lists.newArrayList() + ); + + final List>> events = toEvents( + new String[]{ + ScanResultValue.timestampKey + ":TIME", + null, + QueryRunnerTestHelper.qualityDimension + ":STRING", + null, + null, + QueryRunnerTestHelper.indexMetric + ":FLOAT" + }, + // filtered values with day granularity + new String[]{ + "2011-01-12T00:00:00.000Z spot automotive preferred apreferred 100.000000", + "2011-01-12T00:00:00.000Z spot business preferred bpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot entertainment preferred epreferred 100.000000", + "2011-01-12T00:00:00.000Z spot health preferred hpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot mezzanine preferred mpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot news preferred npreferred 100.000000", + "2011-01-12T00:00:00.000Z spot premium preferred ppreferred 100.000000", + "2011-01-12T00:00:00.000Z spot technology preferred tpreferred 100.000000", + "2011-01-12T00:00:00.000Z spot travel preferred tpreferred 100.000000" + }, + new String[]{ + "2011-01-13T00:00:00.000Z spot automotive preferred apreferred 94.874713", + "2011-01-13T00:00:00.000Z spot business preferred bpreferred 103.629399", + "2011-01-13T00:00:00.000Z spot entertainment preferred epreferred 110.087299", + "2011-01-13T00:00:00.000Z spot health preferred hpreferred 114.947403", + "2011-01-13T00:00:00.000Z spot mezzanine preferred mpreferred 104.465767", + "2011-01-13T00:00:00.000Z spot news preferred npreferred 102.851683", + "2011-01-13T00:00:00.000Z spot premium preferred ppreferred 108.863011", + "2011-01-13T00:00:00.000Z spot technology preferred tpreferred 111.356672", + "2011-01-13T00:00:00.000Z spot travel preferred tpreferred 106.236928" + } + ); + + List expectedResults = toExpected( + events, + Lists.newArrayList(ScanResultValue.timestampKey, "quality", "index"), + 0, + limit + ); + verify(expectedResults, results); + } + } + + @Test + public void testSelectWithFilterLookupExtractionFn() + { + Map extractionMap = new HashMap<>(); + extractionMap.put("total_market", "replaced"); + MapLookupExtractor mapLookupExtractor = new MapLookupExtractor(extractionMap, false); + LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(mapLookupExtractor, false, null, true, true); + ScanQuery query = newTestQuery() + .intervals(I_0112_0114) + .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "replaced", lookupExtractionFn)) + .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) + .build(); + + Iterable results = Sequences.toList( + runner.run(query, Maps.newHashMap()), + Lists.newArrayList() + ); + Iterable resultsOptimize = Sequences.toList( + toolChest.postMergeQueryDecoration(toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner))). + run(query, Maps.newHashMap()), Lists.newArrayList() + ); + + final List>> events = toEvents( + new String[]{ + ScanResultValue.timestampKey + ":TIME", + null, + QueryRunnerTestHelper.qualityDimension + ":STRING", + null, + null, + QueryRunnerTestHelper.indexMetric + ":FLOAT" + }, + // filtered values with day granularity + new String[]{ + "2011-01-12T00:00:00.000Z total_market mezzanine preferred mpreferred 1000.000000", + "2011-01-12T00:00:00.000Z total_market premium preferred ppreferred 1000.000000" + }, + new String[]{ + "2011-01-13T00:00:00.000Z total_market mezzanine preferred mpreferred 1040.945505", + "2011-01-13T00:00:00.000Z total_market premium preferred ppreferred 1689.012875" + } + ); + + List expectedResults = toExpected( + events, + Lists.newArrayList(ScanResultValue.timestampKey, QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric), + 0, + 3 + ); + + verify(expectedResults, results); + verify(expectedResults, resultsOptimize); + } + + @Test + public void testFullSelectNoResults() + { + ScanQuery query = newTestQuery() + .intervals(I_0112_0114) + .filters( + new AndDimFilter( + Arrays.asList( + new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null), + new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "foo", null) + ) + ) + ) + .build(); + + Iterable results = Sequences.toList( + runner.run(query, Maps.newHashMap()), + Lists.newArrayList() + ); + + List expectedResults = Arrays.asList( + ); + + verify(expectedResults, populateNullColumnAtLastForQueryableIndexCase(results, "null_column")); + } + + @Test + public void testFullSelectNoDimensionAndMetric() + { + ScanQuery query = newTestQuery() + .intervals(I_0112_0114) + .columns("foo", "foo2") + .build(); + + Iterable results = Sequences.toList( + runner.run(query, Maps.newHashMap()), + Lists.newArrayList() + ); + + final List>> events = toEvents( + new String[]{ + ScanResultValue.timestampKey + ":TIME" + }, + V_0112_0114 + ); + + List expectedResults = toExpected( + events, + Lists.newArrayList(ScanResultValue.timestampKey, "foo", "foo2"), + 0, + 3 + ); + verify(expectedResults, results); + } + + private List>> toFullEvents(final String[]... valueSet) + { + return toEvents( + new String[]{ + ScanResultValue.timestampKey + ":TIME", + QueryRunnerTestHelper.marketDimension + ":STRING", + QueryRunnerTestHelper.qualityDimension + ":STRING", + QueryRunnerTestHelper.placementDimension + ":STRING", + QueryRunnerTestHelper.placementishDimension + ":STRINGS", + QueryRunnerTestHelper.indexMetric + ":FLOAT", + QueryRunnerTestHelper.partialNullDimension + ":STRING" + }, + valueSet + ); + } + + private List>> toEvents(final String[] dimSpecs, final String[]... valueSet) + { + List values = Lists.newArrayList(); + for (String[] vSet : valueSet) { + values.addAll(Arrays.asList(vSet)); + } + List>> events = Lists.newArrayList(); + events.add( + Lists.newArrayList( + Iterables.transform( + values, new Function>() + { + @Override + public Map apply(String input) + { + Map event = Maps.newHashMap(); + String[] values = input.split("\\t"); + for (int i = 0; i < dimSpecs.length; i++) { + if (dimSpecs[i] == null || i >= dimSpecs.length || i >= values.length) { + continue; + } + String[] specs = dimSpecs[i].split(":"); + event.put( + specs[0], + specs.length == 1 || specs[1].equals("STRING") ? values[i] : + specs[1].equals("TIME") ? new DateTime(values[i]) : + specs[1].equals("FLOAT") ? Float.valueOf(values[i]) : + specs[1].equals("DOUBLE") ? Double.valueOf(values[i]) : + specs[1].equals("LONG") ? Long.valueOf(values[i]) : + specs[1].equals("NULL") ? null : + specs[1].equals("STRINGS") ? Arrays.asList(values[i].split("\u0001")) : + values[i] + ); + } + return event; + } + } + ) + ) + ); + return events; + } + + private List toExpected( + List>> targets, + List columns, + final int offset, + final int limit + ) + { + List expected = Lists.newArrayListWithExpectedSize(targets.size()); + for (List> group : targets) { + List> events = Lists.newArrayListWithExpectedSize(limit); + int end = Math.min(group.size(), offset + limit); + if (end == 0) { + end = group.size(); + } + events.addAll(group.subList(offset, end)); + expected.add( + new ScanResultValue( + QueryRunnerTestHelper.segmentId, + columns, + events + ) + ); + } + return expected; + } + + private static void verify( + Iterable expectedResults, + Iterable actualResults + ) + { + Iterator expectedIter = expectedResults.iterator(); + Iterator actualIter = actualResults.iterator(); + + while (expectedIter.hasNext()) { + ScanResultValue expected = expectedIter.next(); + ScanResultValue actual = actualIter.next(); + + Assert.assertEquals(expected.getSegmentId(), actual.getSegmentId()); + + Set exColumns = Sets.newTreeSet(expected.getColumns()); + Set acColumns = Sets.newTreeSet(actual.getColumns()); + Assert.assertEquals(exColumns, acColumns); + + Iterator> expectedEvts = ((List>) expected.getEvents()).iterator(); + Iterator> actualEvts = ((List>) actual.getEvents()).iterator(); + + while (expectedEvts.hasNext()) { + Map exHolder = expectedEvts.next(); + Map acHolder = actualEvts.next(); + + for (Map.Entry ex : exHolder.entrySet()) { + Object actVal = acHolder.get(ex.getKey()); + + // work around for current II limitations + if (acHolder.get(ex.getKey()) instanceof Double) { + actVal = ((Double) actVal).floatValue(); + } + Assert.assertEquals("invalid value for " + ex.getKey(), ex.getValue(), actVal); + } + } + + if (actualEvts.hasNext()) { + throw new ISE("This event iterator should be exhausted!"); + } + } + + if (actualIter.hasNext()) { + throw new ISE("This iterator should be exhausted!"); + } + } + + private static Iterable populateNullColumnAtLastForQueryableIndexCase( + Iterable results, + String columnName + ) + { + // A Queryable index does not have the null column when it has loaded a index. + for (ScanResultValue value : results) { + List columns = value.getColumns(); + if (columns.contains(columnName)) { + break; + } + columns.add(columnName); + } + + return results; + } + + private Iterable compactedListToRow(Iterable results) { + return Iterables.transform(results, new Function() + { + @Override + public ScanResultValue apply(ScanResultValue input) + { + List mapEvents = Lists.newLinkedList(); + List events = ((List) input.getEvents()); + for (int i = 0; i < events.size(); i++) { + Iterator compactedEventIter = ((List) events.get(i)).iterator(); + Map mapEvent = new LinkedHashMap(); + for (String column : input.getColumns()) { + mapEvent.put(column, compactedEventIter.next()); + } + mapEvents.add(mapEvent); + } + return new ScanResultValue(input.getSegmentId(), input.getColumns(), mapEvents); + } + }); + } +} diff --git a/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQuerySpecTest.java b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQuerySpecTest.java new file mode 100644 index 000000000000..64d92b71547f --- /dev/null +++ b/extensions-contrib/scan-query/src/test/java/io/druid/query/scan/ScanQuerySpecTest.java @@ -0,0 +1,75 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.scan; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.TableDataSource; +import io.druid.query.spec.LegacySegmentSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; + +public class ScanQuerySpecTest +{ + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + @Test + public void testSerializationLegacyString() throws Exception + { + String legacy = + "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"}," + + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]}," + + "\"filter\":null," + + "\"columns\":[\"market\",\"quality\",\"index\"]," + + "\"limit\":3," + + "\"context\":null}"; + + String current = + "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"}," + + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]}," + + "\"resultFormat\":\"list\"," + + "\"batchSize\":20480," + + "\"limit\":3," + + "\"filter\":null," + + "\"columns\":[\"market\",\"quality\",\"index\"]," + + "\"context\":null," + + "\"descending\":false}"; + + ScanQuery query = new ScanQuery( + new TableDataSource(QueryRunnerTestHelper.dataSource), + new LegacySegmentSpec(new Interval("2011-01-12/2011-01-14")), + null, + 0, + 3, + null, + Arrays.asList("market", "quality", "index"), + null + ); + + String actual = jsonMapper.writeValueAsString(query); + Assert.assertEquals(current, actual); + Assert.assertEquals(query, jsonMapper.readValue(actual, ScanQuery.class)); + Assert.assertEquals(query, jsonMapper.readValue(legacy, ScanQuery.class)); + } +} diff --git a/extensions-contrib/sqlserver-metadata-storage/pom.xml b/extensions-contrib/sqlserver-metadata-storage/pom.xml index 81f32df4f220..917d1a95b6df 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.9.2-SNAPSHOT + 0.9.3-SNAPSHOT ../../pom.xml diff --git a/extensions-core/histogram/pom.xml b/extensions-core/histogram/pom.xml index f5e359717209..794eee0b5fb7 100644 --- a/extensions-core/histogram/pom.xml +++ b/extensions-core/histogram/pom.xml @@ -37,6 +37,12 @@ ${project.parent.version} provided + + io.druid + druid-sql + ${project.parent.version} + provided + @@ -46,6 +52,13 @@ test test-jar + + io.druid + druid-sql + ${project.parent.version} + test-jar + test + junit junit diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramDruidModule.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramDruidModule.java index 56bb4035509e..288865b06c59 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramDruidModule.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramDruidModule.java @@ -24,7 +24,9 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import io.druid.initialization.DruidModule; +import io.druid.query.aggregation.histogram.sql.QuantileSqlAggregator; import io.druid.segment.serde.ComplexMetrics; +import io.druid.sql.guice.SqlBindings; import java.util.List; @@ -56,5 +58,10 @@ public void configure(Binder binder) if (ComplexMetrics.getSerdeForType("approximateHistogram") == null) { ComplexMetrics.registerSerde("approximateHistogram", new ApproximateHistogramFoldingSerde()); } + + if (binder != null) { + // Binder is null in some tests. + SqlBindings.addAggregator(binder, QuantileSqlAggregator.class); + } } } diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/QuantilePostAggregator.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/QuantilePostAggregator.java index efd6b7904b0a..7eefe04ff59f 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/QuantilePostAggregator.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/QuantilePostAggregator.java @@ -86,6 +86,32 @@ public float getProbability() return probability; } + @Override + public boolean equals(final Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + final QuantilePostAggregator that = (QuantilePostAggregator) o; + + if (Float.compare(that.probability, probability) != 0) { + return false; + } + return fieldName != null ? fieldName.equals(that.fieldName) : that.fieldName == null; + } + + @Override + public int hashCode() + { + int result = (probability != +0.0f ? Float.floatToIntBits(probability) : 0); + result = 31 * result + (fieldName != null ? fieldName.hashCode() : 0); + return result; + } + @Override public String toString() { diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java new file mode 100644 index 000000000000..fe3896f3b77d --- /dev/null +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java @@ -0,0 +1,163 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation.histogram.sql; + +import com.google.common.collect.ImmutableList; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.histogram.ApproximateHistogram; +import io.druid.query.aggregation.histogram.ApproximateHistogramAggregatorFactory; +import io.druid.query.aggregation.histogram.ApproximateHistogramFoldingAggregatorFactory; +import io.druid.query.aggregation.histogram.QuantilePostAggregator; +import io.druid.segment.column.ValueType; +import io.druid.sql.calcite.aggregation.Aggregation; +import io.druid.sql.calcite.aggregation.SqlAggregator; +import io.druid.sql.calcite.expression.Expressions; +import io.druid.sql.calcite.expression.RowExtraction; +import io.druid.sql.calcite.table.RowSignature; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlAggFunction; +import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.type.OperandTypes; +import org.apache.calcite.sql.type.ReturnTypes; +import org.apache.calcite.sql.type.SqlTypeFamily; +import org.apache.calcite.sql.type.SqlTypeName; + +import java.util.List; + +public class QuantileSqlAggregator implements SqlAggregator +{ + private static final SqlAggFunction FUNCTION_INSTANCE = new QuantileSqlAggFunction(); + private static final String NAME = "QUANTILE"; + + @Override + public SqlAggFunction calciteFunction() + { + return FUNCTION_INSTANCE; + } + + @Override + public Aggregation toDruidAggregation( + final String name, + final RowSignature rowSignature, + final List existingAggregations, + final Project project, + final AggregateCall aggregateCall + ) + { + final RowExtraction rex = Expressions.toRowExtraction( + rowSignature.getRowOrder(), + Expressions.fromFieldAccess( + rowSignature, + project, + aggregateCall.getArgList().get(0) + ) + ); + if (rex == null) { + return null; + } + + final RexNode probabilityArg = Expressions.fromFieldAccess( + rowSignature, + project, + aggregateCall.getArgList().get(1) + ); + final float probability = ((Number) RexLiteral.value(probabilityArg)).floatValue(); + + final AggregatorFactory aggregatorFactory; + final String histogramName = String.format("%s:agg", name); + + final int resolution = ApproximateHistogram.DEFAULT_HISTOGRAM_SIZE; + final int numBuckets = ApproximateHistogram.DEFAULT_BUCKET_SIZE; + final float lowerLimit = Float.NEGATIVE_INFINITY; + final float upperLimit = Float.POSITIVE_INFINITY; + + // Look for existing matching aggregatorFactory. + for (final Aggregation existing : existingAggregations) { + for (AggregatorFactory factory : existing.getAggregatorFactories()) { + if (factory instanceof ApproximateHistogramAggregatorFactory) { + final ApproximateHistogramAggregatorFactory theFactory = (ApproximateHistogramAggregatorFactory) factory; + if (theFactory.getFieldName().equals(rex.getColumn()) + && theFactory.getResolution() == resolution + && theFactory.getNumBuckets() == numBuckets + && theFactory.getLowerLimit() == lowerLimit + && theFactory.getUpperLimit() == upperLimit) { + // Found existing one. Use this. + return Aggregation.create( + ImmutableList.of(), + new QuantilePostAggregator(name, theFactory.getName(), probability) + ); + } + } + } + } + + if (rowSignature.getColumnType(rex.getColumn()) == ValueType.COMPLEX) { + aggregatorFactory = new ApproximateHistogramFoldingAggregatorFactory( + histogramName, + rex.getColumn(), + resolution, + numBuckets, + lowerLimit, + upperLimit + ); + } else { + aggregatorFactory = new ApproximateHistogramAggregatorFactory( + histogramName, + rex.getColumn(), + resolution, + numBuckets, + lowerLimit, + upperLimit + ); + } + + return Aggregation.create( + ImmutableList.of(aggregatorFactory), + new QuantilePostAggregator(name, histogramName, probability) + ); + } + + private static class QuantileSqlAggFunction extends SqlAggFunction + { + private static final String SIGNATURE = "'" + NAME + "(column, probability)'"; + + QuantileSqlAggFunction() + { + super( + NAME, + null, + SqlKind.OTHER_FUNCTION, + ReturnTypes.explicit(SqlTypeName.DOUBLE), + null, + OperandTypes.and( + OperandTypes.sequence(SIGNATURE, OperandTypes.ANY, OperandTypes.LITERAL), + OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC) + ), + SqlFunctionCategory.NUMERIC, + false, + false + ); + } + } +} diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java new file mode 100644 index 000000000000..1e92d5e9d41e --- /dev/null +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java @@ -0,0 +1,134 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation.histogram.sql; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import io.druid.granularity.QueryGranularities; +import io.druid.java.util.common.guava.Sequences; +import io.druid.query.Druids; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.histogram.ApproximateHistogramAggregatorFactory; +import io.druid.query.aggregation.histogram.QuantilePostAggregator; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.sql.calcite.aggregation.SqlAggregator; +import io.druid.sql.calcite.filtration.Filtration; +import io.druid.sql.calcite.planner.Calcites; +import io.druid.sql.calcite.planner.DruidOperatorTable; +import io.druid.sql.calcite.planner.PlannerConfig; +import io.druid.sql.calcite.planner.PlannerFactory; +import io.druid.sql.calcite.planner.PlannerResult; +import io.druid.sql.calcite.util.CalciteTests; +import io.druid.sql.calcite.util.QueryLogHook; +import io.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.tools.Planner; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.util.ArrayList; +import java.util.List; + +import static io.druid.sql.calcite.CalciteQueryTest.TIMESERIES_CONTEXT; + +public class QuantileSqlAggregatorTest +{ + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Rule + public QueryLogHook queryLogHook = QueryLogHook.create(); + + private SpecificSegmentsQuerySegmentWalker walker; + private PlannerFactory plannerFactory; + + @Before + public void setUp() throws Exception + { + Calcites.setSystemProperties(); + walker = CalciteTests.createMockWalker(temporaryFolder.newFolder()); + final PlannerConfig plannerConfig = new PlannerConfig(); + final SchemaPlus rootSchema = Calcites.createRootSchema( + CalciteTests.createMockSchema( + walker, + plannerConfig + ) + ); + final DruidOperatorTable operatorTable = new DruidOperatorTable( + ImmutableSet.of( + new QuantileSqlAggregator() + ) + ); + plannerFactory = new PlannerFactory(rootSchema, operatorTable, plannerConfig); + } + + @After + public void tearDown() throws Exception + { + walker.close(); + walker = null; + } + + @Test + public void testQuantileOnFloatAndLongs() throws Exception + { + try (final Planner planner = plannerFactory.createPlanner()) { + final String sql = "SELECT QUANTILE(m1, 0.01), QUANTILE(m1, 0.5), QUANTILE(m1, 0.99), QUANTILE(cnt, 0.5) FROM foo"; + final PlannerResult plannerResult = Calcites.plan(planner, sql); + + // Verify results + final List results = Sequences.toList(plannerResult.run(), new ArrayList()); + final List expectedResults = ImmutableList.of( + new Object[]{1.0, 3.0, 5.940000057220459, 1.0} + ); + Assert.assertEquals(expectedResults.size(), results.size()); + for (int i = 0; i < expectedResults.size(); i++) { + Assert.assertArrayEquals(expectedResults.get(i), results.get(i)); + } + + // Verify query + Assert.assertEquals( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .granularity(QueryGranularities.ALL) + .aggregators(ImmutableList.of( + new ApproximateHistogramAggregatorFactory("a0:agg", "m1", null, null, null, null), + new ApproximateHistogramAggregatorFactory("a3:agg", "cnt", null, null, null, null) + )) + .postAggregators(ImmutableList.of( + new QuantilePostAggregator("a0", "a0:agg", 0.01f), + new QuantilePostAggregator("a1", "a0:agg", 0.50f), + new QuantilePostAggregator("a2", "a0:agg", 0.99f), + new QuantilePostAggregator("a3", "a3:agg", 0.50f) + )) + .context(TIMESERIES_CONTEXT) + .build(), + Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) + ); + } + } +} diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaIndexingServiceTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaIndexingServiceTest.java index 1f5c11d4c531..e3580da9cff2 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaIndexingServiceTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITKafkaIndexingServiceTest.java @@ -21,7 +21,6 @@ import com.google.common.base.Throwables; import com.google.inject.Inject; - import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import io.druid.testing.IntegrationTestingConfig; @@ -56,12 +55,12 @@ public class ITKafkaIndexingServiceTest extends AbstractIndexerTest { private static final Logger LOG = new Logger(ITKafkaIndexingServiceTest.class); - private static final int DELAY_BETWEEN_EVENTS_SECS = 5; private static final String INDEXER_FILE = "/indexer/kafka_supervisor_spec.json"; private static final String QUERIES_FILE = "/indexer/kafka_index_queries.json"; private static final String DATASOURCE = "kafka_indexing_service_test"; private static final String TOPIC_NAME = "kafka_indexing_service_topic"; - private static final int MINUTES_TO_SEND = 4; + private static final int NUM_EVENTS_TO_SEND = 60; + private static final long WAIT_TIME_MILLIS = 2 * 60 * 1000L; // We'll fill in the current time and numbers for added, deleted and changed // before sending the event. @@ -163,15 +162,13 @@ public void testKafka() DateTime dt = new DateTime(zone); // timestamp to put on events dtFirst = dt; // timestamp of 1st event dtLast = dt; // timestamp of last event - // stop sending events when time passes this - DateTime dtStop = dtFirst.plusMinutes(MINUTES_TO_SEND).plusSeconds(30); // these are used to compute the expected aggregations int added = 0; int num_events = 0; // send data to kafka - while (dt.compareTo(dtStop) < 0) { // as long as we're within the time span + while (num_events < NUM_EVENTS_TO_SEND) { num_events++; added += num_events; // construct the event to send @@ -184,16 +181,22 @@ public void testKafka() throw Throwables.propagate(ioe); } - try { - Thread.sleep(DELAY_BETWEEN_EVENTS_SECS * 1000); - } - catch (InterruptedException ex) { /* nothing */ } dtLast = dt; dt = new DateTime(zone); } producer.close(); + LOG.info("Waiting for [%s] millis for Kafka indexing tasks to consume events", WAIT_TIME_MILLIS); + try { + Thread.sleep(WAIT_TIME_MILLIS); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + + InputStream is = ITKafkaIndexingServiceTest.class.getResourceAsStream(QUERIES_FILE); if (null == is) { throw new ISE("could not open query file: %s", QUERIES_FILE); @@ -214,7 +217,7 @@ public void testKafka() .replace("%%TIMEBOUNDARY_RESPONSE_MAXTIME%%", TIMESTAMP_FMT.print(dtLast)) .replace("%%TIMEBOUNDARY_RESPONSE_MINTIME%%", TIMESTAMP_FMT.print(dtFirst)) .replace("%%TIMESERIES_QUERY_START%%", INTERVAL_FMT.print(dtFirst)) - .replace("%%TIMESERIES_QUERY_END%%", INTERVAL_FMT.print(dtFirst.plusMinutes(MINUTES_TO_SEND + 2))) + .replace("%%TIMESERIES_QUERY_END%%", INTERVAL_FMT.print(dtLast.plusMinutes(2))) .replace("%%TIMESERIES_RESPONSE_TIMESTAMP%%", TIMESTAMP_FMT.print(dtFirst)) .replace("%%TIMESERIES_ADDED%%", Integer.toString(added)) .replace("%%TIMESERIES_NUMEVENTS%%", Integer.toString(num_events)); @@ -227,8 +230,23 @@ public void testKafka() throw Throwables.propagate(e); } + LOG.info("Shutting down Kafka Supervisor"); indexer.shutdownSupervisor(supervisorId); + // wait for all kafka indexing tasks to finish + LOG.info("Waiting for all kafka indexing tasks to finish"); + RetryUtil.retryUntilTrue( + new Callable() + { + @Override + public Boolean call() throws Exception + { + return (indexer.getPendingTasks().size() + indexer.getRunningTasks().size() + indexer.getWaitingTasks() + .size()) == 0; + } + }, "Waiting for Tasks Completion" + ); + // wait for segments to be handed off try { RetryUtil.retryUntil( diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java index 67baf76104b9..8d45083b1556 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java @@ -20,8 +20,6 @@ package io.druid.java.util.common.guava; import com.google.common.base.Supplier; -import com.google.common.base.Throwables; -import io.druid.java.util.common.logger.Logger; import java.io.Closeable; import java.io.IOException; @@ -31,30 +29,9 @@ */ public class BaseSequence> implements Sequence { - private static final Logger log = new Logger(BaseSequence.class); private final IteratorMaker maker; - public static Sequence simple(final Iterable iterable) - { - return new BaseSequence<>( - new BaseSequence.IteratorMaker>() - { - @Override - public Iterator make() - { - return iterable.iterator(); - } - - @Override - public void cleanup(Iterator iterFromMake) - { - - } - } - ); - } - public BaseSequence( IteratorMaker maker ) @@ -85,11 +62,18 @@ private OutType accumulate(IterType iterator, OutType initValue, final while (iterator.hasNext()) { initValue = fn.accumulate(initValue, iterator.next()); } - return initValue; } - finally { - maker.cleanup(iterator); + catch (Throwable t) { + try { + maker.cleanup(iterator); + } + catch (Exception e) { + t.addSuppressed(e); + } + throw t; } + maker.cleanup(iterator); + return initValue; } @Override @@ -100,16 +84,14 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat try { return makeYielder(initValue, accumulator, iterator); } - catch (Exception e) { - // We caught an Exception instead of returning a really, real, live, real boy, errr, iterator - // So we better try to close our stuff, 'cause the exception is what is making it out of here. + catch (Throwable t) { try { maker.cleanup(iterator); } - catch (RuntimeException e1) { - log.error(e1, "Exception thrown when closing maker. Logging and ignoring."); + catch (Exception e) { + t.addSuppressed(e); } - throw Throwables.propagate(e); + throw t; } } @@ -125,16 +107,14 @@ public Yielder toYielder( // after making the iterator return makeYielder(initValue.get(), accumulator, iterator); } - catch (Exception e) { - // We caught an Exception instead of returning a really, real, live, real boy, errr, iterator - // So we better try to close our stuff, 'cause the exception is what is making it out of here. + catch (Throwable t) { try { maker.cleanup(iterator); } - catch (RuntimeException e1) { - log.error(e1, "Exception thrown when closing maker. Logging and ignoring."); + catch (Exception e) { + t.addSuppressed(e); } - throw Throwables.propagate(e); + throw t; } } @@ -179,16 +159,14 @@ public Yielder next(OutType initValue) try { return makeYielder(initValue, accumulator, iter); } - catch (Exception e) { - // We caught an Exception instead of returning a really, real, live, real boy, errr, iterator - // So we better try to close our stuff, 'cause the exception is what is making it out of here. + catch (Throwable t) { try { maker.cleanup(iter); } - catch (RuntimeException e1) { - log.error(e1, "Exception thrown when closing maker. Logging and ignoring."); + catch (Exception e) { + t.addSuppressed(e); } - throw Throwables.propagate(e); + throw t; } } @@ -206,10 +184,10 @@ public void close() throws IOException }; } - public static interface IteratorMaker> + public interface IteratorMaker> { - public IterType make(); + IterType make(); - public void cleanup(IterType iterFromMake); + void cleanup(IterType iterFromMake); } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java index 0ed08a8b311d..6eefb185f3f7 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java @@ -23,6 +23,7 @@ import com.google.common.base.Suppliers; import com.google.common.base.Throwables; +import java.io.Closeable; import java.io.IOException; /** @@ -100,11 +101,14 @@ public Sequence accumulate(Sequence accumulated, Sequence in) try { return makeYielder(yielderYielder, initValue, accumulator); } - catch (RuntimeException e) { - // We caught a RuntimeException instead of returning a really, real, live, real boy, errr, iterator - // So we better try to close our stuff, 'cause the exception is what is making it out of here. - CloseQuietly.close(yielderYielder); - throw e; + catch (Throwable t) { + try { + yielderYielder.close(); + } + catch (Exception e) { + t.addSuppressed(e); + } + throw t; } } @@ -195,8 +199,9 @@ public boolean isDone() @Override public void close() throws IOException { - yielder.close(); - yielderYielder.close(); + try (Closeable toClose = yielderYielder) { + yielder.close(); + } } }; } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/ExecuteWhenDoneYielder.java b/java-util/src/main/java/io/druid/java/util/common/guava/ExecuteWhenDoneYielder.java index 4b1adf8512d5..2b8033c3a914 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/ExecuteWhenDoneYielder.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/ExecuteWhenDoneYielder.java @@ -56,9 +56,28 @@ public boolean isDone() @Override public void close() throws IOException { - if (isDone()) { - executor.execute(runnable); + boolean done = isDone(); + Throwable thrown = null; + try { + baseYielder.close(); + } + catch (Throwable t) { + thrown = t; + throw t; + } + finally { + if (done) { + if (thrown != null) { + try { + executor.execute(runnable); + } + catch (Throwable t) { + thrown.addSuppressed(t); + } + } else { + executor.execute(runnable); + } + } } - baseYielder.close(); } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/ExecutorExecutingSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/ExecutorExecutingSequence.java deleted file mode 100644 index cd4d75e3b874..000000000000 --- a/java-util/src/main/java/io/druid/java/util/common/guava/ExecutorExecutingSequence.java +++ /dev/null @@ -1,197 +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.guava; - -import com.google.common.base.Supplier; -import com.google.common.base.Throwables; - -import java.io.IOException; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; - -/** - */ -public class ExecutorExecutingSequence implements Sequence -{ - private final Sequence sequence; - private final ExecutorService exec; - - public ExecutorExecutingSequence( - Sequence sequence, - ExecutorService exec - ) - { - this.sequence = sequence; - this.exec = exec; - } - - @Override - public OutType accumulate(final OutType initValue, final Accumulator accumulator) - { - Future future = exec.submit( - new Callable() - { - @Override - public OutType call() throws Exception - { - return sequence.accumulate(initValue, accumulator); - } - } - ); - try { - return future.get(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - catch (ExecutionException e) { - throw Throwables.propagate(e); - } - } - - @Override - public OutType accumulate( - final Supplier initValue, final Accumulator accumulator - ) - { - Future future = exec.submit( - new Callable() - { - @Override - public OutType call() throws Exception - { - return sequence.accumulate(initValue, accumulator); - } - } - ); - try { - return future.get(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - catch (ExecutionException e) { - throw Throwables.propagate(e); - } - } - - @Override - public Yielder toYielder( - final OutType initValue, - final YieldingAccumulator accumulator - ) - { - Future> future = exec.submit( - new Callable>() - { - @Override - public Yielder call() throws Exception - { - return makeYielder(sequence.toYielder(initValue, accumulator)); - } - } - ); - try { - return future.get(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - catch (ExecutionException e) { - throw Throwables.propagate(e); - } - } - - @Override - public Yielder toYielder( - final Supplier initValue, final YieldingAccumulator accumulator - ) - { - Future> future = exec.submit( - new Callable>() - { - @Override - public Yielder call() throws Exception - { - return makeYielder(sequence.toYielder(initValue, accumulator)); - } - } - ); - try { - return future.get(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - catch (ExecutionException e) { - throw Throwables.propagate(e); - } - } - - private Yielder makeYielder(final Yielder yielder) - { - return new Yielder() - { - @Override - public OutType get() - { - return yielder.get(); - } - - @Override - public Yielder next(final OutType initValue) - { - Future> future = exec.submit( - new Callable>() - { - @Override - public Yielder call() throws Exception - { - return makeYielder(yielder.next(initValue)); - } - } - ); - try { - return future.get(); - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - catch (ExecutionException e) { - throw Throwables.propagate(e); - } - } - - @Override - public boolean isDone() - { - return yielder.isDone(); - } - - @Override - public void close() throws IOException - { - yielder.close(); - } - }; - } -} diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java index 0df526526260..5706c6f053be 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java @@ -28,12 +28,12 @@ * Limits the number of inputs from this sequence. For example, if there are actually 100 things in the sequence * but the limit is set to 10, the Sequence will act as if it only had 10 things. */ -public class LimitedSequence extends YieldingSequenceBase +final class LimitedSequence extends YieldingSequenceBase { private final Sequence baseSequence; private final int limit; - public LimitedSequence( + LimitedSequence( Sequence baseSequence, int limit ) @@ -72,7 +72,7 @@ private class LimitedYielder implements Yielder private final Yielder subYielder; private final LimitedYieldingAccumulator limitedAccumulator; - public LimitedYielder( + LimitedYielder( Yielder subYielder, LimitedYieldingAccumulator limitedAccumulator ) @@ -123,7 +123,7 @@ private class LimitedYieldingAccumulator extends DelegatingYieldingA int count; boolean interruptYield = false; - public LimitedYieldingAccumulator(YieldingAccumulator accumulator) + LimitedYieldingAccumulator(YieldingAccumulator accumulator) { super(accumulator); count = 0; @@ -151,7 +151,7 @@ public OutType accumulate(OutType accumulated, T in) return retVal; } - public boolean isInterruptYield() + boolean isInterruptYield() { return interruptYield; } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java index b248b264d01b..36b66e454804 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java @@ -23,6 +23,7 @@ import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Ordering; +import com.google.common.io.Closer; import java.io.IOException; import java.util.PriorityQueue; @@ -166,9 +167,11 @@ public boolean isDone() @Override public void close() throws IOException { + Closer closer = Closer.create(); while (!pQueue.isEmpty()) { - pQueue.remove().close(); + closer.register(pQueue.remove()); } + closer.close(); } }; } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/ResourceClosingSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/ResourceClosingSequence.java deleted file mode 100644 index d299bd5aad1c..000000000000 --- a/java-util/src/main/java/io/druid/java/util/common/guava/ResourceClosingSequence.java +++ /dev/null @@ -1,96 +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.guava; - -import com.google.common.base.Supplier; - -import java.io.Closeable; - -/** - */ -public class ResourceClosingSequence implements Sequence -{ - private final Sequence baseSequence; - private final Closeable closeable; - - public ResourceClosingSequence(Sequence baseSequence, Closeable closeable) - { - this.baseSequence = baseSequence; - this.closeable = closeable; - } - - @Override - public OutType accumulate(OutType initValue, Accumulator accumulator) - { - try { - return baseSequence.accumulate(initValue, accumulator); - } - finally { - CloseQuietly.close(closeable); - } - } - - @Override - public OutType accumulate( - Supplier initValue, Accumulator accumulator - ) - { - try { - return baseSequence.accumulate(initValue, accumulator); - } - finally { - CloseQuietly.close(closeable); - } - } - - @Override - public Yielder toYielder( - OutType initValue, YieldingAccumulator accumulator - ) - { - final Yielder baseYielder; - try { - baseYielder = baseSequence.toYielder(initValue, accumulator); - } - catch (RuntimeException e) { - CloseQuietly.close(closeable); - throw e; - } - - return new ResourceClosingYielder<>(baseYielder, closeable); - } - - @Override - public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator - ) - { - final Yielder baseYielder; - try { - baseYielder = baseSequence.toYielder(initValue, accumulator); - } - catch (RuntimeException e) { - CloseQuietly.close(closeable); - throw e; - } - - return new ResourceClosingYielder<>(baseYielder, closeable); - } -} diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/SequenceWrapper.java b/java-util/src/main/java/io/druid/java/util/common/guava/SequenceWrapper.java new file mode 100644 index 000000000000..fb9b5c014f47 --- /dev/null +++ b/java-util/src/main/java/io/druid/java/util/common/guava/SequenceWrapper.java @@ -0,0 +1,73 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.java.util.common.guava; + +import com.google.common.base.Supplier; + +/** + * @see Sequences#wrap(Sequence, SequenceWrapper) + */ +public abstract class SequenceWrapper +{ + /** + * Executed before sequence processing, i. e. before {@link Sequence#accumulate} or {@link Sequence#toYielder} on the + * wrapped sequence. Default implementation does nothing. + */ + public void before() + { + // do nothing + } + + /** + * Wraps any bits of the wrapped sequence processing: {@link Sequence#accumulate} or {@link Sequence#toYielder} and + * {@link Yielder#next(Object)} on the wrapped yielder. Doesn't wrap {@link #before} and {@link #after}. + * + *

{@code sequenceProcessing.get()} must be called just once. Implementation of this method should look like + *

+   * ... do something
+   * try {
+   *   return sequenceProcessing.get();
+   * }
+   * finally {
+   *   ... do something else
+   * }
+   * 
+ */ + public RetType wrap(Supplier sequenceProcessing) throws Exception + { + return sequenceProcessing.get(); + } + + /** + * Executed after sequence processing, i. e. after {@link Sequence#accumulate} on the wrapped sequence or after {@link + * Yielder#close()} on the wrapped yielder, or if exception was thrown from any method called on the wrapped sequence + * or yielder, or from {@link #before}, or from {@link #wrap} methods of this SequenceWrapper. + * + *

Even if {@code thrown} is not null, implementation of this method shouldn't rethrow it, it is done outside. + * + * @param isDone true if all elements in the sequence were processed and no exception was thrown, false otherwise + * @param thrown an exception thrown from any method called on the wrapped sequence or yielder, or from {@link + * #before()}, or from {@link #wrap} methods of this SequenceWrapper, or null if no exception was thrown. + */ + public void after(boolean isDone, Throwable thrown) throws Exception + { + // do nothing + } +} diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java index 8923efeef293..a9da58ce25df 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java @@ -20,6 +20,7 @@ package io.druid.java.util.common.guava; import com.google.common.base.Function; +import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.collect.Lists; @@ -28,6 +29,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.Comparator; +import java.util.Iterator; import java.util.List; import java.util.concurrent.Executor; @@ -40,7 +42,22 @@ public class Sequences public static Sequence simple(final Iterable iterable) { - return BaseSequence.simple(iterable); + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + return iterable.iterator(); + } + + @Override + public void cleanup(Iterator iterFromMake) + { + + } + } + ); } @SuppressWarnings("unchecked") @@ -79,33 +96,40 @@ public static Sequence limit(final Sequence sequence, final int limit) return new LimitedSequence<>(sequence, limit); } - public static Sequence withBaggage(final Sequence seq, Closeable baggage) + public static Sequence withBaggage(final Sequence seq, final Closeable baggage) { - return new ResourceClosingSequence<>(seq, baggage); - } - - public static Sequence withEffect(final Sequence seq, final Runnable effect, final Executor exec) - { - return new Sequence() + Preconditions.checkNotNull(baggage, "baggage"); + return wrap(seq, new SequenceWrapper() { @Override - public OutType accumulate(OutType initValue, Accumulator accumulator) + public void after(boolean isDone, Throwable thrown) throws Exception { - final OutType out = seq.accumulate(initValue, accumulator); - exec.execute(effect); - return out; + baggage.close(); } + }); + } - @Override - public OutType accumulate( - Supplier initValue, Accumulator accumulator - ) - { - final OutType out = seq.accumulate(initValue, accumulator); - exec.execute(effect); - return out; - } + /** + * Allows to execute something before, after or around the processing of the given sequence. See documentation to + * {@link SequenceWrapper} methods for some details. + */ + public static Sequence wrap(Sequence seq, SequenceWrapper wrapper) + { + Preconditions.checkNotNull(seq, "seq"); + Preconditions.checkNotNull(wrapper, "wrapper"); + return new WrappingSequence<>(seq, wrapper); + } + public static Sequence withEffect(final Sequence seq, final Runnable effect, final Executor exec) + { + // Uses YieldingSequenceBase to be able to execute the effect if all elements of the wrapped seq are processed + // (i. e. it "is done"), but the yielder of the underlying seq throws some exception from close(). This logic could + // be found in ExecuteWhenDoneYielder.close(). If accumulate() is implemented manually in this anonymous class + // instead of extending YieldingSequenceBase, it's not possible to distinguish exception thrown during elements + // processing in accumulate() of the underlying seq, from exception thrown after all elements are processed, + // in close(). + return new YieldingSequenceBase() + { @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { @@ -127,7 +151,7 @@ public static Sequence sort(final Sequence sequence, final Comparator< { List seqList = Sequences.toList(sequence, Lists.newArrayList()); Collections.sort(seqList, comparator); - return BaseSequence.simple(seqList); + return simple(seqList); } public static > ListType toList(Sequence seq, ListType list) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/WrappingSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/WrappingSequence.java new file mode 100644 index 000000000000..810c525e8cfd --- /dev/null +++ b/java-util/src/main/java/io/druid/java/util/common/guava/WrappingSequence.java @@ -0,0 +1,119 @@ +/* + * 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.guava; + +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.base.Throwables; + +/** + */ +final class WrappingSequence implements Sequence +{ + private final Sequence baseSequence; + private final SequenceWrapper wrapper; + + WrappingSequence(Sequence baseSequence, SequenceWrapper wrapper) + { + this.baseSequence = Preconditions.checkNotNull(baseSequence, "baseSequence"); + this.wrapper = Preconditions.checkNotNull(wrapper, "wrapper"); + } + + @Override + public OutType accumulate(final OutType outType, final Accumulator accumulator) + { + return accumulate(Suppliers.ofInstance(outType), accumulator); + } + + @Override + public OutType accumulate( + final Supplier initValSupplier, final Accumulator accumulator + ) + { + OutType result; + try { + wrapper.before(); + result = wrapper.wrap(new Supplier() + { + @Override + public OutType get() + { + return baseSequence.accumulate(initValSupplier, accumulator); + } + }); + } + catch (Throwable t) { + // Close on failure + try { + wrapper.after(false, t); + } + catch (Exception e) { + t.addSuppressed(e); + } + throw Throwables.propagate(t); + } + // "Normal" close + try { + wrapper.after(true, null); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + return result; + } + + @Override + public Yielder toYielder( + final OutType initValue, + final YieldingAccumulator accumulator + ) + { + return toYielder(Suppliers.ofInstance(initValue), accumulator); + } + + @Override + public Yielder toYielder( + final Supplier initValSupplier, final YieldingAccumulator accumulator + ) + { + try { + wrapper.before(); + return wrapper.wrap(new Supplier>() + { + @Override + public Yielder get() + { + return new WrappingYielder<>(baseSequence.toYielder(initValSupplier, accumulator), wrapper); + } + }); + } + catch (Throwable t) { + // Close on failure + try { + wrapper.after(false, t); + } + catch (Exception e) { + t.addSuppressed(e); + } + throw Throwables.propagate(t); + } + } +} diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/WrappingYielder.java b/java-util/src/main/java/io/druid/java/util/common/guava/WrappingYielder.java new file mode 100644 index 000000000000..93015e865e5d --- /dev/null +++ b/java-util/src/main/java/io/druid/java/util/common/guava/WrappingYielder.java @@ -0,0 +1,103 @@ +/* + * 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.guava; + +import com.google.common.base.Supplier; +import com.google.common.base.Throwables; + +import java.io.IOException; + +final class WrappingYielder implements Yielder +{ + private final Yielder baseYielder; + private final SequenceWrapper wrapper; + + WrappingYielder(Yielder baseYielder, SequenceWrapper wrapper) + { + this.baseYielder = baseYielder; + this.wrapper = wrapper; + } + + @Override + public OutType get() + { + return baseYielder.get(); + } + + @Override + public Yielder next(final OutType initValue) + { + try { + return wrapper.wrap(new Supplier>() + { + @Override + public Yielder get() + { + return new WrappingYielder<>(baseYielder.next(initValue), wrapper); + } + }); + } + catch (Throwable t) { + // Close on failure + try { + wrapper.after(false, t); + } + catch (Exception e) { + t.addSuppressed(e); + } + throw Throwables.propagate(t); + } + } + + @Override + public boolean isDone() + { + return baseYielder.isDone(); + } + + @Override + public void close() throws IOException + { + boolean isDone; + try { + isDone = isDone(); + baseYielder.close(); + } + catch (Throwable t) { + // Close on failure + try { + wrapper.after(false, t); + } + catch (Exception e) { + t.addSuppressed(e); + } + Throwables.propagateIfInstanceOf(t, IOException.class); + throw Throwables.propagate(t); + } + // "Normal" close + try { + wrapper.after(isDone, null); + } + catch (Exception e) { + Throwables.propagateIfInstanceOf(e, IOException.class); + throw Throwables.propagate(e); + } + } +} diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Yielders.java b/java-util/src/main/java/io/druid/java/util/common/guava/Yielders.java index 31f30c5037d4..e391b2186184 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Yielders.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Yielders.java @@ -19,16 +19,31 @@ package io.druid.java.util.common.guava; -import com.google.common.io.Closeables; +import com.google.common.base.Throwables; -import java.io.Closeable; import java.io.IOException; /** */ public class Yielders { - public static Yielder done(final T finalVal, final Closeable closeable) + public static Yielder each(final Sequence sequence) + { + return sequence.toYielder( + (T) null, + new YieldingAccumulator() + { + @Override + public T accumulate(T accumulated, T in) + { + yield(); + return in; + } + } + ); + } + + public static Yielder done(final T finalVal, final AutoCloseable closeable) { return new Yielder() { @@ -53,7 +68,15 @@ public boolean isDone() @Override public void close() throws IOException { - Closeables.close(closeable, false); + if (closeable != null) { + try { + closeable.close(); + } + catch (Exception e) { + Throwables.propagateIfInstanceOf(e, IOException.class); + throw Throwables.propagate(e); + } + } } }; } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java b/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java index 3e9174d58362..c02c692e505e 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java @@ -20,6 +20,7 @@ package io.druid.java.util.common.guava; import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; /** * A Sequence that is based entirely on the Yielder implementation. @@ -31,14 +32,7 @@ public abstract class YieldingSequenceBase implements Sequence @Override public OutType accumulate(OutType initValue, Accumulator accumulator) { - Yielder yielder = toYielder(initValue, YieldingAccumulators.fromAccumulator(accumulator)); - - try { - return yielder.get(); - } - finally { - CloseQuietly.close(yielder); - } + return accumulate(Suppliers.ofInstance(initValue), accumulator); } @Override diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/BaseSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/BaseSequenceTest.java index fbf06db6adda..6434657cefb8 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/BaseSequenceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/BaseSequenceTest.java @@ -34,14 +34,14 @@ public class BaseSequenceTest public void testSanity() throws Exception { final List vals = Arrays.asList(1, 2, 3, 4, 5); - SequenceTestHelper.testAll(BaseSequence.simple(vals), vals); + SequenceTestHelper.testAll(Sequences.simple(vals), vals); } @Test public void testNothing() throws Exception { final List vals = Arrays.asList(); - SequenceTestHelper.testAll(BaseSequence.simple(vals), vals); + SequenceTestHelper.testAll(Sequences.simple(vals), vals); } @Test diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/ExecutorExecutingSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/ExecutorExecutingSequenceTest.java deleted file mode 100644 index 18382d43790c..000000000000 --- a/java-util/src/test/java/io/druid/java/util/common/guava/ExecutorExecutingSequenceTest.java +++ /dev/null @@ -1,173 +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.guava; - -import com.google.common.base.Throwables; -import com.google.common.util.concurrent.Futures; -import junit.framework.Assert; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Collection; -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; - -/** - */ -public class ExecutorExecutingSequenceTest -{ - @Test - public void testSanity() throws Exception - { - TestExecutor exec = new TestExecutor(); - final List vals = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13); - ExecutorExecutingSequence seq = new ExecutorExecutingSequence<>(Sequences.simple(vals), exec); - - SequenceTestHelper.testAccumulation("", seq, vals); - Assert.assertEquals(1, exec.getTimesCalled()); - - exec.reset(); - - SequenceTestHelper.testYield("", 3, seq, vals); - Assert.assertEquals(5, exec.getTimesCalled()); - } - - @Test - public void testSanity2() throws Exception - { - TestExecutor exec = new TestExecutor(); - final List vals = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15); - ExecutorExecutingSequence seq = new ExecutorExecutingSequence<>(Sequences.simple(vals), exec); - - SequenceTestHelper.testAccumulation("", seq, vals); - Assert.assertEquals(1, exec.getTimesCalled()); - - exec.reset(); - - SequenceTestHelper.testYield("", 3, seq, vals); - Assert.assertEquals(6, exec.getTimesCalled()); - } - - public static class TestExecutor implements ExecutorService - { - int timesCalled = 0; - - public int getTimesCalled() - { - return timesCalled; - } - - public void reset() - { - timesCalled = 0; - } - - @Override - public void shutdown() - { - throw new UnsupportedOperationException(); - } - - @Override - public List shutdownNow() - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isShutdown() - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isTerminated() - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException - { - throw new UnsupportedOperationException(); - } - - @Override - public Future submit(Callable task) - { - ++timesCalled; - try { - return Futures.immediateCheckedFuture(task.call()); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - - @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/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java b/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java index 67481f90f7b6..09a0761e341f 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java @@ -21,12 +21,12 @@ import com.google.common.base.Supplier; +import java.io.Closeable; import java.util.Arrays; -import java.util.Iterator; import java.util.concurrent.atomic.AtomicBoolean; /** -*/ + */ public class TestSequence implements Sequence { public static TestSequence create(Iterable iterable) @@ -44,21 +44,17 @@ public static TestSequence create(T... vals) public TestSequence(final Iterable iterable) { - base = new BaseSequence<>( - new BaseSequence.IteratorMaker>() - { - @Override - public Iterator make() - { - return iterable.iterator(); - } - - @Override - public void cleanup(Iterator iterFromMake) - { - closed.set(true); - } - }); + base = Sequences.withBaggage( + Sequences.simple(iterable), + new Closeable() + { + @Override + public void close() + { + closed.set(true); + } + } + ); } @Override diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java new file mode 100644 index 000000000000..3f91b6bc7489 --- /dev/null +++ b/java-util/src/test/java/io/druid/java/util/common/guava/WithEffectSequenceTest.java @@ -0,0 +1,109 @@ +/* + * 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.guava; + +import com.google.common.util.concurrent.MoreExecutors; +import org.junit.Assert; +import org.junit.Test; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +public class WithEffectSequenceTest +{ + @Test + public void testConsistentEffectApplicationOrder() + { + final AtomicInteger effect1 = new AtomicInteger(); + final AtomicInteger effect2 = new AtomicInteger(); + final AtomicInteger counter = new AtomicInteger(); + + Sequence sequence = Sequences.withEffect( + Sequences.withEffect( + Sequences.simple(Arrays.asList(1, 2, 3)), + new Runnable() + { + @Override + public void run() + { + effect1.set(counter.incrementAndGet()); + } + }, + MoreExecutors.sameThreadExecutor() + ), + new Runnable() + { + @Override + public void run() + { + effect2.set(counter.incrementAndGet()); + } + }, + MoreExecutors.sameThreadExecutor() + ); + // Run sequence via accumulate + Sequences.toList(sequence, new ArrayList()); + Assert.assertEquals(1, effect1.get()); + Assert.assertEquals(2, effect2.get()); + + // Ensure sequence runs via Yielder, because LimitedSequence extends YieldingSequenceBase which + // implements accumulate() via yielder(). + // "Limiting" a sequence of 3 elements with 4 to let effects be executed. If e. g. limit with 1 or 2, effects are + // not executed. + Sequence yieldingSequence = Sequences.limit(sequence, 4); + Sequences.toList(yieldingSequence, new ArrayList()); + Assert.assertEquals(3, effect1.get()); + Assert.assertEquals(4, effect2.get()); + } + + @Test + public void testEffectExecutedIfWrappedSequenceThrowsExceptionFromClose() { + Sequence baseSeq = Sequences.simple(Arrays.asList(1, 2, 3)); + Sequence throwingSeq = Sequences.withBaggage(baseSeq, new Closeable() + { + @Override + public void close() throws IOException + { + throw new RuntimeException(); + + } + }); + final AtomicBoolean effectExecuted = new AtomicBoolean(); + Sequence seqWithEffect = Sequences.withEffect(throwingSeq, new Runnable() + { + @Override + public void run() + { + effectExecuted.set(true); + } + }, MoreExecutors.sameThreadExecutor()); + try { + Sequences.toList(seqWithEffect, new ArrayList()); + Assert.fail("expected RuntimeException"); + } catch (RuntimeException e) { + // expected + Assert.assertTrue(effectExecuted.get()); + } + } +} diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/ResourceClosingSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/WrappingSequenceTest.java similarity index 55% rename from java-util/src/test/java/io/druid/java/util/common/guava/ResourceClosingSequenceTest.java rename to java-util/src/test/java/io/druid/java/util/common/guava/WrappingSequenceTest.java index c535881f8bbe..d6bec5d28ba4 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/ResourceClosingSequenceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/WrappingSequenceTest.java @@ -24,13 +24,14 @@ import java.io.Closeable; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; /** */ -public class ResourceClosingSequenceTest +public class WrappingSequenceTest { @Test public void testSanity() throws Exception @@ -54,4 +55,44 @@ public void close() throws IOException closedCounter.set(0); SequenceTestHelper.testClosed(closedCounter, Sequences.withBaggage(new UnsupportedSequence(), closeable)); } + + @Test + public void testConsistentCloseOrder() + { + final AtomicInteger closed1 = new AtomicInteger(); + final AtomicInteger closed2 = new AtomicInteger(); + final AtomicInteger counter = new AtomicInteger(); + + Sequence sequence = Sequences.withBaggage( + Sequences.withBaggage( + Sequences.simple(Arrays.asList(1, 2, 3)), + new Closeable() + { + @Override + public void close() throws IOException + { + closed1.set(counter.incrementAndGet()); + } + } + ), + new Closeable() + { + @Override + public void close() throws IOException + { + closed2.set(counter.incrementAndGet()); + } + } + ); + // Run sequence via accumulate + Sequences.toList(sequence, new ArrayList()); + Assert.assertEquals(1, closed1.get()); + Assert.assertEquals(2, closed2.get()); + + // Ensure sequence runs via Yielder, because LimitedSequence extends YieldingSequenceBase + Sequence yieldingSequence = Sequences.limit(sequence, 1); + Sequences.toList(yieldingSequence, new ArrayList()); + Assert.assertEquals(3, closed1.get()); + Assert.assertEquals(4, closed2.get()); + } } diff --git a/pom.xml b/pom.xml index 40ce85499255..be94b9c89e7a 100644 --- a/pom.xml +++ b/pom.xml @@ -123,6 +123,8 @@ extensions-contrib/virtual-columns extensions-contrib/thrift-extensions extensions-contrib/ambari-metrics-emitter + extensions-contrib/scan-query + extensions-contrib/sqlserver-metadata-storage @@ -200,6 +202,13 @@ io.airlift airline 0.7 + + + + com.google.code.findbugs + annotations + + org.skife.config @@ -807,6 +816,30 @@ + + org.apache.maven.plugins + maven-enforcer-plugin + 1.4.1 + + + enforce-banned-dependencies + + enforce + + + + + + + com.google.code.findbugs:annotations + + + + true + + + + diff --git a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java index 5337dd10e354..805c9b2e25ea 100644 --- a/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java +++ b/processing/src/main/java/io/druid/query/CPUTimeMetricQueryRunner.java @@ -21,20 +21,15 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; - import com.google.common.base.Supplier; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.common.utils.VMUtils; import io.druid.java.util.common.ISE; -import io.druid.java.util.common.guava.Accumulator; import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.SequenceWrapper; import io.druid.java.util.common.guava.Sequences; -import io.druid.java.util.common.guava.Yielder; -import io.druid.java.util.common.guava.YieldingAccumulator; -import java.io.Closeable; -import java.io.IOException; import java.util.Map; import java.util.concurrent.atomic.AtomicLong; @@ -71,120 +66,23 @@ public Sequence run( ) { final Sequence baseSequence = delegate.run(query, responseContext); - return Sequences.withBaggage( - new Sequence() + return Sequences.wrap( + baseSequence, + new SequenceWrapper() { @Override - public OutType accumulate(OutType initValue, Accumulator accumulator) - { - final long start = VMUtils.getCurrentThreadCpuTime(); - try { - return baseSequence.accumulate(initValue, accumulator); - } - finally { - cpuTimeAccumulator.addAndGet(VMUtils.getCurrentThreadCpuTime() - start); - } - } - - @Override - public OutType accumulate( - Supplier initValue, Accumulator accumulator - ) + public RetType wrap(Supplier sequenceProcessing) { final long start = VMUtils.getCurrentThreadCpuTime(); try { - return baseSequence.accumulate(initValue, accumulator); - } - finally { + return sequenceProcessing.get(); + } finally { cpuTimeAccumulator.addAndGet(VMUtils.getCurrentThreadCpuTime() - start); } } @Override - public Yielder toYielder( - final OutType initValue, - final YieldingAccumulator accumulator - ) - { - return toYielder(new Supplier>() - { - @Override - public Yielder get() - { - return baseSequence.toYielder(initValue, accumulator); - } - }); - } - - @Override - public Yielder toYielder( - final Supplier initValue, final YieldingAccumulator accumulator - ) - { - return toYielder(new Supplier>() - { - @Override - public Yielder get() - { - return baseSequence.toYielder(initValue, accumulator); - } - }); - } - - private Yielder toYielder(Supplier> supplier) - { - final long start = VMUtils.getCurrentThreadCpuTime(); - final Yielder delegateYielder; - try { - delegateYielder = supplier.get(); - } - finally { - cpuTimeAccumulator.addAndGet(VMUtils.getCurrentThreadCpuTime() - start); - } - return new Yielder() - { - @Override - public OutType get() - { - final long start = VMUtils.getCurrentThreadCpuTime(); - try { - return delegateYielder.get(); - } - finally { - cpuTimeAccumulator.addAndGet(VMUtils.getCurrentThreadCpuTime() - start); - } - } - - @Override - public Yielder next(OutType initValue) - { - final long start = VMUtils.getCurrentThreadCpuTime(); - try { - return delegateYielder.next(initValue); - } - finally { - cpuTimeAccumulator.addAndGet(VMUtils.getCurrentThreadCpuTime() - start); - } - } - - @Override - public boolean isDone() - { - return delegateYielder.isDone(); - } - - @Override - public void close() throws IOException - { - delegateYielder.close(); - } - }; - } - }, - new Closeable() - { - @Override - public void close() + public void after(boolean isDone, Throwable thrown) throws Exception { if (report) { final long cpuTime = cpuTimeAccumulator.get(); diff --git a/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java index f299c12ee864..d416589f199a 100644 --- a/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByMergedQueryRunner.java @@ -34,7 +34,6 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.guava.Accumulator; -import io.druid.java.util.common.guava.ResourceClosingSequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; @@ -154,7 +153,7 @@ public Void call() throws Exception return Sequences.simple(bySegmentAccumulatorPair.lhs); } - return new ResourceClosingSequence( + return Sequences.withBaggage( Sequences.simple( Iterables.transform( indexAccumulatorPair.lhs.iterableWithPostAggregations(null, query.isDescending()), diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index ba361d796ea2..c35547c051df 100644 --- a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java @@ -23,12 +23,11 @@ import com.google.common.base.Supplier; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import io.druid.java.util.common.guava.Accumulator; +import io.druid.java.util.common.guava.LazySequence; import io.druid.java.util.common.guava.Sequence; -import io.druid.java.util.common.guava.Yielder; -import io.druid.java.util.common.guava.YieldingAccumulator; +import io.druid.java.util.common.guava.SequenceWrapper; +import io.druid.java.util.common.guava.Sequences; -import java.io.IOException; import java.util.Map; /** @@ -91,173 +90,44 @@ public Sequence run(final Query query, final Map responseC builder.setDimension(userDimension.getKey(), userDimension.getValue()); } - return new Sequence() - { - @Override - public OutType accumulate(final OutType outType, final Accumulator accumulator) - { - return accumulate(new Supplier() + return Sequences.wrap( + // Use LazySequence because want to account execution time of queryRunner.run() (it prepares the underlying + // Sequence) as part of the reported query time, i. e. we want to execute queryRunner.run() after + // `startTime = System.currentTimeMillis();` (see below). + new LazySequence<>(new Supplier>() { @Override - public OutType get() + public Sequence get() { - return queryRunner.run(query, responseContext).accumulate(outType, accumulator); + return queryRunner.run(query, responseContext); } - }); - } - - @Override - public OutType accumulate( - final Supplier initValue, final Accumulator accumulator - ) - { - return accumulate(new Supplier() + }), + new SequenceWrapper() { - @Override - public OutType get() - { - return queryRunner.run(query, responseContext).accumulate(initValue, accumulator); - } - }); - } - - private OutType accumulate(Supplier retValSupplier) - { - OutType retVal; - - long startTime = System.currentTimeMillis(); - try { - retVal = retValSupplier.get(); - } - catch (RuntimeException e) { - builder.setDimension(DruidMetrics.STATUS, "failed"); - throw e; - } - catch (Error e) { - builder.setDimension(DruidMetrics.STATUS, "failed"); - throw e; - } - finally { - long timeTaken = System.currentTimeMillis() - startTime; - - emitter.emit(builder.build(metricName, timeTaken)); - - if (creationTime > 0) { - emitter.emit(builder.build("query/wait/time", startTime - creationTime)); - } - } - - return retVal; - } - - @Override - public Yielder toYielder( - final OutType initValue, - final YieldingAccumulator accumulator - ) - { - return toYielder(new Supplier>() - { - @Override - public Yielder get() - { - return queryRunner.run(query, responseContext).toYielder(initValue, accumulator); - } - }); - } - - @Override - public Yielder toYielder( - final Supplier initValue, final YieldingAccumulator accumulator - ) - { - return toYielder(new Supplier>() - { - @Override - public Yielder get() - { - return queryRunner.run(query, responseContext).toYielder(initValue, accumulator); - } - }); - } - - private Yielder toYielder(Supplier> supplier) - { - Yielder retVal; - - long startTime = System.currentTimeMillis(); - try { - retVal = supplier.get(); - } - catch (RuntimeException e) { - builder.setDimension(DruidMetrics.STATUS, "failed"); - throw e; - } - catch (Error e) { - builder.setDimension(DruidMetrics.STATUS, "failed"); - throw e; - } + private long startTime; - return makeYielder(startTime, retVal, builder); - } - - private Yielder makeYielder( - final long startTime, - final Yielder yielder, - final ServiceMetricEvent.Builder builder - ) - { - return new Yielder() - { @Override - public OutType get() + public void before() { - return yielder.get(); + startTime = System.currentTimeMillis(); } @Override - public Yielder next(OutType initValue) + public void after(boolean isDone, Throwable thrown) { - try { - return makeYielder(startTime, yielder.next(initValue), builder); - } - catch (RuntimeException e) { + if (thrown != null) { builder.setDimension(DruidMetrics.STATUS, "failed"); - throw e; + } else if (!isDone) { + builder.setDimension(DruidMetrics.STATUS, "short"); } - catch (Error e) { - builder.setDimension(DruidMetrics.STATUS, "failed"); - throw e; - } - } + long timeTaken = System.currentTimeMillis() - startTime; + emitter.emit(builder.build(metricName, timeTaken)); - @Override - public boolean isDone() - { - return yielder.isDone(); - } - - @Override - public void close() throws IOException - { - try { - if (!isDone() && builder.getDimension(DruidMetrics.STATUS) == null) { - builder.setDimension(DruidMetrics.STATUS, "short"); - } - - long timeTaken = System.currentTimeMillis() - startTime; - emitter.emit(builder.build(metricName, timeTaken)); - - if (creationTime > 0) { - emitter.emit(builder.build("query/wait/time", startTime - creationTime)); - } - } - finally { - yielder.close(); + if (creationTime > 0) { + emitter.emit(builder.build("query/wait/time", startTime - creationTime)); } } - }; - } - }; + } + ); } } diff --git a/processing/src/main/java/io/druid/query/QueryRunnerHelper.java b/processing/src/main/java/io/druid/query/QueryRunnerHelper.java index 0d1cd979d9c9..76842d684979 100644 --- a/processing/src/main/java/io/druid/query/QueryRunnerHelper.java +++ b/processing/src/main/java/io/druid/query/QueryRunnerHelper.java @@ -23,7 +23,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Predicates; import io.druid.granularity.QueryGranularity; -import io.druid.java.util.common.guava.ResourceClosingSequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; @@ -80,7 +79,7 @@ public static QueryRunner makeClosingQueryRunner(final QueryRunner ru @Override public Sequence run(Query query, Map responseContext) { - return new ResourceClosingSequence<>(runner.run(query, responseContext), closeable); + return Sequences.withBaggage(runner.run(query, responseContext), closeable); } }; } diff --git a/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java index 5d390deddc63..a85a6f911286 100644 --- a/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java @@ -20,8 +20,8 @@ package io.druid.query; import io.druid.java.util.common.guava.CloseQuietly; -import io.druid.java.util.common.guava.ResourceClosingSequence; import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Sequences; import io.druid.segment.ReferenceCountingSegment; import java.io.Closeable; @@ -54,7 +54,7 @@ public Sequence run(final Query query, Map responseContext try { final Sequence baseSequence = factory.createRunner(adapter).run(query, responseContext); - return new ResourceClosingSequence(baseSequence, closeable); + return Sequences.withBaggage(baseSequence, closeable); } catch (RuntimeException e) { CloseQuietly.close(closeable); 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 4a6974dcd2a2..052d2f512fcf 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 @@ -32,7 +32,6 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.BaseSequence; import io.druid.java.util.common.guava.CloseQuietly; -import io.druid.java.util.common.guava.ResourceClosingSequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.ColumnSelectorPlus; @@ -121,7 +120,7 @@ public static Sequence process( : new DateTime(Long.parseLong(fudgeTimestampString)); return Sequences.concat( - new ResourceClosingSequence<>( + Sequences.withBaggage( Sequences.map( cursors, new Function>() diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java index 5c37209b571f..1e37270b9b9e 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java @@ -176,15 +176,18 @@ public ByteBuffer get() final Accumulator, Row> accumulator = pair.rhs; closeOnFailure.add(grouper); - final Grouper retVal = filteredSequence.accumulate(new Supplier>() - { - @Override - public Grouper get() - { - grouper.init(); - return grouper; - } - }, accumulator); + final Grouper retVal = filteredSequence.accumulate( + new Supplier>() + { + @Override + public Grouper get() + { + grouper.init(); + return grouper; + } + }, + accumulator + ); if (retVal != grouper) { throw new ResourceLimitExceededException("Grouping resources exhausted"); } 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 f01365f2a3d3..794fe68d7e6b 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 @@ -33,7 +33,6 @@ import io.druid.data.input.Row; import io.druid.guice.annotations.Global; import io.druid.java.util.common.IAE; -import io.druid.java.util.common.guava.ResourceClosingSequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.GroupByMergedQueryRunner; @@ -120,7 +119,7 @@ public Sequence mergeResults( true ); - return new ResourceClosingSequence<>(query.applyLimit(GroupByQueryHelper.postAggregate(query, index)), index); + return Sequences.withBaggage(query.applyLimit(GroupByQueryHelper.postAggregate(query, index)), index); } @Override @@ -222,7 +221,7 @@ public Sequence apply(Interval interval) innerQueryResultIndex.close(); - return new ResourceClosingSequence<>( + return Sequences.withBaggage( outerQuery.applyLimit(GroupByQueryHelper.postAggregate(query, outerQueryResultIndex)), outerQueryResultIndex ); diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java index 411e5db119c6..41597aab2bb8 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryEngine.java @@ -64,7 +64,7 @@ public class SelectQueryEngine { private static final SelectStrategyFactory STRATEGY_FACTORY = new SelectStrategyFactory(); - private static class SelectStrategyFactory implements ColumnSelectorStrategyFactory + public static class SelectStrategyFactory implements ColumnSelectorStrategyFactory { @Override public SelectColumnSelectorStrategy makeColumnSelectorStrategy( @@ -202,23 +202,12 @@ public Result apply(Cursor cursor) int lastOffset = offset.startOffset(); for (; !cursor.isDone() && offset.hasNext(); cursor.advance(), offset.next()) { - final Map theEvent = Maps.newLinkedHashMap(); - theEvent.put(EventHolder.timestampKey, new DateTime(timestampColumnSelector.get())); - - for (ColumnSelectorPlus selectorPlus : selectorPlusList) { - selectorPlus.getColumnSelectorStrategy().addRowValuesToSelectResult(selectorPlus.getOutputName(), selectorPlus.getSelector(), theEvent); - } - - for (Map.Entry metSelector : metSelectors.entrySet()) { - final String metric = metSelector.getKey(); - final ObjectColumnSelector selector = metSelector.getValue(); - - if (selector == null) { - theEvent.put(metric, null); - } else { - theEvent.put(metric, selector.get()); - } - } + final Map theEvent = singleEvent( + EventHolder.timestampKey, + timestampColumnSelector, + selectorPlusList, + metSelectors + ); builder.addEntry( new EventHolder( @@ -236,4 +225,31 @@ public Result apply(Cursor cursor) } ); } + + public static Map singleEvent( + String timestampKey, + LongColumnSelector timestampColumnSelector, + List> selectorPlusList, + Map metSelectors + ) + { + final Map theEvent = Maps.newLinkedHashMap(); + theEvent.put(timestampKey, new DateTime(timestampColumnSelector.get())); + + for (ColumnSelectorPlus selectorPlus : selectorPlusList) { + selectorPlus.getColumnSelectorStrategy().addRowValuesToSelectResult(selectorPlus.getOutputName(), selectorPlus.getSelector(), theEvent); + } + + for (Map.Entry metSelector : metSelectors.entrySet()) { + final String metric = metSelector.getKey(); + final ObjectColumnSelector selector = metSelector.getValue(); + + if (selector == null) { + theEvent.put(metric, null); + } else { + theEvent.put(metric, selector.get()); + } + } + return theEvent; + } } diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java index 699ba7874293..ca48564da0b1 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -20,10 +20,12 @@ package io.druid.query.spec; import com.google.common.base.Supplier; -import com.google.common.base.Throwables; +import com.google.common.base.Suppliers; import com.google.common.collect.Lists; import io.druid.java.util.common.guava.Accumulator; import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.SequenceWrapper; +import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Yielder; import io.druid.java.util.common.guava.Yielders; import io.druid.java.util.common.guava.YieldingAccumulator; @@ -36,7 +38,6 @@ import java.io.IOException; import java.util.List; import java.util.Map; -import java.util.concurrent.Callable; /** */ @@ -64,37 +65,22 @@ public Sequence run(final Query input, final Map responseC final String newName = String.format("%s_%s_%s", query.getType(), query.getDataSource(), query.getIntervals()); final Sequence baseSequence = doNamed( - currThread, currThreadName, newName, new Callable>() + currThread, currThreadName, newName, new Supplier>() { @Override - public Sequence call() throws Exception + public Sequence get() { return base.run(query, responseContext); } } ); - return new Sequence() + Sequence segmentMissingCatchingSequence = new Sequence() { @Override public OutType accumulate(final OutType initValue, final Accumulator accumulator) { - return doItNamed( - new Callable() - { - @Override - public OutType call() throws Exception - { - try { - return baseSequence.accumulate(initValue, accumulator); - } - catch (SegmentMissingException e) { - appendMissingSegment(responseContext); - return initValue; - } - } - } - ); + return accumulate(Suppliers.ofInstance(initValue), accumulator); } @Override @@ -102,22 +88,13 @@ public OutType accumulate( final Supplier initValue, final Accumulator accumulator ) { - return doItNamed( - new Callable() - { - @Override - public OutType call() throws Exception - { - try { - return baseSequence.accumulate(initValue, accumulator); - } - catch (SegmentMissingException e) { - appendMissingSegment(responseContext); - return initValue.get(); - } - } - } - ); + try { + return baseSequence.accumulate(initValue, accumulator); + } + catch (SegmentMissingException e) { + appendMissingSegment(responseContext); + return initValue.get(); + } } @Override @@ -126,22 +103,7 @@ public Yielder toYielder( final YieldingAccumulator accumulator ) { - return doItNamed( - new Callable>() - { - @Override - public Yielder call() throws Exception - { - try { - return makeYielder(baseSequence.toYielder(initValue, accumulator)); - } - catch (SegmentMissingException e) { - appendMissingSegment(responseContext); - return Yielders.done(initValue, null); - } - } - } - ); + return toYielder(Suppliers.ofInstance(initValue), accumulator); } @Override @@ -149,22 +111,13 @@ public Yielder toYielder( final Supplier initValue, final YieldingAccumulator accumulator ) { - return doItNamed( - new Callable>() - { - @Override - public Yielder call() throws Exception - { - try { - return makeYielder(baseSequence.toYielder(initValue, accumulator)); - } - catch (SegmentMissingException e) { - appendMissingSegment(responseContext); - return Yielders.done(initValue.get(), null); - } - } - } - ); + try { + return makeYielder(baseSequence.toYielder(initValue, accumulator)); + } + catch (SegmentMissingException e) { + appendMissingSegment(responseContext); + return Yielders.done(initValue.get(), null); + } } private Yielder makeYielder(final Yielder yielder) @@ -180,16 +133,13 @@ public OutType get() @Override public Yielder next(final OutType initValue) { - return doItNamed( - new Callable>() - { - @Override - public Yielder call() throws Exception - { - return yielder.next(initValue); - } - } - ); + try { + return yielder.next(initValue); + } + catch (SegmentMissingException e) { + appendMissingSegment(responseContext); + return Yielders.done(initValue, null); + } } @Override @@ -205,12 +155,18 @@ public void close() throws IOException } }; } - - private RetType doItNamed(Callable toRun) - { - return doNamed(currThread, currThreadName, newName, toRun); - } }; + return Sequences.wrap( + segmentMissingCatchingSequence, + new SequenceWrapper() + { + @Override + public RetType wrap(Supplier sequenceProcessing) + { + return doNamed(currThread, currThreadName, newName, sequenceProcessing); + } + } + ); } private void appendMissingSegment(Map responseContext) @@ -223,14 +179,11 @@ private void appendMissingSegment(Map responseContext) missingSegments.add(specificSpec.getDescriptor()); } - private RetType doNamed(Thread currThread, String currName, String newName, Callable toRun) + private RetType doNamed(Thread currThread, String currName, String newName, Supplier toRun) { try { currThread.setName(newName); - return toRun.call(); - } - catch (Exception e) { - throw Throwables.propagate(e); + return toRun.get(); } finally { currThread.setName(currName); diff --git a/processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java b/processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java deleted file mode 100644 index 74823061aeda..000000000000 --- a/processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java +++ /dev/null @@ -1,61 +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.segment; - -import com.google.common.base.Supplier; -import io.druid.java.util.common.guava.ResourceClosingYielder; -import io.druid.java.util.common.guava.Sequence; -import io.druid.java.util.common.guava.Yielder; -import io.druid.java.util.common.guava.YieldingAccumulator; -import io.druid.java.util.common.guava.YieldingSequenceBase; - -import java.io.Closeable; - -/** - */ -public class ReferenceCountingSequence extends YieldingSequenceBase -{ - private final Sequence baseSequence; - private final ReferenceCountingSegment segment; - - public ReferenceCountingSequence(Sequence baseSequence, ReferenceCountingSegment segment) - { - this.baseSequence = baseSequence; - this.segment = segment; - } - - @Override - public Yielder toYielder( - OutType initValue, YieldingAccumulator accumulator - ) - { - final Closeable closeable = segment.increment(); - return new ResourceClosingYielder(baseSequence.toYielder(initValue, accumulator), closeable); - } - - @Override - public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator - ) - { - final Closeable closeable = segment.increment(); - return new ResourceClosingYielder(baseSequence.toYielder(initValue, accumulator), closeable); - } -} diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/IngestSegmentFirehose.java b/server/src/main/java/io/druid/segment/realtime/firehose/IngestSegmentFirehose.java index dfa163b8be34..f599c1806615 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/IngestSegmentFirehose.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/IngestSegmentFirehose.java @@ -23,7 +23,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; - import io.druid.data.input.Firehose; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; @@ -31,7 +30,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Yielder; -import io.druid.java.util.common.guava.YieldingAccumulator; +import io.druid.java.util.common.guava.Yielders; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.filter.DimFilter; import io.druid.query.select.EventHolder; @@ -171,18 +170,7 @@ public void remove() } ) ); - rowYielder = rows.toYielder( - (InputRow) null, - new YieldingAccumulator() - { - @Override - public InputRow accumulate(InputRow accumulated, InputRow in) - { - yield(); - return in; - } - } - ); + rowYielder = Yielders.each(rows); } @Override diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index fe7abba08c0b..2cbd1b361a4c 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -36,7 +36,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Yielder; -import io.druid.java.util.common.guava.YieldingAccumulator; +import io.druid.java.util.common.guava.Yielders; import io.druid.query.DruidMetrics; import io.druid.query.Query; import io.druid.query.QueryContextKeys; @@ -226,18 +226,7 @@ public Response doPost( results = res; } - final Yielder yielder = results.toYielder( - (Object) null, - new YieldingAccumulator() - { - @Override - public Object accumulate(Object accumulated, Object in) - { - yield(); - return in; - } - } - ); + final Yielder yielder = Yielders.each(results); try { final Query theQuery = query; @@ -250,38 +239,43 @@ public Object accumulate(Object accumulated, Object in) @Override public void write(OutputStream outputStream) throws IOException, WebApplicationException { - // json serializer will always close the yielder - CountingOutputStream os = new CountingOutputStream(outputStream); - jsonWriter.writeValue(os, yielder); - - os.flush(); // Some types of OutputStream suppress flush errors in the .close() method. - os.close(); - successfulQueryCount.incrementAndGet(); - final long queryTime = System.currentTimeMillis() - start; - emitter.emit( - DruidMetrics.makeQueryTimeMetric(theToolChest, jsonMapper, theQuery, req.getRemoteAddr()) - .setDimension("success", "true") - .build("query/time", queryTime) - ); - emitter.emit( - DruidMetrics.makeQueryTimeMetric(theToolChest, jsonMapper, theQuery, req.getRemoteAddr()) - .build("query/bytes", os.getCount()) - ); - - requestLogger.log( - new RequestLogLine( - new DateTime(start), - req.getRemoteAddr(), - theQuery, - new QueryStats( - ImmutableMap.of( - "query/time", queryTime, - "query/bytes", os.getCount(), - "success", true - ) - ) - ) - ); + try { + // json serializer will always close the yielder + CountingOutputStream os = new CountingOutputStream(outputStream); + jsonWriter.writeValue(os, yielder); + + os.flush(); // Some types of OutputStream suppress flush errors in the .close() method. + os.close(); + successfulQueryCount.incrementAndGet(); + final long queryTime = System.currentTimeMillis() - start; + emitter.emit( + DruidMetrics.makeQueryTimeMetric(theToolChest, jsonMapper, theQuery, req.getRemoteAddr()) + .setDimension("success", "true") + .build("query/time", queryTime) + ); + emitter.emit( + DruidMetrics.makeQueryTimeMetric(theToolChest, jsonMapper, theQuery, req.getRemoteAddr()) + .build("query/bytes", os.getCount()) + ); + + requestLogger.log( + new RequestLogLine( + new DateTime(start), + req.getRemoteAddr(), + theQuery, + new QueryStats( + ImmutableMap.of( + "query/time", queryTime, + "query/bytes", os.getCount(), + "success", true + ) + ) + ) + ); + } + finally { + Thread.currentThread().setName(currThreadName); + } } }, context.getContentType() diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java index bdba9f4981f4..50c693ad0f14 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java @@ -776,6 +776,7 @@ public ImmutableDruidServer apply(DruidServer input) if (!loadManagementPeons.containsKey(server.getName())) { String basePath = ZKPaths.makePath(zkPaths.getLoadQueuePath(), server.getName()); LoadQueuePeon loadQueuePeon = taskMaster.giveMePeon(basePath); + loadQueuePeon.start(); log.info("Creating LoadQueuePeon for server[%s] at path[%s]", server.getName(), basePath); loadManagementPeons.put(server.getName(), loadQueuePeon); diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java index 529bd06aba0f..1271af0ec751 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java @@ -80,4 +80,10 @@ public String getConsoleStatic() { return null; } + + @Config("druid.coordinator.loadqueuepeon.repeatDelay") + public Duration getLoadQueuePeonRepeatDelay() + { + return Duration.millis(50); + } } diff --git a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java index 8826513a9f55..3019284634c0 100644 --- a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java +++ b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java @@ -25,6 +25,7 @@ import com.metamx.emitter.EmittingLogger; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.concurrent.ScheduledExecutors; import io.druid.server.coordination.DataSegmentChangeRequest; import io.druid.server.coordination.SegmentChangeRequestDrop; import io.druid.server.coordination.SegmentChangeRequestLoad; @@ -41,6 +42,7 @@ import java.util.Collection; import java.util.List; import java.util.Set; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; @@ -154,7 +156,6 @@ public void loadSegment( log.info("Asking server peon[%s] to load segment[%s]", basePath, segment.getIdentifier()); queuedSize.addAndGet(segment.getSize()); segmentsToLoad.put(segment, new SegmentHolder(segment, LOAD, Arrays.asList(callback))); - doNext(); } public void dropSegment( @@ -184,115 +185,98 @@ public void dropSegment( log.info("Asking server peon[%s] to drop segment[%s]", basePath, segment.getIdentifier()); segmentsToDrop.put(segment, new SegmentHolder(segment, DROP, Arrays.asList(callback))); - doNext(); } - private void doNext() - { - synchronized (lock) { - if (currentlyProcessing == null) { - if (!segmentsToDrop.isEmpty()) { - currentlyProcessing = segmentsToDrop.firstEntry().getValue(); - log.info("Server[%s] dropping [%s]", basePath, currentlyProcessing.getSegmentIdentifier()); - } else if (!segmentsToLoad.isEmpty()) { - currentlyProcessing = segmentsToLoad.firstEntry().getValue(); - log.info("Server[%s] loading [%s]", basePath, currentlyProcessing.getSegmentIdentifier()); - } else { + private void processSegmentChangeRequest() { + if (currentlyProcessing == null) { + if (!segmentsToDrop.isEmpty()) { + currentlyProcessing = segmentsToDrop.firstEntry().getValue(); + log.info("Server[%s] dropping [%s]", basePath, currentlyProcessing.getSegmentIdentifier()); + } else if (!segmentsToLoad.isEmpty()) { + currentlyProcessing = segmentsToLoad.firstEntry().getValue(); + log.info("Server[%s] loading [%s]", basePath, currentlyProcessing.getSegmentIdentifier()); + } else { + return; + } + + try { + if (currentlyProcessing == null) { + if(!stopped) { + log.makeAlert("Crazy race condition! server[%s]", basePath) + .emit(); + } + actionCompleted(); return; } - processingExecutor.execute( - new Runnable() - { - @Override - public void run() - { - synchronized (lock) { - try { - // expected when the coordinator looses leadership and LoadQueuePeon is stopped. - if (currentlyProcessing == null) { - if(!stopped) { - log.makeAlert("Crazy race condition! server[%s]", basePath) - .emit(); + log.info("Server[%s] processing segment[%s]", basePath, currentlyProcessing.getSegmentIdentifier()); + final String path = ZKPaths.makePath(basePath, currentlyProcessing.getSegmentIdentifier()); + final byte[] payload = jsonMapper.writeValueAsBytes(currentlyProcessing.getChangeRequest()); + curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, payload); + + processingExecutor.schedule( + new Runnable() + { + @Override + public void run() + { + try { + if (curator.checkExists().forPath(path) != null) { + failAssign(new ISE("%s was never removed! Failing this operation!", path)); } - actionCompleted(); - doNext(); - return; } - log.info("Server[%s] processing segment[%s]", basePath, currentlyProcessing.getSegmentIdentifier()); - final String path = ZKPaths.makePath(basePath, currentlyProcessing.getSegmentIdentifier()); - final byte[] payload = jsonMapper.writeValueAsBytes(currentlyProcessing.getChangeRequest()); - curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, payload); - - processingExecutor.schedule( - new Runnable() - { - @Override - public void run() - { - try { - if (curator.checkExists().forPath(path) != null) { - failAssign(new ISE("%s was never removed! Failing this operation!", path)); - } - } - catch (Exception e) { - failAssign(e); - } - } - }, - config.getLoadTimeoutDelay().getMillis(), - TimeUnit.MILLISECONDS - ); - - final Stat stat = curator.checkExists().usingWatcher( - new CuratorWatcher() - { - @Override - public void process(WatchedEvent watchedEvent) throws Exception - { - switch (watchedEvent.getType()) { - case NodeDeleted: - entryRemoved(watchedEvent.getPath()); - } - } - } - ).forPath(path); - - if (stat == null) { - final byte[] noopPayload = jsonMapper.writeValueAsBytes(new SegmentChangeRequestNoop()); - - // Create a node and then delete it to remove the registered watcher. This is a work-around for - // a zookeeper race condition. Specifically, when you set a watcher, it fires on the next event - // that happens for that node. If no events happen, the watcher stays registered foreverz. - // Couple that with the fact that you cannot set a watcher when you create a node, but what we - // want is to create a node and then watch for it to get deleted. The solution is that you *can* - // set a watcher when you check to see if it exists so, we first create the node and then set a - // watcher on its existence. However, if already does not exist by the time the existence check - // returns, then the watcher that was set will never fire (nobody will ever create the node - // again) and thus lead to a slow, but real, memory leak. So, we create another node to cause - // that watcher to fire and delete it right away. - // - // We do not create the existence watcher first, because then it will fire when we create the - // node and we'll have the same race when trying to refresh that watcher. - curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, noopPayload); - - entryRemoved(path); + catch (Exception e) { + failAssign(e); } } - catch (Exception e) { - failAssign(e); + }, + config.getLoadTimeoutDelay().getMillis(), + TimeUnit.MILLISECONDS + ); + + final Stat stat = curator.checkExists().usingWatcher( + new CuratorWatcher() + { + @Override + public void process(WatchedEvent watchedEvent) throws Exception + { + switch (watchedEvent.getType()) { + case NodeDeleted: + entryRemoved(watchedEvent.getPath()); + } } } - } - } - ); - } else { - log.info( - "Server[%s] skipping doNext() because something is currently loading[%s].", - basePath, - currentlyProcessing.getSegmentIdentifier() - ); + ).forPath(path); + + if (stat == null) { + final byte[] noopPayload = jsonMapper.writeValueAsBytes(new SegmentChangeRequestNoop()); + + // Create a node and then delete it to remove the registered watcher. This is a work-around for + // a zookeeper race condition. Specifically, when you set a watcher, it fires on the next event + // that happens for that node. If no events happen, the watcher stays registered foreverz. + // Couple that with the fact that you cannot set a watcher when you create a node, but what we + // want is to create a node and then watch for it to get deleted. The solution is that you *can* + // set a watcher when you check to see if it exists so, we first create the node and then set a + // watcher on its existence. However, if already does not exist by the time the existence check + // returns, then the watcher that was set will never fire (nobody will ever create the node + // again) and thus lead to a slow, but real, memory leak. So, we create another node to cause + // that watcher to fire and delete it right away. + // + // We do not create the existence watcher first, because then it will fire when we create the + // node and we'll have the same race when trying to refresh that watcher. + curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, noopPayload); + + entryRemoved(path); + } + } catch (Exception e) { + failAssign(e); } + } else { + log.info( + "Server[%s] skipping doNext() because something is currently loading[%s].", + basePath, + currentlyProcessing.getSegmentIdentifier() + ); } } @@ -326,6 +310,29 @@ public void run() } } + public void start() + { + ScheduledExecutors.scheduleAtFixedRate( + processingExecutor, + config.getLoadQueuePeonRepeatDelay(), + config.getLoadQueuePeonRepeatDelay(), + new Callable() + { + @Override + public ScheduledExecutors.Signal call() + { + processSegmentChangeRequest(); + + if (stopped) { + return ScheduledExecutors.Signal.STOP; + } else { + return ScheduledExecutors.Signal.REPEAT; + } + } + } + ); + } + public void stop() { synchronized (lock) { @@ -371,8 +378,6 @@ private void entryRemoved(String path) actionCompleted(); log.info("Server[%s] done processing [%s]", basePath, path); } - - doNext(); } private void failAssign(Exception e) @@ -382,7 +387,6 @@ private void failAssign(Exception e) failedAssignCount.getAndIncrement(); // Act like it was completed so that the coordinator gives it to someone else actionCompleted(); - doNext(); } } diff --git a/server/src/main/java/io/druid/server/coordinator/ReplicationThrottler.java b/server/src/main/java/io/druid/server/coordinator/ReplicationThrottler.java index a32afc5ca581..6d55e751b87f 100644 --- a/server/src/main/java/io/druid/server/coordinator/ReplicationThrottler.java +++ b/server/src/main/java/io/druid/server/coordinator/ReplicationThrottler.java @@ -28,16 +28,14 @@ import java.util.concurrent.ConcurrentHashMap; /** - * The ReplicationThrottler is used to throttle the number of replicants that are created and destroyed. + * The ReplicationThrottler is used to throttle the number of replicants that are created. */ public class ReplicationThrottler { private static final EmittingLogger log = new EmittingLogger(ReplicationThrottler.class); private final Map replicatingLookup = Maps.newHashMap(); - private final Map terminatingLookup = Maps.newHashMap(); private final ReplicatorSegmentHolder currentlyReplicating = new ReplicatorSegmentHolder(); - private final ReplicatorSegmentHolder currentlyTerminating = new ReplicatorSegmentHolder(); private volatile int maxReplicants; private volatile int maxLifetime; @@ -58,11 +56,6 @@ public void updateReplicationState(String tier) update(tier, currentlyReplicating, replicatingLookup, "create"); } - public void updateTerminationState(String tier) - { - update(tier, currentlyTerminating, terminatingLookup, "terminate"); - } - private void update(String tier, ReplicatorSegmentHolder holder, Map lookup, String type) { int size = holder.getNumProcessing(tier); @@ -95,11 +88,6 @@ public boolean canCreateReplicant(String tier) return replicatingLookup.get(tier) && !currentlyReplicating.isAtMaxReplicants(tier); } - public boolean canDestroyReplicant(String tier) - { - return terminatingLookup.get(tier) && !currentlyTerminating.isAtMaxReplicants(tier); - } - public void registerReplicantCreation(String tier, String segmentId, String serverId) { currentlyReplicating.addSegment(tier, segmentId, serverId); @@ -110,16 +98,6 @@ public void unregisterReplicantCreation(String tier, String segmentId, String se currentlyReplicating.removeSegment(tier, segmentId, serverId); } - public void registerReplicantTermination(String tier, String segmentId, String serverId) - { - currentlyTerminating.addSegment(tier, segmentId, serverId); - } - - public void unregisterReplicantTermination(String tier, String segmentId, String serverId) - { - currentlyTerminating.removeSegment(tier, segmentId, serverId); - } - private class ReplicatorSegmentHolder { private final Map> currentlyProcessingSegments = Maps.newHashMap(); diff --git a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java index bec45b8b6ce5..c4a93feff6cd 100644 --- a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java +++ b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java @@ -118,7 +118,6 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) for (String tier : cluster.getTierNames()) { replicatorThrottler.updateReplicationState(tier); - replicatorThrottler.updateTerminationState(tier); } DruidCoordinatorRuntimeParams paramsWithReplicationManager = params.buildFromExistingWithoutAvailableSegments() diff --git a/server/src/main/java/io/druid/server/coordinator/rules/LoadRule.java b/server/src/main/java/io/druid/server/coordinator/rules/LoadRule.java index a19362267e69..d7e8e73015dc 100644 --- a/server/src/main/java/io/druid/server/coordinator/rules/LoadRule.java +++ b/server/src/main/java/io/druid/server/coordinator/rules/LoadRule.java @@ -203,33 +203,9 @@ private CoordinatorStats drop( } if (holder.isServingSegment(segment)) { - if (expectedNumReplicantsForTier > 0) { // don't throttle unless we are removing extra replicants - if (!replicationManager.canDestroyReplicant(tier)) { - serverQueue.add(holder); - break; - } - - replicationManager.registerReplicantTermination( - tier, - segment.getIdentifier(), - holder.getServer().getHost() - ); - } - holder.getPeon().dropSegment( segment, - new LoadPeonCallback() - { - @Override - public void execute() - { - replicationManager.unregisterReplicantTermination( - tier, - segment.getIdentifier(), - holder.getServer().getHost() - ); - } - } + null ); --loadedNumReplicantsForTier; stats.addToTieredStat(droppedCount, tier, 1); diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index 8283f254503d..5b5b53695c2b 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -25,18 +25,15 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.util.concurrent.MoreExecutors; - import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.MapCache; import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.ISE; -import io.druid.java.util.common.guava.ResourceClosingSequence; import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.SequenceWrapper; import io.druid.java.util.common.guava.Sequences; -import io.druid.java.util.common.guava.Yielder; -import io.druid.java.util.common.guava.YieldingAccumulator; import io.druid.query.CacheStrategy; import io.druid.query.Druids; import io.druid.query.Query; @@ -168,20 +165,23 @@ private void testCloseAndPopulate( throws Exception { final AssertingClosable closable = new AssertingClosable(); - final Sequence resultSeq = new ResourceClosingSequence( - Sequences.simple(expectedRes), closable - ) - { - @Override - public Yielder toYielder(Object initValue, YieldingAccumulator accumulator) - { - Assert.assertFalse(closable.isClosed()); - return super.toYielder( - initValue, - accumulator - ); - } - }; + final Sequence resultSeq = Sequences.wrap( + Sequences.simple(expectedRes), + new SequenceWrapper() + { + @Override + public void before() + { + Assert.assertFalse(closable.isClosed()); + } + + @Override + public void after(boolean isDone, Throwable thrown) throws Exception + { + closable.close(); + } + } + ); Cache cache = MapCache.create(1024 * 1024); diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorConfigTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorConfigTest.java index 2b38247a7699..5836515d0683 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorConfigTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorConfigTest.java @@ -51,6 +51,7 @@ public void testDeserialization() throws Exception Assert.assertEquals(0, config.getCoordinatorKillMaxSegments()); Assert.assertEquals(new Duration(15 * 60 * 1000), config.getLoadTimeoutDelay()); Assert.assertNull(config.getConsoleStatic()); + Assert.assertEquals(Duration.millis(50), config.getLoadQueuePeonRepeatDelay()); //with non-defaults Properties props = new Properties(); @@ -65,6 +66,7 @@ public void testDeserialization() throws Exception props.setProperty("druid.coordinator.kill.maxSegments", "10000"); props.setProperty("druid.coordinator.load.timeout", "PT1s"); props.setProperty("druid.coordinator.console.static", "test"); + props.setProperty("druid.coordinator.loadqueuepeon.repeatDelay", "PT0.100s"); factory = Config.createFactory(props); config = factory.build(DruidCoordinatorConfig.class); @@ -80,5 +82,6 @@ public void testDeserialization() throws Exception Assert.assertEquals(10000, config.getCoordinatorKillMaxSegments()); Assert.assertEquals(new Duration("PT1s"), config.getLoadTimeoutDelay()); Assert.assertEquals("test", config.getConsoleStatic()); + Assert.assertEquals(Duration.millis(100), config.getLoadQueuePeonRepeatDelay()); } } diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java index 41fada88accf..f82cb5eb8507 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java @@ -1268,7 +1268,8 @@ public void testDropReplicantThrottle() throws Exception DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); - Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get("normal").get() == 24); + // There is no throttling on drop + Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get("normal").get() == 25); EasyMock.verify(mockPeon); exec.shutdown(); } diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java index ca22bf61b923..d48420d6756c 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java @@ -130,7 +130,8 @@ public void setUp() throws Exception 10, null, false, - false + false, + new Duration("PT0s") ); pathChildrenCache = new PathChildrenCache(curator, LOADPATH, true, true, Execs.singleThreaded("coordinator_test_path_children_cache-%d")); loadQueuePeon = new LoadQueuePeon( @@ -141,6 +142,7 @@ public void setUp() throws Exception Execs.singleThreaded("coordinator_test_load_queue_peon-%d"), druidCoordinatorConfig ); + loadQueuePeon.start(); druidNode = new DruidNode("hey", "what", 1234); loadManagementPeons = new MapMaker().makeMap(); scheduledExecutorFactory = new ScheduledExecutorFactory() @@ -197,6 +199,7 @@ public void unannounce(DruidNode node) @After public void tearDown() throws Exception { + loadQueuePeon.stop(); pathChildrenCache.close(); tearDownServerAndCurator(); } diff --git a/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTest.java b/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTest.java index 8adedea909ee..68e5d8cd8953 100644 --- a/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTest.java +++ b/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTest.java @@ -88,9 +88,11 @@ public void testMultipleLoadDropSegments() throws Exception jsonMapper, Execs.scheduledSingleThreaded("test_load_queue_peon_scheduled-%d"), Execs.singleThreaded("test_load_queue_peon-%d"), - new TestDruidCoordinatorConfig(null, null, null, null, null, null, 10, null, false, false) + new TestDruidCoordinatorConfig(null, null, null, null, null, null, 10, null, false, false, Duration.ZERO) ); + loadQueuePeon.start(); + final CountDownLatch[] loadRequestSignal = new CountDownLatch[5]; final CountDownLatch[] dropRequestSignal = new CountDownLatch[5]; final CountDownLatch[] segmentLoadedSignal = new CountDownLatch[5]; @@ -294,9 +296,11 @@ public void testFailAssign() throws Exception Execs.scheduledSingleThreaded("test_load_queue_peon_scheduled-%d"), Execs.singleThreaded("test_load_queue_peon-%d"), // set time-out to 1 ms so that LoadQueuePeon will fail the assignment quickly - new TestDruidCoordinatorConfig(null, null, null, new Duration(1), null, null, 10, null, false, false) + new TestDruidCoordinatorConfig(null, null, null, new Duration(1), null, null, 10, null, false, false, new Duration("PT1s")) ); + loadQueuePeon.start(); + loadQueueCache.getListenable().addListener( new PathChildrenCacheListener() { diff --git a/server/src/test/java/io/druid/server/coordinator/TestDruidCoordinatorConfig.java b/server/src/test/java/io/druid/server/coordinator/TestDruidCoordinatorConfig.java index 112da1440a21..40dd77016f61 100644 --- a/server/src/test/java/io/druid/server/coordinator/TestDruidCoordinatorConfig.java +++ b/server/src/test/java/io/druid/server/coordinator/TestDruidCoordinatorConfig.java @@ -30,6 +30,7 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig private final Duration loadTimeoutDelay; private final Duration coordinatorKillPeriod; private final Duration coordinatorKillDurationToRetain; + private final Duration getLoadQueuePeonRepeatDelay; private final int coordinatorKillMaxSegments; private final String consoleStatic; @@ -47,7 +48,8 @@ public TestDruidCoordinatorConfig( int coordinatorKillMaxSegments, String consoleStatic, boolean mergeSegments, - boolean convertSegments + boolean convertSegments, + Duration getLoadQueuePeonRepeatDelay ) { this.coordinatorStartDelay = coordinatorStartDelay; @@ -60,6 +62,7 @@ public TestDruidCoordinatorConfig( this.consoleStatic = consoleStatic; this.mergeSegments = mergeSegments; this.convertSegments = convertSegments; + this.getLoadQueuePeonRepeatDelay = getLoadQueuePeonRepeatDelay; } @Override @@ -121,4 +124,9 @@ public String getConsoleStatic() { return consoleStatic; } + + @Override public Duration getLoadQueuePeonRepeatDelay() + { + return getLoadQueuePeonRepeatDelay; + } } diff --git a/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java b/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java index cbefc9836bba..89e1d6c9887b 100644 --- a/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java @@ -111,7 +111,8 @@ private void testFindIntervalForKillTask(List segmentManagerResult, In 1000, null, false, - false + false, + Duration.ZERO ) ); diff --git a/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java b/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java index 2d949ea25dd6..9e6d77dd5599 100644 --- a/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java +++ b/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java @@ -89,7 +89,6 @@ public void setUp() throws Exception throttler = new ReplicationThrottler(2, 1); for (String tier : Arrays.asList("hot", DruidServer.DEFAULT_TIER)) { throttler.updateReplicationState(tier); - throttler.updateTerminationState(tier); } segment = new DataSegment( "foo", diff --git a/services/src/main/java/io/druid/cli/DumpSegment.java b/services/src/main/java/io/druid/cli/DumpSegment.java index 9ef500fc2c92..b9fdcdebaf6d 100644 --- a/services/src/main/java/io/druid/cli/DumpSegment.java +++ b/services/src/main/java/io/druid/cli/DumpSegment.java @@ -487,8 +487,6 @@ public Object accumulate(Object accumulated, T in) { return null; } - - public void close() {} } ); } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/SimpleSequence.java b/sql/src/main/java/io/druid/sql/avatica/AvaticaServerConfig.java similarity index 57% rename from java-util/src/main/java/io/druid/java/util/common/guava/SimpleSequence.java rename to sql/src/main/java/io/druid/sql/avatica/AvaticaServerConfig.java index a3aa801a9d39..40f990bafa76 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/SimpleSequence.java +++ b/sql/src/main/java/io/druid/sql/avatica/AvaticaServerConfig.java @@ -17,40 +17,34 @@ * under the License. */ -package io.druid.java.util.common.guava; +package io.druid.sql.avatica; -import java.util.Iterator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Period; - -/** - */ -public class SimpleSequence extends BaseSequence> +public class AvaticaServerConfig { - public static Sequence create(Iterable iterable) + @JsonProperty + public int maxConnections = 25; + + @JsonProperty + public int maxStatementsPerConnection = 4; + + @JsonProperty + public Period connectionIdleTimeout = new Period("PT30M"); + + public int getMaxConnections() { - return new SimpleSequence<>(iterable); + return maxConnections; } - public SimpleSequence( - final Iterable iterable - ) + public int getMaxStatementsPerConnection() { - super( - new IteratorMaker>() - { - @Override - public Iterator make() - { - return iterable.iterator(); - } - - @Override - public void cleanup(Iterator iterFromMake) - { - - } - } - ); + return maxStatementsPerConnection; } + public Period getConnectionIdleTimeout() + { + return connectionIdleTimeout; + } } diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidAvaticaHandler.java b/sql/src/main/java/io/druid/sql/avatica/DruidAvaticaHandler.java index 2d904e5c4db8..117fcdb8985c 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidAvaticaHandler.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidAvaticaHandler.java @@ -22,12 +22,9 @@ import com.google.inject.Inject; import io.druid.guice.annotations.Self; import io.druid.server.DruidNode; -import org.apache.calcite.avatica.Meta; import org.apache.calcite.avatica.remote.LocalService; import org.apache.calcite.avatica.remote.Service; import org.apache.calcite.avatica.server.AvaticaJsonHandler; -import org.apache.calcite.jdbc.CalciteConnection; -import org.apache.calcite.jdbc.CalciteMetaImpl; import org.eclipse.jetty.server.Request; import javax.servlet.ServletException; @@ -42,16 +39,12 @@ public class DruidAvaticaHandler extends AvaticaJsonHandler @Inject public DruidAvaticaHandler( - final CalciteConnection connection, + final DruidMeta druidMeta, @Self final DruidNode druidNode, final AvaticaMonitor avaticaMonitor ) throws InstantiationException, IllegalAccessException, InvocationTargetException { - super( - new LocalService((Meta) CalciteMetaImpl.class.getConstructors()[0].newInstance(connection), avaticaMonitor), - avaticaMonitor - ); - + super(new LocalService(druidMeta), avaticaMonitor); setServerRpcMetadata(new Service.RpcMetadataResponse(druidNode.getHostAndPort())); } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Fns.java b/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java similarity index 57% rename from java-util/src/main/java/io/druid/java/util/common/guava/Fns.java rename to sql/src/main/java/io/druid/sql/avatica/DruidConnection.java index 9447d2f41c27..0355939f7f32 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Fns.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java @@ -17,36 +17,36 @@ * under the License. */ -package io.druid.java.util.common.guava; - -import com.google.common.base.Function; +package io.druid.sql.avatica; +import java.util.HashMap; import java.util.Map; +import java.util.concurrent.Future; /** + * Connection tracking for {@link DruidMeta}. Not thread-safe. */ -public class Fns +public class DruidConnection { - public static Function splitFn(final String splitChar, final int numCols) + private final Map statements; + private Future timeoutFuture; + + public DruidConnection() + { + this.statements = new HashMap<>(); + } + + public Map statements() { - return new Function() - { - public String[] apply(String input) - { - return input.split(splitChar, numCols); - } - }; + return statements; } - public static Function, OutType> getFromMap(final KeyType key) + public DruidConnection sync(final Future newTimeoutFuture) { - return new Function, OutType>() - { - @Override - public OutType apply(Map in) - { - return in.get(key); - } - }; + if (timeoutFuture != null) { + timeoutFuture.cancel(false); + } + timeoutFuture = newTimeoutFuture; + return this; } } diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java b/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java new file mode 100644 index 000000000000..02bf45f64e8e --- /dev/null +++ b/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java @@ -0,0 +1,592 @@ +/* + * 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.sql.avatica; + +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.io.Closer; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.google.inject.Inject; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.logger.Logger; +import io.druid.sql.calcite.planner.Calcites; +import io.druid.sql.calcite.planner.PlannerFactory; +import org.apache.calcite.avatica.MetaImpl; +import org.apache.calcite.avatica.MissingResultsException; +import org.apache.calcite.avatica.NoSuchStatementException; +import org.apache.calcite.avatica.QueryState; +import org.apache.calcite.avatica.remote.TypedValue; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +public class DruidMeta extends MetaImpl +{ + private static final Logger log = new Logger(DruidMeta.class); + + private final PlannerFactory plannerFactory; + private final ScheduledExecutorService exec; + private final AvaticaServerConfig config; + + // Used to track statements for a connection. Connection id -> statement id -> statement. + // Not concurrent; synchronize on it when reading or writing. + private final Map connections = new HashMap<>(); + + // Used to generate statement ids. + private final AtomicInteger statementCounter = new AtomicInteger(); + + @Inject + public DruidMeta(final PlannerFactory plannerFactory, final AvaticaServerConfig config) + { + super(null); + this.plannerFactory = Preconditions.checkNotNull(plannerFactory, "plannerFactory"); + this.config = config; + this.exec = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setNameFormat(String.format("DruidMeta@%s-ScheduledExecutor", Integer.toHexString(hashCode()))) + .setDaemon(true) + .build() + ); + } + + @Override + public void openConnection(final ConnectionHandle ch, final Map info) + { + getDruidConnection(ch.id, true); + } + + @Override + public void closeConnection(final ConnectionHandle ch) + { + final List statements = new ArrayList<>(); + + synchronized (connections) { + final DruidConnection connection = connections.remove(ch.id); + if (connection != null) { + connection.sync(null); + statements.addAll(connection.statements().values()); + log.debug("Connection[%s] closed, closing %,d statements.", ch.id, statements.size()); + } + } + + final Closer closer = Closer.create(); + for (final DruidStatement statement : statements) { + closer.register(statement); + } + try { + closer.close(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + @Override + public ConnectionProperties connectionSync(final ConnectionHandle ch, final ConnectionProperties connProps) + { + // getDruidConnection re-syncs it. + getDruidConnection(ch.id); + return connProps; + } + + @Override + public StatementHandle createStatement(final ConnectionHandle ch) + { + synchronized (connections) { + final DruidConnection connection = getDruidConnection(ch.id); + final StatementHandle statement = new StatementHandle(ch.id, statementCounter.incrementAndGet(), null); + + if (connection.statements().containsKey(statement.id)) { + // Will only happen if statementCounter rolls over before old statements are cleaned up. If this + // ever happens then something fishy is going on, because we shouldn't have billions of statements. + throw new ISE("Uh oh, too many statements"); + } + + if (connection.statements().size() >= config.getMaxStatementsPerConnection()) { + throw new ISE("Too many open statements, limit is[%,d]", config.getMaxStatementsPerConnection()); + } + + connection.statements().put(statement.id, new DruidStatement(ch.id, statement.id)); + log.debug("Connection[%s] opened statement[%s].", ch.id, statement.id); + return statement; + } + } + + @Override + public StatementHandle prepare( + final ConnectionHandle ch, + final String sql, + final long maxRowCount + ) + { + final StatementHandle statement = createStatement(ch); + final DruidStatement druidStatement = getDruidStatement(statement); + statement.signature = druidStatement.prepare(plannerFactory, sql, maxRowCount).getSignature(); + return statement; + } + + @Deprecated + @Override + public ExecuteResult prepareAndExecute( + final StatementHandle h, + final String sql, + final long maxRowCount, + final PrepareCallback callback + ) throws NoSuchStatementException + { + // Avatica doesn't call this. + throw new UnsupportedOperationException("Deprecated"); + } + + @Override + public ExecuteResult prepareAndExecute( + final StatementHandle statement, + final String sql, + final long maxRowCount, + final int maxRowsInFirstFrame, + final PrepareCallback callback + ) throws NoSuchStatementException + { + // Ignore "callback", this class is designed for use with LocalService which doesn't use it. + final DruidStatement druidStatement = getDruidStatement(statement); + final Signature signature = druidStatement.prepare(plannerFactory, sql, maxRowCount).getSignature(); + final Frame firstFrame = druidStatement.execute().nextFrame(DruidStatement.START_OFFSET, maxRowsInFirstFrame); + + return new ExecuteResult( + ImmutableList.of( + MetaResultSet.create( + statement.connectionId, + statement.id, + false, + signature, + firstFrame + ) + ) + ); + } + + @Override + public ExecuteBatchResult prepareAndExecuteBatch( + final StatementHandle statement, + final List sqlCommands + ) throws NoSuchStatementException + { + // Batch statements are used for bulk updates, but we don't support updates. + throw new UnsupportedOperationException("Batch statements not supported"); + } + + @Override + public ExecuteBatchResult executeBatch( + final StatementHandle statement, + final List> parameterValues + ) throws NoSuchStatementException + { + // Batch statements are used for bulk updates, but we don't support updates. + throw new UnsupportedOperationException("Batch statements not supported"); + } + + @Override + public Frame fetch( + final StatementHandle statement, + final long offset, + final int fetchMaxRowCount + ) throws NoSuchStatementException, MissingResultsException + { + return getDruidStatement(statement).nextFrame(offset, fetchMaxRowCount); + } + + @Deprecated + @Override + public ExecuteResult execute( + final StatementHandle statement, + final List parameterValues, + final long maxRowCount + ) throws NoSuchStatementException + { + // Avatica doesn't call this. + throw new UnsupportedOperationException("Deprecated"); + } + + @Override + public ExecuteResult execute( + final StatementHandle statement, + final List parameterValues, + final int maxRowsInFirstFrame + ) throws NoSuchStatementException + { + Preconditions.checkArgument(parameterValues.isEmpty(), "Expected parameterValues to be empty"); + + final DruidStatement druidStatement = getDruidStatement(statement); + final Signature signature = druidStatement.getSignature(); + final Frame firstFrame = druidStatement.execute().nextFrame(DruidStatement.START_OFFSET, maxRowsInFirstFrame); + + return new ExecuteResult( + ImmutableList.of( + MetaResultSet.create( + statement.connectionId, + statement.id, + false, + signature, + firstFrame + ) + ) + ); + } + + @Override + public Iterable createIterable( + final StatementHandle statement, + final QueryState state, + final Signature signature, + final List parameterValues, + final Frame firstFrame + ) + { + // Avatica calls this but ignores the return value. + return null; + } + + @Override + public void closeStatement(final StatementHandle h) + { + closeDruidStatement(getDruidStatement(h)); + } + + @Override + public boolean syncResults( + final StatementHandle sh, + final QueryState state, + final long offset + ) throws NoSuchStatementException + { + final DruidStatement druidStatement = getDruidStatement(sh); + final boolean isDone = druidStatement.isDone(); + final long currentOffset = druidStatement.getCurrentOffset(); + if (currentOffset != offset) { + throw new ISE("Requested offset[%,d] does not match currentOffset[%,d]", offset, currentOffset); + } + return !isDone; + } + + @Override + public void commit(final ConnectionHandle ch) + { + // We don't support writes, just ignore commits. + } + + @Override + public void rollback(final ConnectionHandle ch) + { + // We don't support writes, just ignore rollbacks. + } + + @Override + public Map getDatabaseProperties(final ConnectionHandle ch) + { + return ImmutableMap.of(); + } + + @Override + public MetaResultSet getCatalogs(final ConnectionHandle ch) + { + final String sql = "SELECT\n" + + " DISTINCT CATALOG_NAME AS TABLE_CAT\n" + + "FROM\n" + + " INFORMATION_SCHEMA.SCHEMATA\n" + + "ORDER BY\n" + + " TABLE_CAT\n"; + + return sqlResultSet(ch, sql); + } + + @Override + public MetaResultSet getSchemas( + final ConnectionHandle ch, + final String catalog, + final Pat schemaPattern + ) + { + final List whereBuilder = new ArrayList<>(); + if (catalog != null) { + whereBuilder.add("SCHEMATA.CATALOG_NAME = " + Calcites.escapeStringLiteral(catalog)); + } + + if (schemaPattern.s != null) { + whereBuilder.add("SCHEMATA.SCHEMA_NAME LIKE " + Calcites.escapeStringLiteral(schemaPattern.s)); + } + + final String where = whereBuilder.isEmpty() ? "" : "WHERE " + Joiner.on(" AND ").join(whereBuilder); + final String sql = "SELECT\n" + + " SCHEMA_NAME AS TABLE_SCHEM,\n" + + " CATALOG_NAME AS TABLE_CATALOG\n" + + "FROM\n" + + " INFORMATION_SCHEMA.SCHEMATA\n" + + where + "\n" + + "ORDER BY\n" + + " TABLE_CATALOG, TABLE_SCHEM\n"; + + return sqlResultSet(ch, sql); + } + + @Override + public MetaResultSet getTables( + final ConnectionHandle ch, + final String catalog, + final Pat schemaPattern, + final Pat tableNamePattern, + final List typeList + ) + { + final List whereBuilder = new ArrayList<>(); + if (catalog != null) { + whereBuilder.add("TABLES.TABLE_CATALOG = " + Calcites.escapeStringLiteral(catalog)); + } + + if (schemaPattern.s != null) { + whereBuilder.add("TABLES.TABLE_SCHEMA LIKE " + Calcites.escapeStringLiteral(schemaPattern.s)); + } + + if (tableNamePattern.s != null) { + whereBuilder.add("TABLES.TABLE_NAME LIKE " + Calcites.escapeStringLiteral(tableNamePattern.s)); + } + + if (typeList != null) { + final List escapedTypes = new ArrayList<>(); + for (String type : typeList) { + escapedTypes.add(Calcites.escapeStringLiteral(type)); + } + whereBuilder.add("TABLES.TABLE_TYPE IN (" + Joiner.on(", ").join(escapedTypes) + ")"); + } + + final String where = whereBuilder.isEmpty() ? "" : "WHERE " + Joiner.on(" AND ").join(whereBuilder); + final String sql = "SELECT\n" + + " TABLE_CATALOG AS TABLE_CAT,\n" + + " TABLE_SCHEMA AS TABLE_SCHEM,\n" + + " TABLE_NAME AS TABLE_NAME,\n" + + " TABLE_TYPE AS TABLE_TYPE,\n" + + " CAST(NULL AS VARCHAR) AS REMARKS,\n" + + " CAST(NULL AS VARCHAR) AS TYPE_CAT,\n" + + " CAST(NULL AS VARCHAR) AS TYPE_SCHEM,\n" + + " CAST(NULL AS VARCHAR) AS TYPE_NAME,\n" + + " CAST(NULL AS VARCHAR) AS SELF_REFERENCING_COL_NAME,\n" + + " CAST(NULL AS VARCHAR) AS REF_GENERATION\n" + + "FROM\n" + + " INFORMATION_SCHEMA.TABLES\n" + + where + "\n" + + "ORDER BY\n" + + " TABLE_TYPE, TABLE_CAT, TABLE_SCHEM, TABLE_NAME\n"; + + return sqlResultSet(ch, sql); + } + + @Override + public MetaResultSet getColumns( + final ConnectionHandle ch, + final String catalog, + final Pat schemaPattern, + final Pat tableNamePattern, + final Pat columnNamePattern + ) + { + final List whereBuilder = new ArrayList<>(); + if (catalog != null) { + whereBuilder.add("COLUMNS.TABLE_CATALOG = " + Calcites.escapeStringLiteral(catalog)); + } + + if (schemaPattern.s != null) { + whereBuilder.add("COLUMNS.TABLE_SCHEMA LIKE " + Calcites.escapeStringLiteral(schemaPattern.s)); + } + + if (tableNamePattern.s != null) { + whereBuilder.add("COLUMNS.TABLE_NAME LIKE " + Calcites.escapeStringLiteral(tableNamePattern.s)); + } + + if (columnNamePattern.s != null) { + whereBuilder.add("COLUMNS.COLUMN_NAME LIKE " + Calcites.escapeStringLiteral(columnNamePattern.s)); + } + + final String where = whereBuilder.isEmpty() ? "" : "WHERE " + Joiner.on(" AND ").join(whereBuilder); + final String sql = "SELECT\n" + + " TABLE_CATALOG AS TABLE_CAT,\n" + + " TABLE_SCHEMA AS TABLE_SCHEM,\n" + + " TABLE_NAME AS TABLE_NAME,\n" + + " COLUMN_NAME AS COLUMN_NAME,\n" + + " CAST(JDBC_TYPE AS INTEGER) AS DATA_TYPE,\n" + + " DATA_TYPE AS TYPE_NAME,\n" + + " -1 AS COLUMN_SIZE,\n" + + " -1 AS BUFFER_LENGTH,\n" + + " -1 AS DECIMAL_DIGITS,\n" + + " -1 AS NUM_PREC_RADIX,\n" + + " CASE IS_NULLABLE WHEN 'YES' THEN 1 ELSE 0 END AS NULLABLE,\n" + + " CAST(NULL AS VARCHAR) AS REMARKS,\n" + + " COLUMN_DEFAULT AS COLUMN_DEF,\n" + + " -1 AS SQL_DATA_TYPE,\n" + + " -1 AS SQL_DATETIME_SUB,\n" + + " -1 AS CHAR_OCTET_LENGTH,\n" + + " CAST(ORDINAL_POSITION AS INTEGER) AS ORDINAL_POSITION,\n" + + " IS_NULLABLE AS IS_NULLABLE,\n" + + " CAST(NULL AS VARCHAR) AS SCOPE_CATALOG,\n" + + " CAST(NULL AS VARCHAR) AS SCOPE_SCHEMA,\n" + + " CAST(NULL AS VARCHAR) AS SCOPE_TABLE,\n" + + " -1 AS SOURCE_DATA_TYPE,\n" + + " 'NO' AS IS_AUTOINCREMENT,\n" + + " 'NO' AS IS_GENERATEDCOLUMN\n" + + "FROM\n" + + " INFORMATION_SCHEMA.COLUMNS\n" + + where + "\n" + + "ORDER BY\n" + + " TABLE_CAT, TABLE_SCHEM, TABLE_NAME, ORDINAL_POSITION\n"; + + return sqlResultSet(ch, sql); + } + + @Override + public MetaResultSet getTableTypes(final ConnectionHandle ch) + { + final String sql = "SELECT\n" + + " DISTINCT TABLE_TYPE AS TABLE_TYPE\n" + + "FROM\n" + + " INFORMATION_SCHEMA.TABLES\n" + + "ORDER BY\n" + + " TABLE_TYPE\n"; + + return sqlResultSet(ch, sql); + } + + private DruidConnection getDruidConnection(final String connectionId) + { + return getDruidConnection(connectionId, false); + } + + private DruidConnection getDruidConnection(final String connectionId, final boolean createIfNotExists) + { + DruidConnection connection; + + synchronized (connections) { + connection = connections.get(connectionId); + + if (connection == null && createIfNotExists) { + if (connections.size() >= config.getMaxConnections()) { + throw new ISE("Too many connections, limit is[%,d]", config.getMaxConnections()); + } + connection = new DruidConnection(); + connections.put(connectionId, connection); + log.debug("Connection[%s] opened.", connectionId); + } + + if (connection == null) { + throw new ISE("Connection[%s] not open", connectionId); + } + } + + final DruidConnection finalConnection = connection; + + return finalConnection.sync( + exec.schedule( + new Runnable() + { + @Override + public void run() + { + final List statements = new ArrayList<>(); + + synchronized (connections) { + if (connections.remove(connectionId) == finalConnection) { + statements.addAll(finalConnection.statements().values()); + log.debug("Connection[%s] timed out, closing %,d statements.", connectionId, statements.size()); + } + } + + final Closer closer = Closer.create(); + for (final DruidStatement statement : statements) { + closer.register(statement); + } + try { + closer.close(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + }, + new Interval(new DateTime(), config.getConnectionIdleTimeout()).toDurationMillis(), + TimeUnit.MILLISECONDS + ) + ); + } + + private DruidStatement getDruidStatement(final StatementHandle statement) + { + synchronized (connections) { + final DruidConnection connection = getDruidConnection(statement.connectionId); + final DruidStatement druidStatement = connection.statements().get(statement.id); + Preconditions.checkState(druidStatement != null, "Statement[%s] does not exist", statement.id); + return druidStatement; + } + } + + private void closeDruidStatement(final DruidStatement statement) + { + synchronized (connections) { + final DruidConnection connection = getDruidConnection(statement.getConnectionId()); + if (connection.statements().get(statement.getStatementId()) == statement) { + connection.statements().remove(statement.getStatementId()); + } else { + // "statement" is not actually in the set of open statements for this connection + throw new ISE("Statement[%s] not open", statement.getStatementId()); + } + } + + log.debug("Connection[%s] closed statement[%s].", statement.getConnectionId(), statement.getStatementId()); + statement.close(); + } + + private MetaResultSet sqlResultSet(final ConnectionHandle ch, final String sql) + { + final StatementHandle statement = createStatement(ch); + try { + final ExecuteResult result = prepareAndExecute(statement, sql, -1, -1, null); + final MetaResultSet metaResultSet = Iterables.getOnlyElement(result.resultSets); + if (!metaResultSet.firstFrame.done) { + throw new ISE("Expected all results to be in a single frame!"); + } + return metaResultSet; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + finally { + closeStatement(statement); + } + } +} diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java b/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java new file mode 100644 index 000000000000..e42c793a4016 --- /dev/null +++ b/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java @@ -0,0 +1,282 @@ +/* + * 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.sql.avatica; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Sequences; +import io.druid.java.util.common.guava.Yielder; +import io.druid.java.util.common.guava.Yielders; +import io.druid.sql.calcite.planner.Calcites; +import io.druid.sql.calcite.planner.PlannerFactory; +import io.druid.sql.calcite.planner.PlannerResult; +import io.druid.sql.calcite.rel.QueryMaker; +import org.apache.calcite.avatica.AvaticaParameter; +import org.apache.calcite.avatica.ColumnMetaData; +import org.apache.calcite.avatica.Meta; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.tools.Planner; + +import javax.annotation.concurrent.GuardedBy; +import java.io.Closeable; +import java.io.IOException; +import java.sql.DatabaseMetaData; +import java.util.ArrayList; +import java.util.List; + +/** + * Statement handle for {@link DruidMeta}. Thread-safe. + */ +public class DruidStatement implements Closeable +{ + public static final long START_OFFSET = 0; + + enum State + { + NEW, + PREPARED, + RUNNING, + DONE + } + + private final String connectionId; + private final int statementId; + private final Object lock = new Object(); + + private State state = State.NEW; + private String query; + private long maxRowCount; + private PlannerResult plannerResult; + private Meta.Signature signature; + private Yielder yielder; + private int offset = 0; + + public DruidStatement(final String connectionId, final int statementId) + { + this.connectionId = connectionId; + this.statementId = statementId; + } + + public static List createColumnMetaData(final RelDataType rowType) + { + final List columns = new ArrayList<>(); + List fieldList = rowType.getFieldList(); + + for (int i = 0; i < fieldList.size(); i++) { + RelDataTypeField field = fieldList.get(i); + final ColumnMetaData.Rep rep = QueryMaker.rep(field.getType().getSqlTypeName()); + final ColumnMetaData.ScalarType columnType = ColumnMetaData.scalar( + field.getType().getSqlTypeName().getJdbcOrdinal(), + field.getType().getSqlTypeName().getName(), + rep + ); + columns.add( + new ColumnMetaData( + i, // ordinal + false, // auto increment + true, // case sensitive + false, // searchable + false, // currency + field.getType().isNullable() + ? DatabaseMetaData.columnNullable + : DatabaseMetaData.columnNoNulls, // nullable + true, // signed + field.getType().getPrecision(), // display size + field.getName(), // label + null, // column name + null, // schema name + field.getType().getPrecision(), // precision + field.getType().getScale(), // scale + null, // table name + null, // catalog name + columnType, // avatica type + true, // read only + false, // writable + false, // definitely writable + columnType.columnClassName() // column class name + ) + ); + } + + return columns; + } + + public DruidStatement prepare(final PlannerFactory plannerFactory, final String query, final long maxRowCount) + { + try (final Planner planner = plannerFactory.createPlanner()) { + synchronized (lock) { + ensure(State.NEW); + this.plannerResult = Calcites.plan(planner, query); + this.maxRowCount = maxRowCount; + this.query = query; + this.signature = Meta.Signature.create( + createColumnMetaData(plannerResult.rowType()), + query, + new ArrayList(), + Meta.CursorFactory.ARRAY, + Meta.StatementType.SELECT // We only support SELECT + ); + this.state = State.PREPARED; + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + return this; + } + + public DruidStatement execute() + { + synchronized (lock) { + ensure(State.PREPARED); + + final Sequence baseSequence = plannerResult.run(); + + // We can't apply limits greater than Integer.MAX_VALUE, ignore them. + final Sequence retSequence = + maxRowCount >= 0 && maxRowCount <= Integer.MAX_VALUE + ? Sequences.limit(baseSequence, (int) maxRowCount) + : baseSequence; + + yielder = Yielders.each(retSequence); + state = State.RUNNING; + + return this; + } + } + + public String getConnectionId() + { + return connectionId; + } + + public int getStatementId() + { + return statementId; + } + + public String getQuery() + { + synchronized (lock) { + ensure(State.PREPARED, State.RUNNING, State.DONE); + return query; + } + } + + public Meta.Signature getSignature() + { + synchronized (lock) { + ensure(State.PREPARED, State.RUNNING, State.DONE); + return signature; + } + } + + public RelDataType getRowType() + { + synchronized (lock) { + ensure(State.PREPARED, State.RUNNING, State.DONE); + return plannerResult.rowType(); + } + } + + public long getCurrentOffset() + { + synchronized (lock) { + ensure(State.RUNNING, State.DONE); + return offset; + } + } + + public boolean isDone() + { + synchronized (lock) { + return state == State.DONE; + } + } + + public Meta.Frame nextFrame(final long fetchOffset, final int fetchMaxRowCount) + { + synchronized (lock) { + ensure(State.RUNNING); + Preconditions.checkState(fetchOffset == offset, "fetchOffset[%,d] != offset[%,d]", fetchOffset, offset); + + try { + final List rows = new ArrayList<>(); + while (!yielder.isDone() && (fetchMaxRowCount < 0 || offset < fetchOffset + fetchMaxRowCount)) { + rows.add(yielder.get()); + yielder = yielder.next(null); + offset++; + } + + final boolean done = yielder.isDone(); + if (done) { + close(); + } + + return new Meta.Frame(fetchOffset, done, rows); + } + catch (Throwable t) { + try { + close(); + } + catch (Throwable t1) { + t.addSuppressed(t1); + } + throw t; + } + } + } + + @Override + public void close() + { + synchronized (lock) { + state = State.DONE; + + if (yielder != null) { + Yielder theYielder = this.yielder; + this.yielder = null; + + // Put the close last, so any exceptions it throws are after we did the other cleanup above. + try { + theYielder.close(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + } + } + + @GuardedBy("lock") + private void ensure(final State... desiredStates) + { + for (State desiredState : desiredStates) { + if (state == desiredState) { + return; + } + } + throw new ISE("Invalid action for state[%s]", state); + } +} diff --git a/sql/src/main/java/io/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java b/sql/src/main/java/io/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java new file mode 100644 index 000000000000..36ed5d8cf62e --- /dev/null +++ b/sql/src/main/java/io/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java @@ -0,0 +1,122 @@ +/* + * 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.sql.calcite.aggregation; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory; +import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import io.druid.query.dimension.DimensionSpec; +import io.druid.segment.column.ValueType; +import io.druid.sql.calcite.expression.Expressions; +import io.druid.sql.calcite.expression.RowExtraction; +import io.druid.sql.calcite.table.RowSignature; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.sql.SqlAggFunction; +import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.type.InferTypes; +import org.apache.calcite.sql.type.OperandTypes; +import org.apache.calcite.sql.type.ReturnTypes; +import org.apache.calcite.sql.type.SqlTypeName; + +import java.util.List; + +public class ApproxCountDistinctSqlAggregator implements SqlAggregator +{ + private static final SqlAggFunction FUNCTION_INSTANCE = new ApproxCountDistinctSqlAggFunction(); + private static final String NAME = "APPROX_COUNT_DISTINCT"; + + @Override + public SqlAggFunction calciteFunction() + { + return FUNCTION_INSTANCE; + } + + @Override + public Aggregation toDruidAggregation( + final String name, + final RowSignature rowSignature, + final List existingAggregations, + final Project project, + final AggregateCall aggregateCall + ) + { + final RowExtraction rex = Expressions.toRowExtraction( + rowSignature.getRowOrder(), + Expressions.fromFieldAccess( + rowSignature, + project, + Iterables.getOnlyElement(aggregateCall.getArgList()) + ) + ); + if (rex == null) { + return null; + } + + final AggregatorFactory aggregatorFactory; + + if (rowSignature.getColumnType(rex.getColumn()) == ValueType.COMPLEX) { + aggregatorFactory = new HyperUniquesAggregatorFactory(name, rex.getColumn()); + } else { + final DimensionSpec dimensionSpec = rex.toDimensionSpec(rowSignature, null); + if (dimensionSpec == null) { + return null; + } + + aggregatorFactory = new CardinalityAggregatorFactory(name, ImmutableList.of(dimensionSpec), false); + } + + return Aggregation.createFinalizable( + ImmutableList.of(aggregatorFactory), + null, + new PostAggregatorFactory() + { + @Override + public PostAggregator factorize(String outputName) + { + return new HyperUniqueFinalizingPostAggregator(outputName, name); + } + } + ); + } + + private static class ApproxCountDistinctSqlAggFunction extends SqlAggFunction + { + ApproxCountDistinctSqlAggFunction() + { + super( + NAME, + null, + SqlKind.OTHER_FUNCTION, + ReturnTypes.explicit(SqlTypeName.BIGINT), + InferTypes.VARCHAR_1024, + OperandTypes.ANY, + SqlFunctionCategory.STRING, + false, + false + ); + } + } +} diff --git a/sql/src/main/java/io/druid/sql/calcite/aggregation/SqlAggregator.java b/sql/src/main/java/io/druid/sql/calcite/aggregation/SqlAggregator.java new file mode 100644 index 000000000000..f8e75f1cbfb0 --- /dev/null +++ b/sql/src/main/java/io/druid/sql/calcite/aggregation/SqlAggregator.java @@ -0,0 +1,61 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.sql.calcite.aggregation; + +import io.druid.sql.calcite.table.RowSignature; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.sql.SqlAggFunction; + +import javax.annotation.Nullable; +import java.util.List; + +/** + * Bridge between Druid and SQL aggregators. + */ +public interface SqlAggregator +{ + /** + * Returns the SQL operator corresponding to this aggregation function. Should be a singleton. + * + * @return operator + */ + SqlAggFunction calciteFunction(); + + /** + * Returns Druid Aggregation corresponding to a SQL {@link AggregateCall}. + * + * @param name desired output name of the aggregation + * @param rowSignature signature of the rows being aggregated + * @param existingAggregations existing aggregations for this query; useful for re-using aggregators + * @param project SQL projection to apply before the aggregate call + * @param aggregateCall SQL aggregate call + * + * @return aggregation, or null if the call cannot be translated + */ + @Nullable + Aggregation toDruidAggregation( + final String name, + final RowSignature rowSignature, + final List existingAggregations, + final Project project, + final AggregateCall aggregateCall + ); +} diff --git a/sql/src/main/java/io/druid/sql/calcite/filtration/CombineAndSimplifyBounds.java b/sql/src/main/java/io/druid/sql/calcite/filtration/CombineAndSimplifyBounds.java index 54d45c3c72ee..c0e943305a52 100644 --- a/sql/src/main/java/io/druid/sql/calcite/filtration/CombineAndSimplifyBounds.java +++ b/sql/src/main/java/io/druid/sql/calcite/filtration/CombineAndSimplifyBounds.java @@ -31,6 +31,7 @@ import io.druid.query.filter.NotDimFilter; import io.druid.query.filter.OrDimFilter; +import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -52,22 +53,57 @@ public static CombineAndSimplifyBounds instance() public DimFilter process(DimFilter filter) { if (filter instanceof AndDimFilter) { - final List children = ((AndDimFilter) filter).getFields(); + final List children = getAndFilterChildren((AndDimFilter) filter); final DimFilter one = doSimplifyAnd(children); final DimFilter two = negate(doSimplifyOr(negateAll(children))); return computeCost(one) <= computeCost(two) ? one : two; } else if (filter instanceof OrDimFilter) { - final List children = ((OrDimFilter) filter).getFields(); + final List children = getOrFilterChildren((OrDimFilter) filter); final DimFilter one = doSimplifyOr(children); final DimFilter two = negate(doSimplifyAnd(negateAll(children))); return computeCost(one) <= computeCost(two) ? one : two; } else if (filter instanceof NotDimFilter) { - return negate(((NotDimFilter) filter).getField()); + final DimFilter field = ((NotDimFilter) filter).getField(); + final DimFilter candidate; + if (field instanceof OrDimFilter) { + candidate = doSimplifyAnd(negateAll(getOrFilterChildren((OrDimFilter) field))); + } else if (field instanceof AndDimFilter) { + candidate = doSimplifyOr(negateAll(getAndFilterChildren((AndDimFilter) field))); + } else { + candidate = negate(field); + } + return computeCost(filter) <= computeCost(candidate) ? filter : candidate; } else { return filter; } } + private List getAndFilterChildren(final AndDimFilter filter) + { + final List children = new ArrayList<>(); + for (final DimFilter field : filter.getFields()) { + if (field instanceof AndDimFilter) { + children.addAll(getAndFilterChildren((AndDimFilter) field)); + } else { + children.add(field); + } + } + return children; + } + + private List getOrFilterChildren(final OrDimFilter filter) + { + final List children = new ArrayList<>(); + for (final DimFilter field : filter.getFields()) { + if (field instanceof OrDimFilter) { + children.addAll(getOrFilterChildren((OrDimFilter) field)); + } else { + children.add(field); + } + } + return children; + } + private static DimFilter doSimplifyAnd(final List children) { return doSimplify(children, false); diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/Calcites.java b/sql/src/main/java/io/druid/sql/calcite/planner/Calcites.java index 28e7c6254c0e..f56c53c0b20c 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/Calcites.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/Calcites.java @@ -19,71 +19,275 @@ package io.druid.sql.calcite.planner; -import org.apache.calcite.jdbc.CalciteConnection; -import org.apache.calcite.jdbc.CalciteJdbc41Factory; -import org.apache.calcite.jdbc.CalcitePrepare; -import org.apache.calcite.jdbc.Driver; -import org.apache.calcite.linq4j.function.Function0; +import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; +import com.google.common.io.BaseEncoding; +import com.google.common.primitives.Chars; +import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Sequences; +import io.druid.sql.calcite.rel.DruidConvention; +import io.druid.sql.calcite.rel.DruidRel; +import io.druid.sql.calcite.schema.DruidSchema; +import io.druid.sql.calcite.schema.InformationSchema; +import org.apache.calcite.DataContext; +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.interpreter.BindableConvention; +import org.apache.calcite.interpreter.BindableRel; +import org.apache.calcite.interpreter.Bindables; +import org.apache.calcite.jdbc.CalciteSchema; +import org.apache.calcite.linq4j.Enumerable; +import org.apache.calcite.linq4j.QueryProvider; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelRoot; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexNode; import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.sql.SqlExplain; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.tools.Planner; +import org.apache.calcite.tools.RelConversionException; +import org.apache.calcite.tools.ValidationException; +import org.apache.calcite.util.ConversionUtil; +import org.apache.calcite.util.Pair; -import java.sql.SQLException; -import java.util.Properties; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.List; /** * Entry points for Calcite. */ public class Calcites { - private static final String DRUID_SCHEMA_NAME = "druid"; + private static final Charset DEFAULT_CHARSET = Charset.forName(ConversionUtil.NATIVE_UTF16_CHARSET_NAME); private Calcites() { // No instantiation. } - /** - * Create a Calcite JDBC driver. - * - * @param druidSchema "druid" schema - * - * @return JDBC driver - */ - public static CalciteConnection jdbc( - final Schema druidSchema, - final PlannerConfig plannerConfig - ) throws SQLException + public static void setSystemProperties() { - final Properties props = new Properties(); - props.setProperty("caseSensitive", "true"); - props.setProperty("unquotedCasing", "UNCHANGED"); - - final CalciteJdbc41Factory jdbcFactory = new CalciteJdbc41Factory(); - final Function0 prepareFactory = new Function0() - { - @Override - public CalcitePrepare apply() - { - return new DruidPlannerImpl(plannerConfig); + // These properties control the charsets used for SQL literals. I don't see a way to change this except through + // system properties, so we'll have to set those... + + final String charset = ConversionUtil.NATIVE_UTF16_CHARSET_NAME; + System.setProperty("saffron.default.charset", Calcites.defaultCharset().name()); + System.setProperty("saffron.default.nationalcharset", Calcites.defaultCharset().name()); + System.setProperty("saffron.default.collation.name", String.format("%s$en_US", charset)); + } + + public static Charset defaultCharset() + { + return DEFAULT_CHARSET; + } + + public static SchemaPlus createRootSchema(final Schema druidSchema) + { + final SchemaPlus rootSchema = CalciteSchema.createRootSchema(false, false).plus(); + rootSchema.add(DruidSchema.NAME, druidSchema); + rootSchema.add(InformationSchema.NAME, new InformationSchema(rootSchema)); + return rootSchema; + } + + public static String escapeStringLiteral(final String s) + { + if (s == null) { + return "''"; + } else { + boolean isPlainAscii = true; + final StringBuilder builder = new StringBuilder("'"); + for (int i = 0; i < s.length(); i++) { + final char c = s.charAt(i); + if (Character.isLetterOrDigit(c) || c == ' ') { + builder.append(c); + if (c > 127) { + isPlainAscii = false; + } + } else { + builder.append("\\").append(BaseEncoding.base16().encode(Chars.toByteArray(c))); + isPlainAscii = false; + } } - }; - final Driver driver = new Driver() - { - @Override - protected Function0 createPrepareFactory() - { - return prepareFactory; + builder.append("'"); + return isPlainAscii ? builder.toString() : "U&" + builder.toString(); + } + } + + public static PlannerResult plan( + final Planner planner, + final String sql + ) throws SqlParseException, ValidationException, RelConversionException + { + SqlExplain explain = null; + SqlNode parsed = planner.parse(sql); + if (parsed.getKind() == SqlKind.EXPLAIN) { + explain = (SqlExplain) parsed; + parsed = explain.getExplicandum(); + } + final SqlNode validated = planner.validate(parsed); + final RelRoot root = planner.rel(validated); + + try { + return planWithDruidConvention(planner, explain, root); + } + catch (RelOptPlanner.CannotPlanException e) { + // Try again with BINDABLE convention. Used for querying Values, metadata tables, and fallback. + try { + return planWithBindableConvention(planner, explain, root); + } + catch (Exception e2) { + e.addSuppressed(e2); + throw e; } - }; - final CalciteConnection calciteConnection = (CalciteConnection) jdbcFactory.newConnection( - driver, - jdbcFactory, - "jdbc:calcite:", - props + } + } + + private static PlannerResult planWithDruidConvention( + final Planner planner, + final SqlExplain explain, + final RelRoot root + ) throws RelConversionException + { + final DruidRel druidRel = (DruidRel) planner.transform( + Rules.DRUID_CONVENTION_RULES, + planner.getEmptyTraitSet() + .replace(DruidConvention.instance()) + .plus(root.collation), + root.rel ); - final SchemaPlus druidSchemaPlus = calciteConnection.getRootSchema().add(DRUID_SCHEMA_NAME, druidSchema); - druidSchemaPlus.setCacheEnabled(false); - return calciteConnection; + if (explain != null) { + return planExplanation(druidRel, explain); + } else { + final Supplier> resultsSupplier = new Supplier>() + { + @Override + public Sequence get() + { + if (root.isRefTrivial()) { + return druidRel.runQuery(); + } else { + // Add a mapping on top to accommodate root.fields. + return Sequences.map( + druidRel.runQuery(), + new Function() + { + @Override + public Object[] apply(final Object[] input) + { + final Object[] retVal = new Object[root.fields.size()]; + for (int i = 0; i < root.fields.size(); i++) { + retVal[i] = input[root.fields.get(i).getKey()]; + } + return retVal; + } + } + ); + } + } + }; + return new PlannerResult(resultsSupplier, root.validatedRowType); + } + } + + private static PlannerResult planWithBindableConvention( + final Planner planner, + final SqlExplain explain, + final RelRoot root + ) throws RelConversionException + { + BindableRel bindableRel = (BindableRel) planner.transform( + Rules.BINDABLE_CONVENTION_RULES, + planner.getEmptyTraitSet() + .replace(BindableConvention.INSTANCE) + .plus(root.collation), + root.rel + ); + + if (!root.isRefTrivial()) { + // Add a projection on top to accommodate root.fields. + final List projects = new ArrayList<>(); + final RexBuilder rexBuilder = bindableRel.getCluster().getRexBuilder(); + for (int field : Pair.left(root.fields)) { + projects.add(rexBuilder.makeInputRef(bindableRel, field)); + } + bindableRel = new Bindables.BindableProject( + bindableRel.getCluster(), + bindableRel.getTraitSet(), + bindableRel, + projects, + root.validatedRowType + ); + } + + if (explain != null) { + return planExplanation(bindableRel, explain); + } else { + final BindableRel theRel = bindableRel; + final DataContext dataContext = new DataContext() + { + @Override + public SchemaPlus getRootSchema() + { + return null; + } + + @Override + public JavaTypeFactory getTypeFactory() + { + return (JavaTypeFactory) planner.getTypeFactory(); + } + + @Override + public QueryProvider getQueryProvider() + { + return null; + } + + @Override + public Object get(final String name) + { + return null; + } + }; + final Supplier> resultsSupplier = new Supplier>() + { + @Override + public Sequence get() + { + final Enumerable enumerable = theRel.bind(dataContext); + return Sequences.simple(enumerable); + } + }; + return new PlannerResult(resultsSupplier, root.validatedRowType); + } + } + + private static PlannerResult planExplanation( + final RelNode rel, + final SqlExplain explain + ) + { + final String explanation = RelOptUtil.dumpPlan("", rel, explain.getFormat(), explain.getDetailLevel()); + final Supplier> resultsSupplier = Suppliers.ofInstance( + Sequences.simple(ImmutableList.of(new Object[]{explanation}))); + final RelDataTypeFactory typeFactory = rel.getCluster().getTypeFactory(); + return new PlannerResult( + resultsSupplier, + typeFactory.createStructType( + ImmutableList.of(typeFactory.createSqlType(SqlTypeName.VARCHAR)), + ImmutableList.of("PLAN") + ) + ); } } diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/DruidOperatorTable.java b/sql/src/main/java/io/druid/sql/calcite/planner/DruidOperatorTable.java new file mode 100644 index 000000000000..5db18afe4802 --- /dev/null +++ b/sql/src/main/java/io/druid/sql/calcite/planner/DruidOperatorTable.java @@ -0,0 +1,90 @@ +/* + * 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.sql.calcite.planner; + +import com.google.common.collect.Maps; +import com.google.inject.Inject; +import io.druid.java.util.common.ISE; +import io.druid.sql.calcite.aggregation.SqlAggregator; +import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlOperatorTable; +import org.apache.calcite.sql.SqlSyntax; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class DruidOperatorTable implements SqlOperatorTable +{ + private static final SqlStdOperatorTable STANDARD_TABLE = SqlStdOperatorTable.instance(); + + private final Map aggregators; + + @Inject + public DruidOperatorTable( + final Set aggregators + ) + { + this.aggregators = Maps.newHashMap(); + for (SqlAggregator aggregator : aggregators) { + final String lcname = aggregator.calciteFunction().getName().toLowerCase(); + if (this.aggregators.put(lcname, aggregator) != null) { + throw new ISE("Cannot have two aggregators with name[%s]", lcname); + } + } + } + + public SqlAggregator lookupAggregator(final String opName) + { + return aggregators.get(opName.toLowerCase()); + } + + @Override + public void lookupOperatorOverloads( + final SqlIdentifier opName, + final SqlFunctionCategory category, + final SqlSyntax syntax, + final List operatorList + ) + { + if (opName.names.size() == 1) { + final SqlAggregator aggregator = aggregators.get(opName.getSimple().toLowerCase()); + if (aggregator != null && syntax == SqlSyntax.FUNCTION) { + operatorList.add(aggregator.calciteFunction()); + } + } + STANDARD_TABLE.lookupOperatorOverloads(opName, category, syntax, operatorList); + } + + @Override + public List getOperatorList() + { + final List retVal = new ArrayList<>(); + for (SqlAggregator aggregator : aggregators.values()) { + retVal.add(aggregator.calciteFunction()); + } + retVal.addAll(STANDARD_TABLE.getOperatorList()); + return retVal; + } +} diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlannerImpl.java b/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlannerImpl.java deleted file mode 100644 index 403f735fea42..000000000000 --- a/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlannerImpl.java +++ /dev/null @@ -1,75 +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.sql.calcite.planner; - -import org.apache.calcite.plan.Contexts; -import org.apache.calcite.plan.ConventionTraitDef; -import org.apache.calcite.plan.RelOptCostFactory; -import org.apache.calcite.plan.RelOptPlanner; -import org.apache.calcite.plan.RelOptRule; -import org.apache.calcite.plan.volcano.VolcanoPlanner; -import org.apache.calcite.prepare.CalcitePrepareImpl; -import org.apache.calcite.rel.RelCollationTraitDef; -import org.apache.calcite.sql2rel.SqlRexConvertletTable; - -/** - * Our very own subclass of CalcitePrepareImpl, used to alter behaviors of the JDBC driver as necessary. - * - * When Calcite 1.11.0 is released, we should override "createConvertletTable" and provide the - * DruidConvertletTable. - */ -public class DruidPlannerImpl extends CalcitePrepareImpl -{ - private final PlannerConfig plannerConfig; - - public DruidPlannerImpl(PlannerConfig plannerConfig) - { - this.plannerConfig = plannerConfig; - } - - @Override - protected RelOptPlanner createPlanner( - final Context prepareContext, - final org.apache.calcite.plan.Context externalContext0, - final RelOptCostFactory costFactory - ) - { - final org.apache.calcite.plan.Context externalContext = externalContext0 != null - ? externalContext0 - : Contexts.of(prepareContext.config()); - - final VolcanoPlanner planner = new VolcanoPlanner(costFactory, externalContext); - planner.addRelTraitDef(ConventionTraitDef.INSTANCE); - planner.addRelTraitDef(RelCollationTraitDef.INSTANCE); - - // Register planner rules. - for (RelOptRule rule : Rules.ruleSet(plannerConfig)) { - planner.addRule(rule); - } - - return planner; - } - - @Override - protected SqlRexConvertletTable createConvertletTable() - { - return DruidConvertletTable.instance(); - } -} diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java new file mode 100644 index 000000000000..2befafdf1949 --- /dev/null +++ b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java @@ -0,0 +1,81 @@ +/* + * 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.sql.calcite.planner; + +import com.google.inject.Inject; +import io.druid.sql.calcite.schema.DruidSchema; +import org.apache.calcite.avatica.util.Casing; +import org.apache.calcite.avatica.util.Quoting; +import org.apache.calcite.plan.Contexts; +import org.apache.calcite.plan.ConventionTraitDef; +import org.apache.calcite.rel.RelCollationTraitDef; +import org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.calcite.rex.RexExecutorImpl; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.schema.Schemas; +import org.apache.calcite.sql.parser.SqlParser; +import org.apache.calcite.tools.FrameworkConfig; +import org.apache.calcite.tools.Frameworks; +import org.apache.calcite.tools.Planner; + +public class PlannerFactory +{ + private final SchemaPlus rootSchema; + private final DruidOperatorTable operatorTable; + private final PlannerConfig plannerConfig; + + @Inject + public PlannerFactory( + final SchemaPlus rootSchema, + final DruidOperatorTable operatorTable, + final PlannerConfig plannerConfig + ) + { + this.rootSchema = rootSchema; + this.operatorTable = operatorTable; + this.plannerConfig = plannerConfig; + } + + public Planner createPlanner() + { + final FrameworkConfig frameworkConfig = Frameworks + .newConfigBuilder() + .parserConfig( + SqlParser.configBuilder() + .setCaseSensitive(true) + .setUnquotedCasing(Casing.UNCHANGED) + .setQuotedCasing(Casing.UNCHANGED) + .setQuoting(Quoting.DOUBLE_QUOTE) + .build() + ) + .defaultSchema(rootSchema) + .traitDefs(ConventionTraitDef.INSTANCE, RelCollationTraitDef.INSTANCE) + .convertletTable(DruidConvertletTable.instance()) + .operatorTable(operatorTable) + .programs(Rules.programs(operatorTable, plannerConfig)) + .executor(new RexExecutorImpl(Schemas.createDataContext(null))) + .context(Contexts.EMPTY_CONTEXT) + .typeSystem(RelDataTypeSystem.DEFAULT) + .defaultSchema(rootSchema.getSubSchema(DruidSchema.NAME)) + .build(); + + return Frameworks.getPlanner(frameworkConfig); + } +} diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/ResourceClosingYielder.java b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerResult.java similarity index 50% rename from java-util/src/main/java/io/druid/java/util/common/guava/ResourceClosingYielder.java rename to sql/src/main/java/io/druid/sql/calcite/planner/PlannerResult.java index a5991e2bf32a..261c307060e0 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/ResourceClosingYielder.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerResult.java @@ -17,49 +17,37 @@ * under the License. */ -package io.druid.java.util.common.guava; +package io.druid.sql.calcite.planner; -import java.io.Closeable; -import java.io.IOException; +import com.google.common.base.Supplier; +import io.druid.java.util.common.guava.Sequence; +import org.apache.calcite.rel.type.RelDataType; +import java.util.concurrent.atomic.AtomicBoolean; -/** - */ -public class ResourceClosingYielder implements Yielder +public class PlannerResult { - private final Yielder baseYielder; - private final Closeable closeable; + private final Supplier> resultsSupplier; + private final RelDataType rowType; + private final AtomicBoolean didRun = new AtomicBoolean(); - public ResourceClosingYielder(Yielder baseYielder, Closeable closeable) + public PlannerResult(final Supplier> resultsSupplier, final RelDataType rowType) { - this.baseYielder = baseYielder; - this.closeable = closeable; + this.resultsSupplier = resultsSupplier; + this.rowType = rowType; } - @Override - public OutType get() + public Sequence run() { - return baseYielder.get(); - } - - @Override - public Yielder next(OutType initValue) - { - return new ResourceClosingYielder<>(baseYielder.next(initValue), closeable); - } - - @Override - public boolean isDone() - { - return baseYielder.isDone(); + if (!didRun.compareAndSet(false, true)) { + // Safety check. + throw new IllegalStateException("Cannot run more than once"); + } + return resultsSupplier.get(); } - @Override - public void close() throws IOException + public RelDataType rowType() { - if (closeable != null) { - closeable.close(); - } - baseYielder.close(); + return rowType; } } diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/Rules.java b/sql/src/main/java/io/druid/sql/calcite/planner/Rules.java index 1aba1f447919..fe40265cb45e 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/Rules.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/Rules.java @@ -20,13 +20,12 @@ package io.druid.sql.calcite.planner; import com.google.common.collect.ImmutableList; -import io.druid.sql.calcite.rule.DruidBindableConverterRule; import io.druid.sql.calcite.rule.DruidFilterRule; +import io.druid.sql.calcite.rule.DruidRelToBindableRule; +import io.druid.sql.calcite.rule.DruidRelToDruidRule; import io.druid.sql.calcite.rule.DruidSemiJoinRule; import io.druid.sql.calcite.rule.GroupByRules; import io.druid.sql.calcite.rule.SelectRules; -import org.apache.calcite.adapter.enumerable.EnumerableInterpreterRule; -import org.apache.calcite.adapter.enumerable.EnumerableRules; import org.apache.calcite.interpreter.Bindables; import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.volcano.AbstractConverter; @@ -54,7 +53,6 @@ import org.apache.calcite.rel.rules.ProjectWindowTransposeRule; import org.apache.calcite.rel.rules.PruneEmptyRules; import org.apache.calcite.rel.rules.ReduceExpressionsRule; -import org.apache.calcite.rel.rules.SemiJoinRule; import org.apache.calcite.rel.rules.SortJoinTransposeRule; import org.apache.calcite.rel.rules.SortProjectTransposeRule; import org.apache.calcite.rel.rules.SortRemoveRule; @@ -64,11 +62,16 @@ import org.apache.calcite.rel.rules.UnionPullUpConstantsRule; import org.apache.calcite.rel.rules.UnionToDistinctRule; import org.apache.calcite.rel.rules.ValuesReduceRule; +import org.apache.calcite.tools.Program; +import org.apache.calcite.tools.Programs; import java.util.List; public class Rules { + public static final int DRUID_CONVENTION_RULES = 0; + public static final int BINDABLE_CONVENTION_RULES = 1; + // Rules from CalcitePrepareImpl's DEFAULT_RULES, minus AggregateExpandDistinctAggregatesRule // and AggregateReduceFunctionsRule. private static final List DEFAULT_RULES = @@ -97,9 +100,7 @@ public class Rules ImmutableList.of( Bindables.BINDABLE_TABLE_SCAN_RULE, ProjectTableScanRule.INSTANCE, - ProjectTableScanRule.INTERPRETER, - EnumerableInterpreterRule.INSTANCE, - EnumerableRules.ENUMERABLE_VALUES_RULE + ProjectTableScanRule.INTERPRETER ); // Rules from CalcitePrepareImpl's CONSTANT_REDUCTION_RULES. @@ -115,30 +116,6 @@ public class Rules AggregateValuesRule.INSTANCE ); - // Rules from CalcitePrepareImpl's ENUMERABLE_RULES. - private static final List ENUMERABLE_RULES = - ImmutableList.of( - EnumerableRules.ENUMERABLE_JOIN_RULE, - EnumerableRules.ENUMERABLE_MERGE_JOIN_RULE, - EnumerableRules.ENUMERABLE_SEMI_JOIN_RULE, - EnumerableRules.ENUMERABLE_CORRELATE_RULE, - EnumerableRules.ENUMERABLE_PROJECT_RULE, - EnumerableRules.ENUMERABLE_FILTER_RULE, - EnumerableRules.ENUMERABLE_AGGREGATE_RULE, - EnumerableRules.ENUMERABLE_SORT_RULE, - EnumerableRules.ENUMERABLE_LIMIT_RULE, - EnumerableRules.ENUMERABLE_COLLECT_RULE, - EnumerableRules.ENUMERABLE_UNCOLLECT_RULE, - EnumerableRules.ENUMERABLE_UNION_RULE, - EnumerableRules.ENUMERABLE_INTERSECT_RULE, - EnumerableRules.ENUMERABLE_MINUS_RULE, - EnumerableRules.ENUMERABLE_TABLE_MODIFICATION_RULE, - EnumerableRules.ENUMERABLE_VALUES_RULE, - EnumerableRules.ENUMERABLE_WINDOW_RULE, - EnumerableRules.ENUMERABLE_TABLE_SCAN_RULE, - EnumerableRules.ENUMERABLE_TABLE_FUNCTION_SCAN_RULE - ); - // Rules from VolcanoPlanner's registerAbstractRelationalRules. private static final List VOLCANO_ABSTRACT_RULES = ImmutableList.of( @@ -146,7 +123,6 @@ public class Rules FilterJoinRule.JOIN, AbstractConverter.ExpandConversionRule.INSTANCE, JoinCommuteRule.INSTANCE, - SemiJoinRule.INSTANCE, AggregateRemoveRule.INSTANCE, UnionToDistinctRule.INSTANCE, ProjectRemoveRule.INSTANCE, @@ -180,7 +156,37 @@ private Rules() // No instantiation. } - public static List ruleSet(final PlannerConfig plannerConfig) + public static List programs(final DruidOperatorTable operatorTable, final PlannerConfig plannerConfig) + { + return ImmutableList.of( + Programs.ofRules(druidConventionRuleSet(operatorTable, plannerConfig)), + Programs.ofRules(bindableConventionRuleSet(operatorTable, plannerConfig)) + ); + } + + private static List druidConventionRuleSet( + final DruidOperatorTable operatorTable, + final PlannerConfig plannerConfig + ) + { + return ImmutableList.builder() + .addAll(baseRuleSet(operatorTable, plannerConfig)) + .add(DruidRelToDruidRule.instance()) + .build(); + } + + private static List bindableConventionRuleSet( + final DruidOperatorTable operatorTable, + final PlannerConfig plannerConfig + ) + { + return ImmutableList.builder() + .addAll(baseRuleSet(operatorTable, plannerConfig)) + .addAll(Bindables.RULES) + .build(); + } + + private static List baseRuleSet(final DruidOperatorTable operatorTable, final PlannerConfig plannerConfig) { final ImmutableList.Builder rules = ImmutableList.builder(); @@ -192,7 +198,7 @@ public static List ruleSet(final PlannerConfig plannerConfig) rules.addAll(RELOPTUTIL_ABSTRACT_RULES); if (plannerConfig.isUseFallback()) { - rules.addAll(ENUMERABLE_RULES); + rules.add(DruidRelToBindableRule.instance()); } // Druid-specific rules. @@ -203,10 +209,7 @@ public static List ruleSet(final PlannerConfig plannerConfig) } rules.addAll(SelectRules.rules()); - rules.addAll(GroupByRules.rules(plannerConfig)); - - // Allow conversion of Druid queries to Bindable convention. - rules.add(DruidBindableConverterRule.instance()); + rules.addAll(GroupByRules.rules(operatorTable, plannerConfig)); return rules.build(); } diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidConvention.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidConvention.java index f4eaa5f1502c..d7b2f864e357 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidConvention.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidConvention.java @@ -20,6 +20,7 @@ package io.druid.sql.calcite.rel; import org.apache.calcite.plan.Convention; +import org.apache.calcite.plan.ConventionTraitDef; import org.apache.calcite.plan.RelOptPlanner; import org.apache.calcite.plan.RelTrait; import org.apache.calcite.plan.RelTraitDef; @@ -28,6 +29,7 @@ public class DruidConvention implements Convention { private static final DruidConvention INSTANCE = new DruidConvention(); + private static final String NAME = "DRUID"; private DruidConvention() { @@ -41,13 +43,13 @@ public static DruidConvention instance() @Override public Class getInterface() { - return null; + return DruidRel.class; } @Override public String getName() { - return null; + return NAME; } @Override @@ -57,9 +59,7 @@ public boolean canConvertConvention(Convention toConvention) } @Override - public boolean useAbstractConvertersForConversion( - RelTraitSet fromTraits, RelTraitSet toTraits - ) + public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) { return false; } @@ -67,18 +67,23 @@ public boolean useAbstractConvertersForConversion( @Override public RelTraitDef getTraitDef() { - return null; + return ConventionTraitDef.INSTANCE; } @Override public boolean satisfies(RelTrait trait) { - return false; + return trait == this; } @Override public void register(RelOptPlanner planner) { + } + @Override + public String toString() + { + return NAME; } } diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidNestedGroupBy.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidNestedGroupBy.java index e891745b61b5..29f57b4677bf 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidNestedGroupBy.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidNestedGroupBy.java @@ -19,12 +19,12 @@ package io.druid.sql.calcite.rel; -import com.google.common.base.Function; +import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Sequences; import io.druid.query.QueryDataSource; import io.druid.query.filter.DimFilter; import io.druid.sql.calcite.table.RowSignature; import org.apache.calcite.interpreter.BindableConvention; -import org.apache.calcite.interpreter.Row; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptCost; import org.apache.calcite.plan.RelOptPlanner; @@ -92,16 +92,17 @@ public DruidQueryBuilder getQueryBuilder() } @Override - public void accumulate(final Function sink) + public Sequence runQuery() { final QueryDataSource queryDataSource = sourceRel.asDataSource(); if (queryDataSource != null) { - getQueryMaker().accumulate( + return getQueryMaker().runQuery( queryDataSource, sourceRel.getOutputRowSignature(), - queryBuilder, - sink + queryBuilder ); + } else { + return Sequences.empty(); } } @@ -147,9 +148,14 @@ public DruidNestedGroupBy asBindable() } @Override - public Class getElementType() + public DruidNestedGroupBy asDruidConvention() { - return Object[].class; + return new DruidNestedGroupBy( + getCluster(), + getTraitSet().plus(DruidConvention.instance()), + sourceRel, + queryBuilder + ); } @Override diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java index a2f7afc8ebd6..38ea175937e7 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java @@ -34,6 +34,7 @@ import io.druid.query.groupby.having.DimFilterHavingSpec; import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.query.groupby.orderby.OrderByColumnSpec; +import io.druid.query.ordering.StringComparators; import io.druid.query.select.PagingSpec; import io.druid.query.select.SelectQuery; import io.druid.query.timeseries.TimeseriesQuery; @@ -102,6 +103,8 @@ private DruidQueryBuilder( valueType = ValueType.LONG; } else if (SqlTypeName.CHAR_TYPES.contains(sqlTypeName)) { valueType = ValueType.STRING; + } else if (SqlTypeName.OTHER == sqlTypeName) { + valueType = ValueType.COMPLEX; } else { throw new ISE("Cannot translate sqlTypeName[%s] to Druid type for field[%s]", sqlTypeName, rowOrder.get(i)); } @@ -115,10 +118,7 @@ private DruidQueryBuilder( public static DruidQueryBuilder fullScan(final RowSignature rowSignature, final RelDataTypeFactory relDataTypeFactory) { final RelDataType rowType = rowSignature.getRelDataType(relDataTypeFactory); - final List rowOrder = Lists.newArrayListWithCapacity(rowType.getFieldCount()); - for (RelDataTypeField field : rowType.getFieldList()) { - rowOrder.add(field.getName()); - } + final List rowOrder = rowSignature.getRowOrder(); return new DruidQueryBuilder(null, null, null, null, null, rowType, rowOrder); } @@ -369,18 +369,27 @@ public TopNQuery toTopNQuery( final boolean useApproximateTopN ) { - // Must have GROUP BY one column, ORDER BY one column, limit less than maxTopNLimit, and no HAVING. - if (grouping == null - || grouping.getDimensions().size() != 1 - || limitSpec == null - || limitSpec.getColumns().size() != 1 - || limitSpec.getLimit() > maxTopNLimit - || having != null) { + // Must have GROUP BY one column, ORDER BY zero or one column, limit less than maxTopNLimit, and no HAVING. + final boolean topNOk = grouping != null + && grouping.getDimensions().size() == 1 + && limitSpec != null + && (limitSpec.getColumns().size() <= 1 && limitSpec.getLimit() <= maxTopNLimit) + && having == null; + if (!topNOk) { return null; } final DimensionSpec dimensionSpec = Iterables.getOnlyElement(grouping.getDimensions()); - final OrderByColumnSpec limitColumn = Iterables.getOnlyElement(limitSpec.getColumns()); + final OrderByColumnSpec limitColumn; + if (limitSpec.getColumns().isEmpty()) { + limitColumn = new OrderByColumnSpec( + dimensionSpec.getOutputName(), + OrderByColumnSpec.Direction.ASCENDING, + StringComparators.LEXICOGRAPHIC + ); + } else { + limitColumn = Iterables.getOnlyElement(limitSpec.getColumns()); + } final TopNMetricSpec topNMetricSpec; if (limitColumn.getDimension().equals(dimensionSpec.getOutputName())) { diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryRel.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryRel.java index 0db39970a081..27a066930a6d 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryRel.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryRel.java @@ -19,16 +19,16 @@ package io.druid.sql.calcite.rel; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import io.druid.java.util.common.guava.Sequence; import io.druid.query.QueryDataSource; import io.druid.query.groupby.GroupByQuery; import io.druid.sql.calcite.filtration.Filtration; import io.druid.sql.calcite.table.DruidTable; import io.druid.sql.calcite.table.RowSignature; import org.apache.calcite.interpreter.BindableConvention; -import org.apache.calcite.interpreter.Row; +import org.apache.calcite.plan.Convention; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptCost; import org.apache.calcite.plan.RelOptPlanner; @@ -70,14 +70,13 @@ private DruidQueryRel( */ public static DruidQueryRel fullScan( final RelOptCluster cluster, - final RelTraitSet traitSet, final RelOptTable table, final DruidTable druidTable ) { return new DruidQueryRel( cluster, - traitSet, + cluster.traitSetOf(Convention.NONE), table, druidTable, DruidQueryBuilder.fullScan(druidTable.getRowSignature(), cluster.getTypeFactory()) @@ -114,6 +113,18 @@ public DruidQueryRel asBindable() ); } + @Override + public DruidQueryRel asDruidConvention() + { + return new DruidQueryRel( + getCluster(), + getTraitSet().replace(DruidConvention.instance()), + table, + druidTable, + queryBuilder + ); + } + @Override public RowSignature getSourceRowSignature() { @@ -145,9 +156,9 @@ public int getQueryCount() } @Override - public void accumulate(final Function sink) + public Sequence runQuery() { - getQueryMaker().accumulate(druidTable.getDataSource(), druidTable.getRowSignature(), queryBuilder, sink); + return getQueryMaker().runQuery(druidTable.getDataSource(), druidTable.getRowSignature(), queryBuilder); } @Override @@ -156,12 +167,6 @@ public RelOptTable getTable() return table; } - @Override - public Class getElementType() - { - return Object[].class; - } - @Override protected RelDataType deriveRowType() { diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidRel.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidRel.java index a4d3eda67028..8f93abd9882e 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidRel.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidRel.java @@ -19,7 +19,9 @@ package io.druid.sql.calcite.rel; -import com.google.common.base.Function; +import com.google.common.base.Throwables; +import io.druid.java.util.common.guava.Accumulator; +import io.druid.java.util.common.guava.Sequence; import io.druid.query.QueryDataSource; import io.druid.sql.calcite.table.RowSignature; import org.apache.calcite.DataContext; @@ -58,7 +60,7 @@ public final RowSignature getOutputRowSignature() */ public abstract int getQueryCount(); - public abstract void accumulate(Function sink); + public abstract Sequence runQuery(); public abstract T withQueryBuilder(DruidQueryBuilder newQueryBuilder); @@ -80,6 +82,14 @@ public QueryMaker getQueryMaker() return queryMaker; } + public abstract T asDruidConvention(); + + @Override + public Class getElementType() + { + return Object[].class; + } + @Override public Node implement(InterpreterImplementor implementor) { @@ -89,7 +99,23 @@ public Node implement(InterpreterImplementor implementor) @Override public void run() throws InterruptedException { - accumulate(QueryMaker.sinkFunction(sink)); + runQuery().accumulate( + sink, + new Accumulator() + { + @Override + public Sink accumulate(final Sink theSink, final Object[] in) + { + try { + theSink.send(Row.of(in)); + } + catch (InterruptedException e) { + throw Throwables.propagate(e); + } + return theSink; + } + } + ); } }; } diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java index 1d8ac0360e44..090b677e85ec 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java @@ -19,45 +19,37 @@ package io.druid.sql.calcite.rel; -import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import io.druid.java.util.common.ISE; -import io.druid.java.util.common.Pair; +import io.druid.java.util.common.guava.Accumulator; +import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Sequences; import io.druid.query.QueryDataSource; import io.druid.query.ResourceLimitExceededException; -import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.AndDimFilter; import io.druid.query.filter.BoundDimFilter; import io.druid.query.filter.DimFilter; import io.druid.query.filter.OrDimFilter; -import io.druid.sql.calcite.aggregation.Aggregation; import io.druid.sql.calcite.expression.RowExtraction; +import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.table.RowSignature; -import org.apache.calcite.DataContext; import org.apache.calcite.interpreter.BindableConvention; -import org.apache.calcite.interpreter.Row; -import org.apache.calcite.linq4j.Enumerable; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptCost; import org.apache.calcite.plan.RelOptPlanner; import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.RelWriter; -import org.apache.calcite.rel.core.SemiJoin; import org.apache.calcite.rel.metadata.RelMetadataQuery; import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rex.RexNode; import java.util.List; import java.util.Set; public class DruidSemiJoin extends DruidRel { - private final SemiJoin semiJoin; - private final DruidRel left; - private final DruidRel right; - private final RexNode condition; + private final DruidRel left; + private final DruidRel right; private final List leftRowExtractions; private final List rightKeys; private final int maxSemiJoinRowsInMemory; @@ -65,38 +57,31 @@ public class DruidSemiJoin extends DruidRel private DruidSemiJoin( final RelOptCluster cluster, final RelTraitSet traitSet, - final SemiJoin semiJoin, final DruidRel left, final DruidRel right, - final RexNode condition, final List leftRowExtractions, final List rightKeys, final int maxSemiJoinRowsInMemory ) { super(cluster, traitSet, left.getQueryMaker()); - this.semiJoin = semiJoin; this.left = left; this.right = right; - this.condition = condition; this.leftRowExtractions = ImmutableList.copyOf(leftRowExtractions); this.rightKeys = ImmutableList.copyOf(rightKeys); this.maxSemiJoinRowsInMemory = maxSemiJoinRowsInMemory; } public static DruidSemiJoin from( - final SemiJoin semiJoin, final DruidRel left, final DruidRel right, - final int maxSemiJoinRowsInMemory + final List leftKeys, + final List rightKeys, + final PlannerConfig plannerConfig ) { - if (semiJoin.getLeftKeys().size() != semiJoin.getRightKeys().size()) { - throw new ISE("WTF?! SemiJoin with different left/right key count?"); - } - final ImmutableList.Builder listBuilder = ImmutableList.builder(); - for (Integer key : semiJoin.getLeftKeys()) { + for (Integer key : leftKeys) { final RowExtraction rex = RowExtraction.fromQueryBuilder(left.getQueryBuilder(), key); if (rex == null) { // Can't figure out what to filter the left-hand side on... @@ -106,24 +91,16 @@ public static DruidSemiJoin from( } return new DruidSemiJoin( - semiJoin.getCluster(), - semiJoin.getTraitSet(), - semiJoin, + left.getCluster(), + left.getTraitSet(), left, right, - semiJoin.getCondition(), listBuilder.build(), - semiJoin.getRightKeys(), - maxSemiJoinRowsInMemory + rightKeys, + plannerConfig.getMaxSemiJoinRowsInMemory() ); } - @Override - public Class getElementType() - { - return Object[].class; - } - @Override public RowSignature getSourceRowSignature() { @@ -142,10 +119,8 @@ public DruidSemiJoin withQueryBuilder(final DruidQueryBuilder newQueryBuilder) return new DruidSemiJoin( getCluster(), getTraitSet().plusAll(newQueryBuilder.getRelTraits()), - semiJoin, left.withQueryBuilder(newQueryBuilder), right, - condition, leftRowExtractions, rightKeys, maxSemiJoinRowsInMemory @@ -164,11 +139,9 @@ public DruidSemiJoin asBindable() { return new DruidSemiJoin( getCluster(), - getTraitSet().plus(BindableConvention.INSTANCE), - semiJoin, + getTraitSet().replace(BindableConvention.INSTANCE), left, right, - condition, leftRowExtractions, rightKeys, maxSemiJoinRowsInMemory @@ -176,24 +149,34 @@ public DruidSemiJoin asBindable() } @Override - public int getQueryCount() + public DruidSemiJoin asDruidConvention() { - return left.getQueryCount() + right.getQueryCount(); + return new DruidSemiJoin( + getCluster(), + getTraitSet().replace(DruidConvention.instance()), + left, + right, + leftRowExtractions, + rightKeys, + maxSemiJoinRowsInMemory + ); } @Override - public void accumulate(final Function sink) + public int getQueryCount() { - final DruidRel rel = getLeftRelWithFilter(); - if (rel != null) { - rel.accumulate(sink); - } + return left.getQueryCount() + right.getQueryCount(); } @Override - public Enumerable bind(final DataContext dataContext) + public Sequence runQuery() { - throw new UnsupportedOperationException(); + final DruidRel rel = getLeftRelWithFilter(); + if (rel != null) { + return rel.runQuery(); + } else { + return Sequences.empty(); + } } @Override @@ -205,12 +188,11 @@ protected RelDataType deriveRowType() @Override public RelWriter explainTerms(RelWriter pw) { - final Pair> rightQueryBuilderWithGrouping = getRightQueryBuilderWithGrouping(); return pw .item("leftRowExtractions", leftRowExtractions) .item("leftQuery", left.getQueryBuilder()) - .item("rightKeysAdjusted", rightQueryBuilderWithGrouping.rhs) - .item("rightQuery", rightQueryBuilderWithGrouping.lhs); + .item("rightKeys", rightKeys) + .item("rightQuery", right.getQueryBuilder()); } @Override @@ -219,71 +201,26 @@ public RelOptCost computeSelfCost(final RelOptPlanner planner, final RelMetadata return right.computeSelfCost(planner, mq).plus(left.computeSelfCost(planner, mq).multiplyBy(50)); } - private Pair> getRightQueryBuilderWithGrouping() - { - if (right.getQueryBuilder().getGrouping() != null) { - return Pair.of(right.getQueryBuilder(), rightKeys); - } else { - // Add grouping on the join key to limit resultset from data nodes. - final List dimensionSpecs = Lists.newArrayList(); - final List rowTypes = Lists.newArrayList(); - final List rowOrder = Lists.newArrayList(); - final List rightKeysAdjusted = Lists.newArrayList(); - - int counter = 0; - for (final int key : rightKeys) { - final String keyDimensionOutputName = "v" + key; - final RowExtraction rex = RowExtraction.fromQueryBuilder(right.getQueryBuilder(), key); - if (rex == null) { - throw new ISE("WTF?! Can't find dimensionSpec to group on!"); - } - - final DimensionSpec dimensionSpec = rex.toDimensionSpec(left.getSourceRowSignature(), keyDimensionOutputName); - if (dimensionSpec == null) { - throw new ISE("WTF?! Can't translate row expression to dimensionSpec: %s", rex); - } - - dimensionSpecs.add(dimensionSpec); - rowTypes.add(right.getQueryBuilder().getRowType().getFieldList().get(key).getType()); - rowOrder.add(dimensionSpec.getOutputName()); - rightKeysAdjusted.add(counter++); - } - - final DruidQueryBuilder newQueryBuilder = right - .getQueryBuilder() - .withGrouping( - Grouping.create(dimensionSpecs, ImmutableList.of()), - getCluster().getTypeFactory().createStructType(rowTypes, rowOrder), - rowOrder - ); - - return Pair.of(newQueryBuilder, rightKeysAdjusted); - } - } - /** * Returns a copy of the left rel with the filter applied from the right-hand side. This is an expensive operation * since it actually executes the right-hand side query. */ - private DruidRel getLeftRelWithFilter() + private DruidRel getLeftRelWithFilter() { - final Pair> pair = getRightQueryBuilderWithGrouping(); - final DruidRel rightRelAdjusted = right.withQueryBuilder(pair.lhs); - final List rightKeysAdjusted = pair.rhs; - // Build list of acceptable values from right side. final Set> valuess = Sets.newHashSet(); final List filters = Lists.newArrayList(); - rightRelAdjusted.accumulate( - new Function() + right.runQuery().accumulate( + (Object) null, + new Accumulator() { @Override - public Void apply(final Row row) + public Object accumulate(final Object dummyValue, final Object[] row) { - final List values = Lists.newArrayListWithCapacity(rightKeysAdjusted.size()); + final List values = Lists.newArrayListWithCapacity(rightKeys.size()); - for (int i : rightKeysAdjusted) { - final Object value = row.getObject(i); + for (int i : rightKeys) { + final Object value = row[i]; final String stringValue = value != null ? String.valueOf(value) : ""; values.add(stringValue); if (values.size() > maxSemiJoinRowsInMemory) { diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java index 5a7bece0fa47..12c09fe46141 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java @@ -22,14 +22,14 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Strings; -import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.primitives.Doubles; import com.google.common.primitives.Ints; import io.druid.common.guava.GuavaUtils; import io.druid.java.util.common.ISE; -import io.druid.java.util.common.guava.Accumulator; +import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Sequences; import io.druid.query.DataSource; import io.druid.query.QueryDataSource; import io.druid.query.QuerySegmentWalker; @@ -48,15 +48,16 @@ import io.druid.segment.column.Column; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.table.RowSignature; -import org.apache.calcite.interpreter.Row; -import org.apache.calcite.interpreter.Sink; +import org.apache.calcite.avatica.ColumnMetaData; import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.runtime.Hook; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.util.NlsString; import org.joda.time.DateTime; +import java.util.ArrayList; import java.util.Calendar; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; @@ -77,29 +78,10 @@ public QueryMaker( this.plannerConfig = plannerConfig; } - public static Function sinkFunction(final Sink sink) - { - return new Function() - { - @Override - public Void apply(final Row row) - { - try { - sink.send(row); - return null; - } - catch (InterruptedException e) { - throw Throwables.propagate(e); - } - } - }; - } - - public void accumulate( + public Sequence runQuery( final DataSource dataSource, final RowSignature sourceRowSignature, - final DruidQueryBuilder queryBuilder, - final Function sink + final DruidQueryBuilder queryBuilder ) { if (dataSource instanceof QueryDataSource) { @@ -109,14 +91,12 @@ public void accumulate( throw new IllegalStateException("Can't use QueryDataSource without an outer groupBy query!"); } - executeGroupBy(queryBuilder, outerQuery, sink); - return; + return executeGroupBy(queryBuilder, outerQuery); } final TimeseriesQuery timeseriesQuery = queryBuilder.toTimeseriesQuery(dataSource, sourceRowSignature); if (timeseriesQuery != null) { - executeTimeseries(queryBuilder, timeseriesQuery, sink); - return; + return executeTimeseries(queryBuilder, timeseriesQuery); } final TopNQuery topNQuery = queryBuilder.toTopNQuery( @@ -126,214 +106,250 @@ public void accumulate( plannerConfig.isUseApproximateTopN() ); if (topNQuery != null) { - executeTopN(queryBuilder, topNQuery, sink); - return; + return executeTopN(queryBuilder, topNQuery); } final GroupByQuery groupByQuery = queryBuilder.toGroupByQuery(dataSource, sourceRowSignature); if (groupByQuery != null) { - executeGroupBy(queryBuilder, groupByQuery, sink); - return; + return executeGroupBy(queryBuilder, groupByQuery); } final SelectQuery selectQuery = queryBuilder.toSelectQuery(dataSource, sourceRowSignature); if (selectQuery != null) { - executeSelect(queryBuilder, selectQuery, sink); - return; + return executeSelect(queryBuilder, selectQuery); } throw new IllegalStateException("WTF?! Cannot execute query even though we planned it?"); } - private void executeSelect( + private Sequence executeSelect( final DruidQueryBuilder queryBuilder, - final SelectQuery baseQuery, - final Function sink + final SelectQuery baseQuery ) { Preconditions.checkState(queryBuilder.getGrouping() == null, "grouping must be null"); final List fieldList = queryBuilder.getRowType().getFieldList(); - final Row.RowBuilder rowBuilder = Row.newBuilder(fieldList.size()); final Integer limit = queryBuilder.getLimitSpec() != null ? queryBuilder.getLimitSpec().getLimit() : null; - // Loop through pages. - final AtomicBoolean morePages = new AtomicBoolean(true); - final AtomicReference> pagingIdentifiers = new AtomicReference<>(); - final AtomicLong rowsRead = new AtomicLong(); - - while (morePages.get()) { - final SelectQuery query = baseQuery.withPagingSpec( - new PagingSpec( - pagingIdentifiers.get(), - plannerConfig.getSelectThreshold(), - true - ) - ); - - Hook.QUERY_PLAN.run(query); - - morePages.set(false); - final AtomicBoolean gotResult = new AtomicBoolean(); - - query.run(walker, Maps.newHashMap()).accumulate( - (Object) null, - new Accumulator>() + // Select is paginated, we need to make multiple queries. + final Sequence> sequenceOfSequences = Sequences.simple( + new Iterable>() + { + @Override + public Iterator> iterator() { - @Override - public Object accumulate(final Object accumulated, final Result result) + final AtomicBoolean morePages = new AtomicBoolean(true); + final AtomicReference> pagingIdentifiers = new AtomicReference<>(); + final AtomicLong rowsRead = new AtomicLong(); + + // Each Sequence is one page. + return new Iterator>() { - if (!gotResult.compareAndSet(false, true)) { - throw new ISE("WTF?! Expected single result from Select query but got multiple!"); + @Override + public boolean hasNext() + { + return morePages.get(); } - pagingIdentifiers.set(result.getValue().getPagingIdentifiers()); - - for (EventHolder holder : result.getValue().getEvents()) { - morePages.set(true); - final Map map = holder.getEvent(); - for (RelDataTypeField field : fieldList) { - final String outputName = queryBuilder.getRowOrder().get(field.getIndex()); - if (outputName.equals(Column.TIME_COLUMN_NAME)) { - rowBuilder.set( - field.getIndex(), - coerce(holder.getTimestamp().getMillis(), field.getType().getSqlTypeName()) - ); - } else { - rowBuilder.set( - field.getIndex(), - coerce(map.get(outputName), field.getType().getSqlTypeName()) - ); - } - } - if (limit == null || rowsRead.incrementAndGet() <= limit) { - sink.apply(rowBuilder.build()); - } else { - morePages.set(false); - break; - } - rowBuilder.reset(); + @Override + public Sequence next() + { + final SelectQuery query = baseQuery.withPagingSpec( + new PagingSpec( + pagingIdentifiers.get(), + plannerConfig.getSelectThreshold(), + true + ) + ); + + Hook.QUERY_PLAN.run(query); + + morePages.set(false); + final AtomicBoolean gotResult = new AtomicBoolean(); + + return Sequences.concat( + Sequences.map( + query.run(walker, Maps.newHashMap()), + new Function, Sequence>() + { + @Override + public Sequence apply(final Result result) + { + if (!gotResult.compareAndSet(false, true)) { + throw new ISE("WTF?! Expected single result from Select query but got multiple!"); + } + + pagingIdentifiers.set(result.getValue().getPagingIdentifiers()); + final List retVals = new ArrayList<>(); + + for (EventHolder holder : result.getValue().getEvents()) { + morePages.set(true); + final Map map = holder.getEvent(); + final Object[] retVal = new Object[fieldList.size()]; + for (RelDataTypeField field : fieldList) { + final String outputName = queryBuilder.getRowOrder().get(field.getIndex()); + if (outputName.equals(Column.TIME_COLUMN_NAME)) { + retVal[field.getIndex()] = coerce( + holder.getTimestamp().getMillis(), + field.getType().getSqlTypeName() + ); + } else { + retVal[field.getIndex()] = coerce( + map.get(outputName), + field.getType().getSqlTypeName() + ); + } + } + if (limit == null || rowsRead.incrementAndGet() <= limit) { + retVals.add(retVal); + } else { + morePages.set(false); + return Sequences.simple(retVals); + } + } + + return Sequences.simple(retVals); + } + } + ) + ); } - return null; - } + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + }; } - ); - } + } + ); + + return Sequences.concat(sequenceOfSequences); } - private void executeTimeseries( + private Sequence executeTimeseries( final DruidQueryBuilder queryBuilder, - final TimeseriesQuery query, - final Function sink + final TimeseriesQuery query ) { final List fieldList = queryBuilder.getRowType().getFieldList(); final List dimensions = queryBuilder.getGrouping().getDimensions(); final String timeOutputName = dimensions.isEmpty() ? null : Iterables.getOnlyElement(dimensions).getOutputName(); - final Row.RowBuilder rowBuilder = Row.newBuilder(fieldList.size()); Hook.QUERY_PLAN.run(query); - query.run(walker, Maps.newHashMap()).accumulate( - (Object) null, - new Accumulator>() + return Sequences.map( + query.run(walker, Maps.newHashMap()), + new Function, Object[]>() { @Override - public Object accumulate(final Object accumulated, final Result result) + public Object[] apply(final Result result) { final Map row = result.getValue().getBaseObject(); + final Object[] retVal = new Object[fieldList.size()]; for (final RelDataTypeField field : fieldList) { final String outputName = queryBuilder.getRowOrder().get(field.getIndex()); if (outputName.equals(timeOutputName)) { - rowBuilder.set(field.getIndex(), coerce(result.getTimestamp(), field.getType().getSqlTypeName())); + retVal[field.getIndex()] = coerce(result.getTimestamp(), field.getType().getSqlTypeName()); } else { - rowBuilder.set(field.getIndex(), coerce(row.get(outputName), field.getType().getSqlTypeName())); + retVal[field.getIndex()] = coerce(row.get(outputName), field.getType().getSqlTypeName()); } } - sink.apply(rowBuilder.build()); - rowBuilder.reset(); - - return null; + return retVal; } } ); } - private void executeTopN( + private Sequence executeTopN( final DruidQueryBuilder queryBuilder, - final TopNQuery query, - final Function sink + final TopNQuery query ) { final List fieldList = queryBuilder.getRowType().getFieldList(); - final Row.RowBuilder rowBuilder = Row.newBuilder(fieldList.size()); Hook.QUERY_PLAN.run(query); - query.run(walker, Maps.newHashMap()).accumulate( - (Object) null, - new Accumulator>() - { - @Override - public Object accumulate(final Object accumulated, final Result result) - { - final List values = result.getValue().getValue(); + return Sequences.concat( + Sequences.map( + query.run(walker, Maps.newHashMap()), + new Function, Sequence>() + { + @Override + public Sequence apply(final Result result) + { + final List rows = result.getValue().getValue(); + final List retVals = new ArrayList<>(rows.size()); + + for (DimensionAndMetricValueExtractor row : rows) { + final Object[] retVal = new Object[fieldList.size()]; + for (final RelDataTypeField field : fieldList) { + final String outputName = queryBuilder.getRowOrder().get(field.getIndex()); + retVal[field.getIndex()] = coerce(row.getMetric(outputName), field.getType().getSqlTypeName()); + } - for (DimensionAndMetricValueExtractor value : values) { - for (final RelDataTypeField field : fieldList) { - final String outputName = queryBuilder.getRowOrder().get(field.getIndex()); - rowBuilder.set(field.getIndex(), coerce(value.getMetric(outputName), field.getType().getSqlTypeName())); - } + retVals.add(retVal); + } - sink.apply(rowBuilder.build()); - rowBuilder.reset(); + return Sequences.simple(retVals); + } } - - return null; - } - } + ) ); } - private void executeGroupBy( + private Sequence executeGroupBy( final DruidQueryBuilder queryBuilder, - final GroupByQuery query, - final Function sink + final GroupByQuery query ) { final List fieldList = queryBuilder.getRowType().getFieldList(); - final Row.RowBuilder rowBuilder = Row.newBuilder(fieldList.size()); Hook.QUERY_PLAN.run(query); - query.run(walker, Maps.newHashMap()).accumulate( - (Object) null, - new Accumulator() + return Sequences.map( + query.run(walker, Maps.newHashMap()), + new Function() { @Override - public Object accumulate(final Object accumulated, final io.druid.data.input.Row row) + public Object[] apply(final io.druid.data.input.Row row) { + final Object[] retVal = new Object[fieldList.size()]; for (RelDataTypeField field : fieldList) { - rowBuilder.set( - field.getIndex(), - coerce( - row.getRaw(queryBuilder.getRowOrder().get(field.getIndex())), - field.getType().getSqlTypeName() - ) + retVal[field.getIndex()] = coerce( + row.getRaw(queryBuilder.getRowOrder().get(field.getIndex())), + field.getType().getSqlTypeName() ); } - sink.apply(rowBuilder.build()); - rowBuilder.reset(); - - return null; + return retVal; } } ); } + public static ColumnMetaData.Rep rep(final SqlTypeName sqlType) + { + if (SqlTypeName.CHAR_TYPES.contains(sqlType)) { + return ColumnMetaData.Rep.of(String.class); + } else if (SqlTypeName.DATETIME_TYPES.contains(sqlType)) { + return ColumnMetaData.Rep.of(Long.class); + } else if (sqlType == SqlTypeName.INTEGER) { + return ColumnMetaData.Rep.of(Integer.class); + } else if (sqlType == SqlTypeName.BIGINT) { + return ColumnMetaData.Rep.of(Long.class); + } else if (sqlType == SqlTypeName.FLOAT || sqlType == SqlTypeName.DOUBLE) { + return ColumnMetaData.Rep.of(Double.class); + } else if (sqlType == SqlTypeName.OTHER) { + return ColumnMetaData.Rep.of(Object.class); + } else { + throw new ISE("No rep for SQL type[%s]", sqlType); + } + } + private static Object coerce(final Object value, final SqlTypeName sqlType) { final Object coercedValue; @@ -391,6 +407,9 @@ private static Object coerce(final Object value, final SqlTypeName sqlType) } else { throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType); } + } else if (sqlType == SqlTypeName.OTHER) { + // Complex type got out somehow. + coercedValue = value.getClass().getName(); } else { throw new ISE("Cannot coerce[%s] to %s", value.getClass().getName(), sqlType); } diff --git a/sql/src/main/java/io/druid/sql/calcite/rule/DruidBindableConverterRule.java b/sql/src/main/java/io/druid/sql/calcite/rule/DruidRelToBindableRule.java similarity index 77% rename from sql/src/main/java/io/druid/sql/calcite/rule/DruidBindableConverterRule.java rename to sql/src/main/java/io/druid/sql/calcite/rule/DruidRelToBindableRule.java index d9e4a89232b4..4ee1e308076c 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rule/DruidBindableConverterRule.java +++ b/sql/src/main/java/io/druid/sql/calcite/rule/DruidRelToBindableRule.java @@ -25,21 +25,21 @@ import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.convert.ConverterRule; -public class DruidBindableConverterRule extends ConverterRule +public class DruidRelToBindableRule extends ConverterRule { - private static DruidBindableConverterRule INSTANCE = new DruidBindableConverterRule(); + private static DruidRelToBindableRule INSTANCE = new DruidRelToBindableRule(); - private DruidBindableConverterRule() + private DruidRelToBindableRule() { super( DruidRel.class, Convention.NONE, BindableConvention.INSTANCE, - DruidBindableConverterRule.class.getSimpleName() + DruidRelToBindableRule.class.getSimpleName() ); } - public static DruidBindableConverterRule instance() + public static DruidRelToBindableRule instance() { return INSTANCE; } @@ -47,7 +47,6 @@ public static DruidBindableConverterRule instance() @Override public RelNode convert(RelNode rel) { - final DruidRel druidRel = (DruidRel) rel; - return druidRel.asBindable(); + return ((DruidRel) rel).asBindable(); } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/IteratorWithBaggage.java b/sql/src/main/java/io/druid/sql/calcite/rule/DruidRelToDruidRule.java similarity index 53% rename from java-util/src/main/java/io/druid/java/util/common/guava/IteratorWithBaggage.java rename to sql/src/main/java/io/druid/sql/calcite/rule/DruidRelToDruidRule.java index 061ed23c70cc..12af93fd7fb4 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/IteratorWithBaggage.java +++ b/sql/src/main/java/io/druid/sql/calcite/rule/DruidRelToDruidRule.java @@ -17,51 +17,36 @@ * under the License. */ -package io.druid.java.util.common.guava; +package io.druid.sql.calcite.rule; -import io.druid.java.util.common.parsers.CloseableIterator; +import io.druid.sql.calcite.rel.DruidConvention; +import io.druid.sql.calcite.rel.DruidRel; +import org.apache.calcite.plan.Convention; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; -import java.io.Closeable; -import java.io.IOException; -import java.util.Iterator; - -/** - */ -public class IteratorWithBaggage implements CloseableIterator +public class DruidRelToDruidRule extends ConverterRule { - private final Iterator baseIter; - private final Closeable baggage; - - public IteratorWithBaggage( - Iterator baseIter, - Closeable baggage - ) - { - this.baseIter = baseIter; - this.baggage = baggage; - } + private static DruidRelToDruidRule INSTANCE = new DruidRelToDruidRule(); - @Override - public boolean hasNext() + private DruidRelToDruidRule() { - return baseIter.hasNext(); + super( + DruidRel.class, + Convention.NONE, + DruidConvention.instance(), + DruidRelToDruidRule.class.getSimpleName() + ); } - @Override - public T next() - { - return baseIter.next(); - } - - @Override - public void remove() + public static DruidRelToDruidRule instance() { - baseIter.remove(); + return INSTANCE; } @Override - public void close() throws IOException + public RelNode convert(RelNode rel) { - baggage.close(); + return ((DruidRel) rel).asDruidConvention(); } } diff --git a/sql/src/main/java/io/druid/sql/calcite/rule/DruidSemiJoinRule.java b/sql/src/main/java/io/druid/sql/calcite/rule/DruidSemiJoinRule.java index 4de14903ea0a..8478498da911 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rule/DruidSemiJoinRule.java +++ b/sql/src/main/java/io/druid/sql/calcite/rule/DruidSemiJoinRule.java @@ -19,24 +19,73 @@ package io.druid.sql.calcite.rule; +import com.google.common.base.Predicate; +import io.druid.query.dimension.DimensionSpec; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.rel.DruidRel; import io.druid.sql.calcite.rel.DruidSemiJoin; import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptRuleCall; -import org.apache.calcite.rel.core.SemiJoin; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinInfo; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.tools.RelBuilder; +import org.apache.calcite.util.ImmutableBitSet; +import java.util.ArrayList; +import java.util.List; + +/** + * Planner rule adapted from Calcite 1.11.0's SemiJoinRule. + * + * This rule identifies a JOIN where the right-hand side is being used like a filter. Requirements are: + * + * 1) Right-hand side is grouping on the join key + * 2) No fields from the right-hand side are selected + * 3) Join is INNER (right-hand side acting as filter) or LEFT (right-hand side can be ignored) + * + * This is used instead of Calcite's built in rule because that rule's un-doing of aggregation is unproductive (we'd + * just want to add it back again). Also, this rule operates on DruidRels. + */ public class DruidSemiJoinRule extends RelOptRule { + private static final Predicate IS_LEFT_OR_INNER = + new Predicate() + { + public boolean apply(Join join) + { + final JoinRelType joinType = join.getJoinType(); + return joinType == JoinRelType.LEFT || joinType == JoinRelType.INNER; + } + }; + + private static final Predicate IS_GROUP_BY = + new Predicate() + { + public boolean apply(DruidRel druidRel) + { + return druidRel.getQueryBuilder().getGrouping() != null; + } + }; + private final PlannerConfig plannerConfig; - public DruidSemiJoinRule(final PlannerConfig plannerConfig) + private DruidSemiJoinRule(final PlannerConfig plannerConfig) { super( operand( - SemiJoin.class, - operand(DruidRel.class, none()), - operand(DruidRel.class, none()) + Project.class, + operand( + Join.class, + null, + IS_LEFT_OR_INNER, + some( + operand(DruidRel.class, any()), + operand(DruidRel.class, null, IS_GROUP_BY, any()) + ) + ) ) ); this.plannerConfig = plannerConfig; @@ -50,23 +99,66 @@ public static DruidSemiJoinRule create(final PlannerConfig plannerConfig) @Override public void onMatch(RelOptRuleCall call) { - final SemiJoin semiJoin = call.rel(0); - final DruidRel left = call.rel(1); - final DruidRel right = call.rel(2); - final DruidSemiJoin druidSemiJoin = DruidSemiJoin.from( - semiJoin, - left, - right, - plannerConfig.getMaxSemiJoinRowsInMemory() - ); + final Project project = call.rel(0); + final Join join = call.rel(1); + final DruidRel left = call.rel(2); + final DruidRel right = call.rel(3); + + final ImmutableBitSet bits = + RelOptUtil.InputFinder.bits(project.getProjects(), null); + final ImmutableBitSet rightBits = + ImmutableBitSet.range( + left.getRowType().getFieldCount(), + join.getRowType().getFieldCount() + ); + + if (bits.intersects(rightBits)) { + return; + } + + final JoinInfo joinInfo = join.analyzeCondition(); + final List rightDimsOut = new ArrayList<>(); + for (DimensionSpec dimensionSpec : right.getQueryBuilder().getGrouping().getDimensions()) { + rightDimsOut.add(right.getOutputRowSignature().getRowOrder().indexOf(dimensionSpec.getOutputName())); + } + + if (!joinInfo.isEqui() || !joinInfo.rightSet().equals(ImmutableBitSet.of(rightDimsOut))) { + // Rule requires that aggregate key to be the same as the join key. + // By the way, neither a super-set nor a sub-set would work. + return; + } + + final RelBuilder relBuilder = call.builder(); + + if (join.getJoinType() == JoinRelType.LEFT) { + // Join can be eliminated since the right-hand side cannot have any effect (nothing is being selected, + // and LEFT means even if there is no match, a left-hand row will still be included). + relBuilder.push(left); + } else { + final DruidSemiJoin druidSemiJoin = DruidSemiJoin.from( + left, + right, + joinInfo.leftKeys, + joinInfo.rightKeys, + plannerConfig + ); + + if (druidSemiJoin == null) { + return; + } - if (druidSemiJoin != null) { // Check maxQueryCount. if (plannerConfig.getMaxQueryCount() > 0 && druidSemiJoin.getQueryCount() > plannerConfig.getMaxQueryCount()) { return; } - call.transformTo(druidSemiJoin); + relBuilder.push(druidSemiJoin); } + + call.transformTo( + relBuilder + .project(project.getProjects(), project.getRowType().getFieldNames()) + .build() + ); } } diff --git a/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java b/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java index 1407aea98316..f724b6177f5e 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java +++ b/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java @@ -34,8 +34,6 @@ import io.druid.query.aggregation.LongMinAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.PostAggregator; -import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory; -import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator; import io.druid.query.aggregation.post.ArithmeticPostAggregator; import io.druid.query.aggregation.post.FieldAccessPostAggregator; import io.druid.query.dimension.DimensionSpec; @@ -47,10 +45,13 @@ import io.druid.query.ordering.StringComparator; import io.druid.query.ordering.StringComparators; import io.druid.sql.calcite.aggregation.Aggregation; +import io.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator; import io.druid.sql.calcite.aggregation.PostAggregatorFactory; +import io.druid.sql.calcite.aggregation.SqlAggregator; import io.druid.sql.calcite.expression.Expressions; import io.druid.sql.calcite.expression.RowExtraction; import io.druid.sql.calcite.filtration.Filtration; +import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.rel.DruidNestedGroupBy; import io.druid.sql.calcite.rel.DruidRel; @@ -77,17 +78,19 @@ public class GroupByRules { + private static final ApproxCountDistinctSqlAggregator APPROX_COUNT_DISTINCT = new ApproxCountDistinctSqlAggregator(); + private GroupByRules() { // No instantiation. } - public static List rules(final PlannerConfig plannerConfig) + public static List rules(final DruidOperatorTable operatorTable, final PlannerConfig plannerConfig) { return ImmutableList.of( - new DruidAggregateRule(plannerConfig), - new DruidAggregateProjectRule(plannerConfig), - new DruidAggregateProjectFilterRule(plannerConfig), + new DruidAggregateRule(operatorTable, plannerConfig), + new DruidAggregateProjectRule(operatorTable, plannerConfig), + new DruidAggregateProjectFilterRule(operatorTable, plannerConfig), new DruidGroupByPostAggregationRule(), new DruidGroupByHavingRule(), new DruidGroupByLimitRule() @@ -150,11 +153,13 @@ public String getExpression() public static class DruidAggregateRule extends RelOptRule { + private final DruidOperatorTable operatorTable; private final PlannerConfig plannerConfig; - private DruidAggregateRule(final PlannerConfig plannerConfig) + private DruidAggregateRule(final DruidOperatorTable operatorTable, final PlannerConfig plannerConfig) { super(operand(Aggregate.class, operand(DruidRel.class, none()))); + this.operatorTable = operatorTable; this.plannerConfig = plannerConfig; } @@ -176,6 +181,7 @@ public void onMatch(RelOptRuleCall call) null, null, aggregate, + operatorTable, plannerConfig.isUseApproximateCountDistinct() ); if (newDruidRel != null) { @@ -186,11 +192,13 @@ public void onMatch(RelOptRuleCall call) public static class DruidAggregateProjectRule extends RelOptRule { + private final DruidOperatorTable operatorTable; private final PlannerConfig plannerConfig; - private DruidAggregateProjectRule(final PlannerConfig plannerConfig) + private DruidAggregateProjectRule(final DruidOperatorTable operatorTable, final PlannerConfig plannerConfig) { super(operand(Aggregate.class, operand(Project.class, operand(DruidRel.class, none())))); + this.operatorTable = operatorTable; this.plannerConfig = plannerConfig; } @@ -214,6 +222,7 @@ public void onMatch(RelOptRuleCall call) null, project, aggregate, + operatorTable, plannerConfig.isUseApproximateCountDistinct() ); if (newDruidRel != null) { @@ -224,11 +233,13 @@ public void onMatch(RelOptRuleCall call) public static class DruidAggregateProjectFilterRule extends RelOptRule { + private final DruidOperatorTable operatorTable; private final PlannerConfig plannerConfig; - private DruidAggregateProjectFilterRule(final PlannerConfig plannerConfig) + private DruidAggregateProjectFilterRule(final DruidOperatorTable operatorTable, final PlannerConfig plannerConfig) { super(operand(Aggregate.class, operand(Project.class, operand(Filter.class, operand(DruidRel.class, none()))))); + this.operatorTable = operatorTable; this.plannerConfig = plannerConfig; } @@ -254,6 +265,7 @@ public void onMatch(RelOptRuleCall call) filter, project, aggregate, + operatorTable, plannerConfig.isUseApproximateCountDistinct() ); if (newDruidRel != null) { @@ -364,6 +376,7 @@ private static DruidRel applyAggregate( final Filter filter0, final Project project0, final Aggregate aggregate, + final DruidOperatorTable operatorTable, final boolean approximateCountDistinct ) { @@ -447,6 +460,8 @@ private static DruidRel applyAggregate( sourceRowSignature, project, aggCall, + operatorTable, + aggregations, i, approximateCountDistinct ); @@ -704,6 +719,8 @@ private static Aggregation translateAggregateCall( final RowSignature sourceRowSignature, final Project project, final AggregateCall call, + final DruidOperatorTable operatorTable, + final List existingAggregations, final int aggNumber, final boolean approximateCountDistinct ) @@ -731,47 +748,25 @@ private static Aggregation translateAggregateCall( filters.add(filter); } - if (call.getAggregation().getKind() == SqlKind.COUNT && call.getArgList().isEmpty()) { + if (kind == SqlKind.COUNT && call.getArgList().isEmpty()) { // COUNT(*) retVal = Aggregation.create(new CountAggregatorFactory(name)); - } else if (call.getAggregation().getKind() == SqlKind.COUNT && call.isDistinct() && approximateCountDistinct) { + } else if (kind == SqlKind.COUNT && call.isDistinct()) { // COUNT(DISTINCT x) - final RowExtraction rex = Expressions.toRowExtraction( - rowOrder, - Expressions.fromFieldAccess( - sourceRowSignature, - project, - Iterables.getOnlyElement(call.getArgList()) - ) - ); - if (rex == null) { - return null; - } - - final DimensionSpec dimensionSpec = rex.toDimensionSpec( + retVal = approximateCountDistinct ? APPROX_COUNT_DISTINCT.toDruidAggregation( + name, sourceRowSignature, - aggInternalName(aggNumber, "dimSpec") - ); - if (dimensionSpec == null) { - return null; - } - - retVal = Aggregation.createFinalizable( - ImmutableList.of( - new CardinalityAggregatorFactory(name, ImmutableList.of(dimensionSpec), false) - ), - null, - new PostAggregatorFactory() - { - @Override - public PostAggregator factorize(String outputName) - { - return new HyperUniqueFinalizingPostAggregator(outputName, name); - } - } - ); - } else if (!call.isDistinct() && call.getArgList().size() == 1) { - // AGG(xxx), not distinct, not COUNT(*) + existingAggregations, + project, + call + ) : null; + } else if (kind == SqlKind.COUNT + || kind == SqlKind.SUM + || kind == SqlKind.SUM0 + || kind == SqlKind.MIN + || kind == SqlKind.MAX + || kind == SqlKind.AVG) { + // Built-in agg, not distinct, not COUNT(*) boolean forceCount = false; final FieldOrExpression input; @@ -831,7 +826,7 @@ public PostAggregator factorize(String outputName) return null; } } else { - // Can't translate aggregator expression. + // Can't translate operand. return null; } @@ -843,7 +838,7 @@ public PostAggregator factorize(String outputName) // COUNT(x) retVal = Aggregation.create(new CountAggregatorFactory(name)); } else { - // All aggregators other than COUNT expect a single argument with no extractionFn. + // Built-in aggregator that is not COUNT. final String fieldName = input.getFieldName(); final String expression = input.getExpression(); @@ -881,11 +876,20 @@ public PostAggregator factorize(String outputName) ) ); } else { - retVal = null; + // Not reached. + throw new ISE("WTF?! Kind[%s] got into the built-in aggregator path somehow?!", kind); } } } else { - retVal = null; + // Not a built-in aggregator, check operator table. + final SqlAggregator sqlAggregator = operatorTable.lookupAggregator(call.getAggregation().getName()); + retVal = sqlAggregator != null ? sqlAggregator.toDruidAggregation( + name, + sourceRowSignature, + existingAggregations, + project, + call + ) : null; } final DimFilter filter = filters.isEmpty() diff --git a/sql/src/main/java/io/druid/sql/calcite/DruidSchema.java b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java similarity index 91% rename from sql/src/main/java/io/druid/sql/calcite/DruidSchema.java rename to sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java index 7cc99305d229..d975746dfa8a 100644 --- a/sql/src/main/java/io/druid/sql/calcite/DruidSchema.java +++ b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java @@ -17,17 +17,15 @@ * under the License. */ -package io.druid.sql.calcite; +package io.druid.sql.calcite.schema; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableMultimap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; @@ -53,10 +51,6 @@ import io.druid.sql.calcite.rel.QueryMaker; import io.druid.sql.calcite.table.DruidTable; import io.druid.timeline.DataSegment; -import org.apache.calcite.linq4j.tree.Expression; -import org.apache.calcite.schema.Function; -import org.apache.calcite.schema.Schema; -import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.schema.Table; import org.apache.calcite.schema.impl.AbstractSchema; import org.joda.time.DateTime; @@ -72,6 +66,8 @@ @ManageLifecycle public class DruidSchema extends AbstractSchema { + public static final String NAME = "druid"; + private static final EmittingLogger log = new EmittingLogger(DruidSchema.class); private final QuerySegmentWalker walker; @@ -274,42 +270,12 @@ public void awaitInitialization() throws InterruptedException initializationLatch.await(); } - @Override - public boolean isMutable() - { - return true; - } - - @Override - public boolean contentsHaveChangedSince(final long lastCheck, final long now) - { - return false; - } - - @Override - public Expression getExpression(final SchemaPlus parentSchema, final String name) - { - return super.getExpression(parentSchema, name); - } - @Override protected Map getTableMap() { return ImmutableMap.copyOf(tables); } - @Override - protected Multimap getFunctionMultimap() - { - return ImmutableMultimap.of(); - } - - @Override - protected Map getSubSchemaMap() - { - return ImmutableMap.of(); - } - private DruidTable computeTable(final String dataSource) { final SegmentMetadataQuery segmentMetadataQuery = new SegmentMetadataQuery( @@ -338,14 +304,14 @@ private DruidTable computeTable(final String dataSource) continue; } - final ValueType valueType; + ValueType valueType; try { valueType = ValueType.valueOf(entry.getValue().getType().toUpperCase()); } catch (IllegalArgumentException e) { - // Ignore unrecognized types. This includes complex types like hyperUnique, etc. - // So, that means currently they are not supported. - continue; + // Assume unrecognized types are some flavor of COMPLEX. This throws away information about exactly + // what kind of complex column it is, which we may want to preserve some day. + valueType = ValueType.COMPLEX; } columnValueTypes.put(entry.getKey(), valueType); diff --git a/sql/src/main/java/io/druid/sql/calcite/schema/InformationSchema.java b/sql/src/main/java/io/druid/sql/calcite/schema/InformationSchema.java new file mode 100644 index 000000000000..e2485bcffda2 --- /dev/null +++ b/sql/src/main/java/io/druid/sql/calcite/schema/InformationSchema.java @@ -0,0 +1,303 @@ +/* + * 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.sql.calcite.schema; + +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import io.druid.segment.column.ValueType; +import io.druid.sql.calcite.table.RowSignature; +import org.apache.calcite.DataContext; +import org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.calcite.linq4j.Enumerable; +import org.apache.calcite.linq4j.Linq4j; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.calcite.schema.ScannableTable; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.schema.Statistic; +import org.apache.calcite.schema.Statistics; +import org.apache.calcite.schema.Table; +import org.apache.calcite.schema.impl.AbstractSchema; +import org.apache.calcite.sql.type.SqlTypeName; + +import java.util.Map; +import java.util.Set; + +public class InformationSchema extends AbstractSchema +{ + public static final String NAME = "INFORMATION_SCHEMA"; + + private static final String SCHEMATA_TABLE = "SCHEMATA"; + private static final String TABLES_TABLE = "TABLES"; + private static final String COLUMNS_TABLE = "COLUMNS"; + private static final RowSignature SCHEMATA_SIGNATURE = RowSignature + .builder() + .add("CATALOG_NAME", ValueType.STRING) + .add("SCHEMA_NAME", ValueType.STRING) + .add("SCHEMA_OWNER", ValueType.STRING) + .add("DEFAULT_CHARACTER_SET_CATALOG", ValueType.STRING) + .add("DEFAULT_CHARACTER_SET_SCHEMA", ValueType.STRING) + .add("DEFAULT_CHARACTER_SET_NAME", ValueType.STRING) + .add("SQL_PATH", ValueType.STRING) + .build(); + private static final RowSignature TABLES_SIGNATURE = RowSignature + .builder() + .add("TABLE_CATALOG", ValueType.STRING) + .add("TABLE_SCHEMA", ValueType.STRING) + .add("TABLE_NAME", ValueType.STRING) + .add("TABLE_TYPE", ValueType.STRING) + .build(); + private static final RowSignature COLUMNS_SIGNATURE = RowSignature + .builder() + .add("TABLE_CATALOG", ValueType.STRING) + .add("TABLE_SCHEMA", ValueType.STRING) + .add("TABLE_NAME", ValueType.STRING) + .add("COLUMN_NAME", ValueType.STRING) + .add("ORDINAL_POSITION", ValueType.STRING) + .add("COLUMN_DEFAULT", ValueType.STRING) + .add("IS_NULLABLE", ValueType.STRING) + .add("DATA_TYPE", ValueType.STRING) + .add("CHARACTER_MAXIMUM_LENGTH", ValueType.STRING) + .add("CHARACTER_OCTET_LENGTH", ValueType.STRING) + .add("NUMERIC_PRECISION", ValueType.STRING) + .add("NUMERIC_PRECISION_RADIX", ValueType.STRING) + .add("NUMERIC_SCALE", ValueType.STRING) + .add("DATETIME_PRECISION", ValueType.STRING) + .add("CHARACTER_SET_NAME", ValueType.STRING) + .add("COLLATION_NAME", ValueType.STRING) + .add("JDBC_TYPE", ValueType.LONG) + .build(); + private static final RelDataTypeSystem TYPE_SYSTEM = RelDataTypeSystem.DEFAULT; + + private final SchemaPlus rootSchema; + private final Map tableMap; + + @Inject + public InformationSchema(final SchemaPlus rootSchema) + { + this.rootSchema = Preconditions.checkNotNull(rootSchema, "rootSchema"); + this.tableMap = ImmutableMap.of( + SCHEMATA_TABLE, new SchemataTable(), + TABLES_TABLE, new TablesTable(), + COLUMNS_TABLE, new ColumnsTable() + ); + } + + @Override + protected Map getTableMap() + { + return tableMap; + } + + class SchemataTable implements ScannableTable + { + @Override + public Enumerable scan(final DataContext root) + { + final FluentIterable results = FluentIterable + .from(rootSchema.getSubSchemaNames()) + .transform( + new Function() + { + @Override + public Object[] apply(final String schemaName) + { + final SchemaPlus subSchema = rootSchema.getSubSchema(schemaName); + return new Object[]{ + "", // CATALOG_NAME + subSchema.getName(), // SCHEMA_NAME + null, // SCHEMA_OWNER + null, // DEFAULT_CHARACTER_SET_CATALOG + null, // DEFAULT_CHARACTER_SET_SCHEMA + null, // DEFAULT_CHARACTER_SET_NAME + null // SQL_PATH + }; + } + } + ); + + return Linq4j.asEnumerable(results); + } + + @Override + public RelDataType getRowType(final RelDataTypeFactory typeFactory) + { + return SCHEMATA_SIGNATURE.getRelDataType(typeFactory); + } + + @Override + public Statistic getStatistic() + { + return Statistics.UNKNOWN; + } + + @Override + public TableType getJdbcTableType() + { + return TableType.SYSTEM_TABLE; + } + } + + class TablesTable implements ScannableTable + { + @Override + public Enumerable scan(final DataContext root) + { + final FluentIterable results = FluentIterable + .from(rootSchema.getSubSchemaNames()) + .transformAndConcat( + new Function>() + { + @Override + public Iterable apply(final String schemaName) + { + final SchemaPlus subSchema = rootSchema.getSubSchema(schemaName); + final Set tableNames = subSchema.getTableNames(); + return FluentIterable.from(tableNames).transform( + new Function() + { + @Override + public Object[] apply(final String tableName) + { + return new Object[]{ + null, // TABLE_CATALOG + schemaName, // TABLE_SCHEMA + tableName, // TABLE_NAME + subSchema.getTable(tableName).getJdbcTableType().toString() // TABLE_TYPE + }; + } + } + ); + } + } + ); + + return Linq4j.asEnumerable(results); + } + + @Override + public RelDataType getRowType(final RelDataTypeFactory typeFactory) + { + return TABLES_SIGNATURE.getRelDataType(typeFactory); + } + + @Override + public Statistic getStatistic() + { + return Statistics.UNKNOWN; + } + + @Override + public TableType getJdbcTableType() + { + return TableType.SYSTEM_TABLE; + } + } + + class ColumnsTable implements ScannableTable + { + @Override + public Enumerable scan(final DataContext root) + { + final FluentIterable results = FluentIterable + .from(rootSchema.getSubSchemaNames()) + .transformAndConcat( + new Function>() + { + @Override + public Iterable apply(final String schemaName) + { + final SchemaPlus subSchema = rootSchema.getSubSchema(schemaName); + final Set tableNames = subSchema.getTableNames(); + final JavaTypeFactoryImpl typeFactory = new JavaTypeFactoryImpl(TYPE_SYSTEM); + return FluentIterable.from(tableNames).transformAndConcat( + new Function>() + { + @Override + public Iterable apply(final String tableName) + { + return FluentIterable + .from(subSchema.getTable(tableName).getRowType(typeFactory).getFieldList()) + .transform( + new Function() + { + @Override + public Object[] apply(final RelDataTypeField field) + { + final RelDataType type = field.getType(); + boolean isNumeric = SqlTypeName.NUMERIC_TYPES.contains(type.getSqlTypeName()); + boolean isCharacter = SqlTypeName.CHAR_TYPES.contains(type.getSqlTypeName()); + boolean isDateTime = SqlTypeName.DATETIME_TYPES.contains(type.getSqlTypeName()); + return new Object[]{ + "", // TABLE_CATALOG + schemaName, // TABLE_SCHEMA + tableName, // TABLE_NAME + field.getName(), // COLUMN_NAME + String.valueOf(field.getIndex()), // ORDINAL_POSITION + "", // COLUMN_DEFAULT + type.isNullable() ? "YES" : "NO", // IS_NULLABLE + type.getSqlTypeName().toString(), // DATA_TYPE + null, // CHARACTER_MAXIMUM_LENGTH + null, // CHARACTER_OCTET_LENGTH + isNumeric ? String.valueOf(type.getPrecision()) : null, // NUMERIC_PRECISION + isNumeric ? "10" : null, // NUMERIC_PRECISION_RADIX + isNumeric ? String.valueOf(type.getScale()) : null, // NUMERIC_SCALE + isDateTime ? String.valueOf(type.getPrecision()) : null, // DATETIME_PRECISION + isCharacter ? type.getCharset().name() : null, // CHARACTER_SET_NAME + isCharacter ? type.getCollation().getCollationName() : null, // COLLATION_NAME + type.getSqlTypeName().getJdbcOrdinal() // JDBC_TYPE (Druid extension) + }; + } + } + ); + } + } + ); + } + } + ); + + return Linq4j.asEnumerable(results); + } + + @Override + public RelDataType getRowType(final RelDataTypeFactory typeFactory) + { + return COLUMNS_SIGNATURE.getRelDataType(typeFactory); + } + + @Override + public Statistic getStatistic() + { + return Statistics.UNKNOWN; + } + + @Override + public TableType getJdbcTableType() + { + return TableType.SYSTEM_TABLE; + } + } +} diff --git a/sql/src/main/java/io/druid/sql/calcite/table/DruidTable.java b/sql/src/main/java/io/druid/sql/calcite/table/DruidTable.java index 374ef492fae0..34f2c8b82c0b 100644 --- a/sql/src/main/java/io/druid/sql/calcite/table/DruidTable.java +++ b/sql/src/main/java/io/druid/sql/calcite/table/DruidTable.java @@ -96,12 +96,7 @@ public RelDataType getRowType(final RelDataTypeFactory typeFactory) public RelNode toRel(final RelOptTable.ToRelContext context, final RelOptTable table) { final RelOptCluster cluster = context.getCluster(); - return DruidQueryRel.fullScan( - cluster, - cluster.traitSet(), - table, - this - ); + return DruidQueryRel.fullScan(cluster, table, this); } @Override diff --git a/sql/src/main/java/io/druid/sql/calcite/table/RowSignature.java b/sql/src/main/java/io/druid/sql/calcite/table/RowSignature.java index cadc4a6f01c4..68e2859146ad 100644 --- a/sql/src/main/java/io/druid/sql/calcite/table/RowSignature.java +++ b/sql/src/main/java/io/druid/sql/calcite/table/RowSignature.java @@ -32,8 +32,10 @@ import io.druid.segment.column.Column; import io.druid.segment.column.ValueType; import io.druid.sql.calcite.expression.RowExtraction; +import io.druid.sql.calcite.planner.Calcites; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.sql.SqlCollation; import org.apache.calcite.sql.type.SqlTypeName; import java.util.List; @@ -118,31 +120,38 @@ public StringComparator naturalStringComparator(final RowExtraction rowExtractio public RelDataType getRelDataType(final RelDataTypeFactory typeFactory) { final RelDataTypeFactory.FieldInfoBuilder builder = typeFactory.builder(); - for (int columnNumber = 0; columnNumber < columnNames.size(); columnNumber++) { - final String columnName = columnNames.get(columnNumber); + for (final String columnName : columnNames) { final ValueType columnType = getColumnType(columnName); - final RelDataType sqlTypeName; + final RelDataType type; if (Column.TIME_COLUMN_NAME.equals(columnName)) { - sqlTypeName = typeFactory.createSqlType(SqlTypeName.TIMESTAMP); + type = typeFactory.createSqlType(SqlTypeName.TIMESTAMP); } else { switch (columnType) { case STRING: // Note that there is no attempt here to handle multi-value in any special way. Maybe one day... - sqlTypeName = typeFactory.createSqlType(SqlTypeName.VARCHAR, RelDataType.PRECISION_NOT_SPECIFIED); + type = typeFactory.createTypeWithCharsetAndCollation( + typeFactory.createSqlType(SqlTypeName.VARCHAR), + Calcites.defaultCharset(), + SqlCollation.IMPLICIT + ); break; case LONG: - sqlTypeName = typeFactory.createSqlType(SqlTypeName.BIGINT); + type = typeFactory.createSqlType(SqlTypeName.BIGINT); break; case FLOAT: - sqlTypeName = typeFactory.createSqlType(SqlTypeName.FLOAT); + type = typeFactory.createSqlType(SqlTypeName.FLOAT); + break; + case COMPLEX: + // Loses information about exactly what kind of complex column this is. + type = typeFactory.createSqlType(SqlTypeName.OTHER); break; default: throw new ISE("WTF?! valueType[%s] not translatable?", columnType); } } - builder.add(columnName, sqlTypeName); + builder.add(columnName, type); } return builder.build(); diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/YieldSign.java b/sql/src/main/java/io/druid/sql/guice/SqlBindings.java similarity index 62% rename from java-util/src/main/java/io/druid/java/util/common/guava/YieldSign.java rename to sql/src/main/java/io/druid/sql/guice/SqlBindings.java index d40b70652aa0..5ab705c044ac 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/YieldSign.java +++ b/sql/src/main/java/io/druid/sql/guice/SqlBindings.java @@ -17,11 +17,20 @@ * under the License. */ -package io.druid.java.util.common.guava; +package io.druid.sql.guice; -/** - */ -public interface YieldSign +import com.google.inject.Binder; +import com.google.inject.multibindings.Multibinder; +import io.druid.sql.calcite.aggregation.SqlAggregator; + +public class SqlBindings { - public T yield(T toYield); + public static void addAggregator( + final Binder binder, + final Class aggregatorClass + ) + { + final Multibinder setBinder = Multibinder.newSetBinder(binder, SqlAggregator.class); + setBinder.addBinding().to(aggregatorClass); + } } diff --git a/sql/src/main/java/io/druid/sql/guice/SqlModule.java b/sql/src/main/java/io/druid/sql/guice/SqlModule.java index c8cb793a9897..e285b4b4360c 100644 --- a/sql/src/main/java/io/druid/sql/guice/SqlModule.java +++ b/sql/src/main/java/io/druid/sql/guice/SqlModule.java @@ -31,21 +31,22 @@ import io.druid.server.initialization.jetty.JettyBindings; import io.druid.server.metrics.MetricsModule; import io.druid.sql.avatica.AvaticaMonitor; +import io.druid.sql.avatica.AvaticaServerConfig; import io.druid.sql.avatica.DruidAvaticaHandler; -import io.druid.sql.calcite.DruidSchema; +import io.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerConfig; +import io.druid.sql.calcite.schema.DruidSchema; import io.druid.sql.http.SqlResource; -import org.apache.calcite.jdbc.CalciteConnection; +import org.apache.calcite.schema.SchemaPlus; -import java.sql.SQLException; import java.util.Properties; public class SqlModule implements Module { private static final String PROPERTY_SQL_ENABLE = "druid.sql.enable"; - private static final String PROPERTY_SQL_ENABLE_JSON_OVER_HTTP = "druid.sql.server.enableJsonOverHttp"; - private static final String PROPERTY_SQL_ENABLE_AVATICA = "druid.sql.server.enableAvatica"; + private static final String PROPERTY_SQL_ENABLE_JSON_OVER_HTTP = "druid.sql.http.enable"; + private static final String PROPERTY_SQL_ENABLE_AVATICA = "druid.sql.avatica.enable"; @Inject private Properties props; @@ -58,8 +59,12 @@ public SqlModule() public void configure(Binder binder) { if (isEnabled()) { + Calcites.setSystemProperties(); + JsonConfigProvider.bind(binder, "druid.sql.planner", PlannerConfig.class); + JsonConfigProvider.bind(binder, "druid.sql.avatica", AvaticaServerConfig.class); LifecycleModule.register(binder, DruidSchema.class); + SqlBindings.addAggregator(binder, ApproxCountDistinctSqlAggregator.class); if (isJsonOverHttpEnabled()) { Jerseys.addResource(binder, SqlResource.class); @@ -74,15 +79,12 @@ public void configure(Binder binder) } @Provides - public CalciteConnection createCalciteConnection( - final DruidSchema druidSchema, - final PlannerConfig plannerConfig - ) throws SQLException + public SchemaPlus createRootSchema(final DruidSchema druidSchema) { if (isEnabled()) { - return Calcites.jdbc(druidSchema, plannerConfig); + return Calcites.createRootSchema(druidSchema); } else { - throw new IllegalStateException("Cannot provide CalciteConnection when SQL is disabled."); + throw new IllegalStateException("Cannot provide SchemaPlus when SQL is disabled."); } } diff --git a/sql/src/main/java/io/druid/sql/http/SqlResource.java b/sql/src/main/java/io/druid/sql/http/SqlResource.java index cb6105c54f57..060fdd6eb95b 100644 --- a/sql/src/main/java/io/druid/sql/http/SqlResource.java +++ b/sql/src/main/java/io/druid/sql/http/SqlResource.java @@ -22,12 +22,20 @@ import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; import com.google.inject.Inject; import io.druid.guice.annotations.Json; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.guava.Yielder; +import io.druid.java.util.common.guava.Yielders; import io.druid.java.util.common.logger.Logger; import io.druid.query.QueryInterruptedException; -import org.apache.calcite.jdbc.CalciteConnection; +import io.druid.sql.calcite.planner.Calcites; +import io.druid.sql.calcite.planner.PlannerFactory; +import io.druid.sql.calcite.planner.PlannerResult; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.tools.Planner; import org.joda.time.DateTime; import javax.ws.rs.Consumes; @@ -40,11 +48,8 @@ import javax.ws.rs.core.StreamingOutput; import java.io.IOException; import java.io.OutputStream; -import java.sql.Connection; -import java.sql.ResultSet; -import java.sql.ResultSetMetaData; import java.sql.SQLException; -import java.sql.Types; +import java.util.List; @Path("/druid/v2/sql/") public class SqlResource @@ -52,16 +57,16 @@ public class SqlResource private static final Logger log = new Logger(SqlResource.class); private final ObjectMapper jsonMapper; - private final Connection connection; + private final PlannerFactory plannerFactory; @Inject public SqlResource( @Json ObjectMapper jsonMapper, - CalciteConnection connection + PlannerFactory plannerFactory ) { this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper"); - this.connection = Preconditions.checkNotNull(connection, "connection"); + this.plannerFactory = Preconditions.checkNotNull(plannerFactory, "connection"); } @POST @@ -72,81 +77,79 @@ public Response doPost(final SqlQuery sqlQuery) throws SQLException, IOException // This is not integrated with the experimental authorization framework. // (Non-trivial since we don't know the dataSources up-front) - try { - final ResultSet resultSet = connection.createStatement().executeQuery(sqlQuery.getQuery()); - final ResultSetMetaData metaData = resultSet.getMetaData(); - - // Remember which columns are time-typed, so we can emit ISO8601 instead of millis values. - final boolean[] timeColumns = new boolean[metaData.getColumnCount()]; - for (int i = 0; i < metaData.getColumnCount(); i++) { - final int columnType = metaData.getColumnType(i + 1); - if (columnType == Types.TIMESTAMP || columnType == Types.TIME || columnType == Types.DATE) { - timeColumns[i] = true; - } else { - timeColumns[i] = false; - } - } - - return Response.ok( - new StreamingOutput() - { - @Override - public void write(final OutputStream outputStream) throws IOException, WebApplicationException - { - try (final JsonGenerator jsonGenerator = jsonMapper.getFactory().createGenerator(outputStream)) { - jsonGenerator.writeStartArray(); - while (resultSet.next()) { - jsonGenerator.writeStartObject(); - for (int i = 0; i < metaData.getColumnCount(); i++) { - final Object value; - - if (timeColumns[i]) { - value = new DateTime(resultSet.getLong(i + 1)); - } else { - value = resultSet.getObject(i + 1); - } - - jsonGenerator.writeObjectField(metaData.getColumnLabel(i + 1), value); - } - jsonGenerator.writeEndObject(); - } - jsonGenerator.writeEndArray(); - jsonGenerator.flush(); + final PlannerResult plannerResult; - // End with CRLF - outputStream.write('\r'); - outputStream.write('\n'); - } - catch (SQLException e) { - throw Throwables.propagate(e); - } - finally { - try { - resultSet.close(); - } - catch (SQLException e) { - log.warn(e, "Failed to close ResultSet, ignoring."); - } - } - } - } - ).build(); + try (final Planner planner = plannerFactory.createPlanner()) { + plannerResult = Calcites.plan(planner, sqlQuery.getQuery()); } catch (Exception e) { log.warn(e, "Failed to handle query: %s", sqlQuery); - // Unwrap preparing exceptions into potentially more useful exceptions. - final Throwable maybeUnwrapped; - if (e instanceof SQLException && e.getMessage().contains("Error while preparing statement")) { - maybeUnwrapped = e.getCause(); + final Exception exceptionToReport; + + if (e instanceof RelOptPlanner.CannotPlanException) { + exceptionToReport = new ISE("Cannot build plan for query: %s", sqlQuery.getQuery()); } else { - maybeUnwrapped = e; + exceptionToReport = e; } return Response.serverError() .type(MediaType.APPLICATION_JSON_TYPE) - .entity(jsonMapper.writeValueAsBytes(QueryInterruptedException.wrapIfNeeded(maybeUnwrapped))) + .entity(jsonMapper.writeValueAsBytes(QueryInterruptedException.wrapIfNeeded(exceptionToReport))) .build(); } + + // Remember which columns are time-typed, so we can emit ISO8601 instead of millis values. + final List fieldList = plannerResult.rowType().getFieldList(); + final boolean[] timeColumns = new boolean[fieldList.size()]; + for (int i = 0; i < fieldList.size(); i++) { + final SqlTypeName sqlTypeName = fieldList.get(i).getType().getSqlTypeName(); + timeColumns[i] = SqlTypeName.DATETIME_TYPES.contains(sqlTypeName); + } + + final Yielder yielder0 = Yielders.each(plannerResult.run()); + + return Response.ok( + new StreamingOutput() + { + @Override + public void write(final OutputStream outputStream) throws IOException, WebApplicationException + { + Yielder yielder = yielder0; + + try (final JsonGenerator jsonGenerator = jsonMapper.getFactory().createGenerator(outputStream)) { + jsonGenerator.writeStartArray(); + + while (!yielder.isDone()) { + final Object[] row = yielder.get(); + jsonGenerator.writeStartObject(); + for (int i = 0; i < fieldList.size(); i++) { + final Object value; + + if (timeColumns[i]) { + value = new DateTime((long) row[i]); + } else { + value = row[i]; + } + + jsonGenerator.writeObjectField(fieldList.get(i).getName(), value); + } + jsonGenerator.writeEndObject(); + yielder = yielder.next(null); + } + + jsonGenerator.writeEndArray(); + jsonGenerator.flush(); + + // End with CRLF + outputStream.write('\r'); + outputStream.write('\n'); + } + finally { + yielder.close(); + } + } + } + ).build(); } } diff --git a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java index 33dbac57985c..7d91102a1f65 100644 --- a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -27,10 +27,16 @@ import io.druid.java.util.common.Pair; import io.druid.server.DruidNode; import io.druid.sql.calcite.planner.Calcites; +import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; +import io.druid.sql.calcite.planner.PlannerFactory; import io.druid.sql.calcite.util.CalciteTests; -import org.apache.calcite.jdbc.CalciteConnection; +import io.druid.sql.calcite.util.QueryLogHook; +import io.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; +import org.apache.calcite.avatica.AvaticaClientRuntimeException; +import org.apache.calcite.schema.SchemaPlus; import org.eclipse.jetty.server.Server; +import org.joda.time.DateTime; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -46,6 +52,8 @@ import java.sql.ResultSet; import java.sql.ResultSetMetaData; import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Types; import java.util.List; import java.util.Map; import java.util.Random; @@ -53,29 +61,49 @@ public class DruidAvaticaHandlerTest { + private static final AvaticaServerConfig AVATICA_CONFIG = new AvaticaServerConfig() + { + @Override + public int getMaxConnections() + { + return 2; + } + + @Override + public int getMaxStatementsPerConnection() + { + return 2; + } + }; + @Rule public ExpectedException expectedException = ExpectedException.none(); @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private CalciteConnection serverConnection; + @Rule + public QueryLogHook queryLogHook = QueryLogHook.create(); + + private SpecificSegmentsQuerySegmentWalker walker; private Server server; private Connection client; @Before public void setUp() throws Exception { + Calcites.setSystemProperties(); + walker = CalciteTests.createMockWalker(temporaryFolder.newFolder()); final PlannerConfig plannerConfig = new PlannerConfig(); - serverConnection = Calcites.jdbc( + final SchemaPlus rootSchema = Calcites.createRootSchema( CalciteTests.createMockSchema( - CalciteTests.createWalker(temporaryFolder.newFolder()), + walker, plannerConfig - ), - plannerConfig + ) ); + final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final DruidAvaticaHandler handler = new DruidAvaticaHandler( - serverConnection, + new DruidMeta(new PlannerFactory(rootSchema, operatorTable, plannerConfig), AVATICA_CONFIG), new DruidNode("dummy", "dummy", 1), new AvaticaMonitor() ); @@ -96,10 +124,10 @@ public void tearDown() throws Exception { client.close(); server.stop(); - serverConnection.close(); + walker.close(); + walker = null; client = null; server = null; - serverConnection = null; } @Test @@ -115,18 +143,46 @@ public void testSelectCount() throws Exception ); } + @Test + public void testSelectCountAlternateStyle() throws Exception + { + final ResultSet resultSet = client.prepareStatement("SELECT COUNT(*) AS cnt FROM druid.foo").executeQuery(); + final List> rows = getRows(resultSet); + Assert.assertEquals( + ImmutableList.of( + ImmutableMap.of("cnt", 6L) + ), + rows + ); + } + + @Test + public void testTimestampsInResponse() throws Exception + { + final ResultSet resultSet = client.createStatement().executeQuery( + "SELECT __time FROM druid.foo LIMIT 1" + ); + + Assert.assertEquals( + ImmutableList.of( + ImmutableMap.of("__time", new DateTime("2000-01-01T00:00:00.000Z").toDate()) + ), + getRows(resultSet) + ); + } + @Test public void testFieldAliasingSelect() throws Exception { final ResultSet resultSet = client.createStatement().executeQuery( "SELECT dim2 AS \"x\", dim2 AS \"y\" FROM druid.foo LIMIT 1" ); - final List> rows = getRows(resultSet); + Assert.assertEquals( ImmutableList.of( ImmutableMap.of("x", "a", "y", "a") ), - rows + getRows(resultSet) ); } @@ -136,16 +192,27 @@ public void testExplainSelectCount() throws Exception final ResultSet resultSet = client.createStatement().executeQuery( "EXPLAIN PLAN FOR SELECT COUNT(*) AS cnt FROM druid.foo" ); - final List> rows = getRows(resultSet); + Assert.assertEquals( ImmutableList.of( ImmutableMap.of( "PLAN", - "EnumerableInterpreter\n" - + " DruidQueryRel(dataSource=[foo], dimensions=[[]], aggregations=[[Aggregation{aggregatorFactories=[CountAggregatorFactory{name='a0'}], postAggregator=null, finalizingPostAggregatorFactory=null}]])\n" + "DruidQueryRel(dataSource=[foo], dimensions=[[]], aggregations=[[Aggregation{aggregatorFactories=[CountAggregatorFactory{name='a0'}], postAggregator=null, finalizingPostAggregatorFactory=null}]])\n" ) ), - rows + getRows(resultSet) + ); + } + + @Test + public void testDatabaseMetaDataCatalogs() throws Exception + { + final DatabaseMetaData metaData = client.getMetaData(); + Assert.assertEquals( + ImmutableList.of( + ROW(Pair.of("TABLE_CAT", "")) + ), + getRows(metaData.getCatalogs()) ); } @@ -155,7 +222,7 @@ public void testDatabaseMetaDataSchemas() throws Exception final DatabaseMetaData metaData = client.getMetaData(); Assert.assertEquals( ImmutableList.of( - ROW(Pair.of("TABLE_CATALOG", null), Pair.of("TABLE_SCHEM", "druid")) + ROW(Pair.of("TABLE_CATALOG", ""), Pair.of("TABLE_SCHEM", "druid")) ), getRows(metaData.getSchemas(null, "druid")) ); @@ -172,6 +239,12 @@ public void testDatabaseMetaDataTables() throws Exception Pair.of("TABLE_NAME", "foo"), Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_TYPE", "TABLE") + ), + ROW( + Pair.of("TABLE_CAT", null), + Pair.of("TABLE_NAME", "foo2"), + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_TYPE", "TABLE") ) ), getRows( @@ -185,57 +258,87 @@ public void testDatabaseMetaDataTables() throws Exception public void testDatabaseMetaDataColumns() throws Exception { final DatabaseMetaData metaData = client.getMetaData(); - final String varcharDescription = "VARCHAR(1) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\" NOT NULL"; Assert.assertEquals( ImmutableList.of( ROW( Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_NAME", "foo"), Pair.of("COLUMN_NAME", "__time"), - Pair.of("DATA_TYPE", 93), - Pair.of("TYPE_NAME", "TIMESTAMP(0) NOT NULL"), + Pair.of("DATA_TYPE", Types.TIMESTAMP), + Pair.of("TYPE_NAME", "TIMESTAMP"), Pair.of("IS_NULLABLE", "NO") ), ROW( Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_NAME", "foo"), Pair.of("COLUMN_NAME", "cnt"), - Pair.of("DATA_TYPE", -5), - Pair.of("TYPE_NAME", "BIGINT NOT NULL"), + Pair.of("DATA_TYPE", Types.BIGINT), + Pair.of("TYPE_NAME", "BIGINT"), Pair.of("IS_NULLABLE", "NO") ), ROW( Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_NAME", "foo"), Pair.of("COLUMN_NAME", "dim1"), - Pair.of("DATA_TYPE", 12), - Pair.of("TYPE_NAME", varcharDescription), + Pair.of("DATA_TYPE", Types.VARCHAR), + Pair.of("TYPE_NAME", "VARCHAR"), Pair.of("IS_NULLABLE", "NO") ), ROW( Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_NAME", "foo"), Pair.of("COLUMN_NAME", "dim2"), - Pair.of("DATA_TYPE", 12), - Pair.of("TYPE_NAME", varcharDescription), + Pair.of("DATA_TYPE", Types.VARCHAR), + Pair.of("TYPE_NAME", "VARCHAR"), Pair.of("IS_NULLABLE", "NO") ), ROW( Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_NAME", "foo"), Pair.of("COLUMN_NAME", "m1"), - Pair.of("DATA_TYPE", 6), - Pair.of("TYPE_NAME", "FLOAT NOT NULL"), + Pair.of("DATA_TYPE", Types.FLOAT), + Pair.of("TYPE_NAME", "FLOAT"), + Pair.of("IS_NULLABLE", "NO") + ), + ROW( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", "foo"), + Pair.of("COLUMN_NAME", "unique_dim1"), + Pair.of("DATA_TYPE", Types.OTHER), + Pair.of("TYPE_NAME", "OTHER"), Pair.of("IS_NULLABLE", "NO") ) ), getRows( - metaData.getColumns(null, "druid", "foo", "%"), + metaData.getColumns(null, "dr_id", "foo", null), ImmutableSet.of("IS_NULLABLE", "TABLE_NAME", "TABLE_SCHEM", "COLUMN_NAME", "DATA_TYPE", "TYPE_NAME") ) ); } + @Test + public void testTooManyStatements() throws Exception + { + final Statement statement1 = client.createStatement(); + final Statement statement2 = client.createStatement(); + + expectedException.expect(AvaticaClientRuntimeException.class); + expectedException.expectMessage("Too many open statements, limit is[2]"); + final Statement statement3 = client.createStatement(); + } + + @Test + public void testNotTooManyStatementsWhenYouCloseThem() throws Exception + { + client.createStatement().close(); + client.createStatement().close(); + client.createStatement().close(); + client.createStatement().close(); + client.createStatement().close(); + + Assert.assertTrue(true); + } + private static List> getRows(final ResultSet resultSet) throws SQLException { return getRows(resultSet, null); diff --git a/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java b/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java new file mode 100644 index 000000000000..f7355def9938 --- /dev/null +++ b/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java @@ -0,0 +1,180 @@ +/* + * 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.sql.avatica; + +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import io.druid.sql.calcite.planner.Calcites; +import io.druid.sql.calcite.planner.DruidOperatorTable; +import io.druid.sql.calcite.planner.PlannerConfig; +import io.druid.sql.calcite.planner.PlannerFactory; +import io.druid.sql.calcite.util.CalciteTests; +import io.druid.sql.calcite.util.QueryLogHook; +import io.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; +import org.apache.calcite.avatica.ColumnMetaData; +import org.apache.calcite.avatica.Meta; +import org.apache.calcite.schema.SchemaPlus; +import org.joda.time.DateTime; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.util.List; + +public class DruidStatementTest +{ + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Rule + public QueryLogHook queryLogHook = QueryLogHook.create(); + + private SpecificSegmentsQuerySegmentWalker walker; + private PlannerFactory plannerFactory; + + @Before + public void setUp() throws Exception + { + Calcites.setSystemProperties(); + walker = CalciteTests.createMockWalker(temporaryFolder.newFolder()); + final PlannerConfig plannerConfig = new PlannerConfig(); + final SchemaPlus rootSchema = Calcites.createRootSchema( + CalciteTests.createMockSchema( + walker, + plannerConfig + ) + ); + final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); + plannerFactory = new PlannerFactory(rootSchema, operatorTable, plannerConfig); + } + + @After + public void tearDown() throws Exception + { + walker.close(); + walker = null; + } + + @Test + public void testSignature() throws Exception + { + final String sql = "SELECT * FROM druid.foo"; + final DruidStatement statement = new DruidStatement("", 0).prepare(plannerFactory, sql, -1); + + // Check signature. + final Meta.Signature signature = statement.getSignature(); + Assert.assertEquals(Meta.CursorFactory.ARRAY, signature.cursorFactory); + Assert.assertEquals(Meta.StatementType.SELECT, signature.statementType); + Assert.assertEquals(sql, signature.sql); + Assert.assertEquals( + Lists.newArrayList( + Lists.newArrayList("__time", "TIMESTAMP", "java.lang.Long"), + Lists.newArrayList("cnt", "BIGINT", "java.lang.Long"), + Lists.newArrayList("dim1", "VARCHAR", "java.lang.String"), + Lists.newArrayList("dim2", "VARCHAR", "java.lang.String"), + Lists.newArrayList("m1", "FLOAT", "java.lang.Double"), + Lists.newArrayList("unique_dim1", "OTHER", "java.lang.Object") + ), + Lists.transform( + signature.columns, + new Function>() + { + @Override + public List apply(final ColumnMetaData columnMetaData) + { + return Lists.newArrayList( + columnMetaData.label, + columnMetaData.type.name, + columnMetaData.type.rep.clazz.getName() + ); + } + } + ) + ); + } + + @Test + public void testSelectAllInFirstFrame() throws Exception + { + final String sql = "SELECT __time, cnt, dim1, dim2, m1 FROM druid.foo"; + final DruidStatement statement = new DruidStatement("", 0).prepare(plannerFactory, sql, -1); + + // First frame, ask for all rows. + Meta.Frame frame = statement.execute().nextFrame(DruidStatement.START_OFFSET, 6); + Assert.assertEquals( + Meta.Frame.create( + 0, + true, + Lists.newArrayList( + new Object[]{new DateTime("2000-01-01").getMillis(), 1L, "", "a", 1.0}, + new Object[]{new DateTime("2000-01-02").getMillis(), 1L, "10.1", "", 2.0}, + new Object[]{new DateTime("2000-01-03").getMillis(), 1L, "2", "", 3.0}, + new Object[]{new DateTime("2001-01-01").getMillis(), 1L, "1", "a", 4.0}, + new Object[]{new DateTime("2001-01-02").getMillis(), 1L, "def", "abc", 5.0}, + new Object[]{new DateTime("2001-01-03").getMillis(), 1L, "abc", "", 6.0} + ) + ), + frame + ); + Assert.assertTrue(statement.isDone()); + } + + @Test + public void testSelectSplitOverTwoFrames() throws Exception + { + final String sql = "SELECT __time, cnt, dim1, dim2, m1 FROM druid.foo"; + final DruidStatement statement = new DruidStatement("", 0).prepare(plannerFactory, sql, -1); + + // First frame, ask for 2 rows. + Meta.Frame frame = statement.execute().nextFrame(DruidStatement.START_OFFSET, 2); + Assert.assertEquals( + Meta.Frame.create( + 0, + false, + Lists.newArrayList( + new Object[]{new DateTime("2000-01-01").getMillis(), 1L, "", "a", 1.0}, + new Object[]{new DateTime("2000-01-02").getMillis(), 1L, "10.1", "", 2.0} + ) + ), + frame + ); + Assert.assertFalse(statement.isDone()); + + // Last frame, ask for all remaining rows. + frame = statement.nextFrame(2, 10); + Assert.assertEquals( + Meta.Frame.create( + 2, + true, + Lists.newArrayList( + new Object[]{new DateTime("2000-01-03").getMillis(), 1L, "2", "", 3.0}, + new Object[]{new DateTime("2001-01-01").getMillis(), 1L, "1", "a", 4.0}, + new Object[]{new DateTime("2001-01-02").getMillis(), 1L, "def", "abc", 5.0}, + new Object[]{new DateTime("2001-01-03").getMillis(), 1L, "abc", "", 6.0} + ) + ), + frame + ); + Assert.assertTrue(statement.isDone()); + } +} diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java index 8ba69b68a9ee..b1ce7dedf562 100644 --- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java @@ -19,20 +19,15 @@ package io.druid.sql.calcite; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import io.druid.granularity.QueryGranularities; -import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.query.Druids; import io.druid.query.Query; import io.druid.query.QueryDataSource; -import io.druid.query.QuerySegmentWalker; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleMaxAggregatorFactory; @@ -44,7 +39,9 @@ import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory; +import io.druid.query.aggregation.hyperloglog.HLLCV1; import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.post.ArithmeticPostAggregator; import io.druid.query.aggregation.post.ConstantPostAggregator; import io.druid.query.aggregation.post.ExpressionPostAggregator; @@ -82,12 +79,16 @@ import io.druid.segment.column.Column; import io.druid.sql.calcite.filtration.Filtration; import io.druid.sql.calcite.planner.Calcites; +import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; +import io.druid.sql.calcite.planner.PlannerFactory; +import io.druid.sql.calcite.planner.PlannerResult; import io.druid.sql.calcite.util.CalciteTests; +import io.druid.sql.calcite.util.QueryLogHook; import io.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; -import org.apache.calcite.jdbc.CalciteConnection; import org.apache.calcite.plan.RelOptPlanner; -import org.apache.calcite.runtime.Hook; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.tools.Planner; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.After; @@ -97,12 +98,7 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.sql.Connection; -import java.sql.ResultSet; -import java.sql.ResultSetMetaData; -import java.sql.SQLException; import java.util.Arrays; -import java.util.Date; import java.util.List; import java.util.Map; @@ -110,7 +106,6 @@ public class CalciteQueryTest { private static final Logger log = new Logger(CalciteQueryTest.class); - private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); private static final PlannerConfig PLANNER_CONFIG_DEFAULT = new PlannerConfig(); private static final PlannerConfig PLANNER_CONFIG_NO_TOPN = new PlannerConfig() { @@ -153,7 +148,7 @@ public int getMaxQueryCount() } }; - private static final Map TIMESERIES_CONTEXT = ImmutableMap.of( + public static final Map TIMESERIES_CONTEXT = ImmutableMap.of( "skipEmptyBuckets", true ); @@ -162,61 +157,23 @@ public int getMaxQueryCount() @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule + public QueryLogHook queryLogHook = QueryLogHook.create(); + private SpecificSegmentsQuerySegmentWalker walker = null; - private final Map connections = Maps.newHashMap(); - private Hook.Closeable unhook = null; - private List recordedQueries = Lists.newCopyOnWriteArrayList(); @Before public void setUp() throws Exception { - walker = CalciteTests.createWalker(temporaryFolder.newFolder()); - connections.put(PLANNER_CONFIG_DEFAULT, connectJdbc(walker, PLANNER_CONFIG_DEFAULT)); - connections.put(PLANNER_CONFIG_NO_TOPN, connectJdbc(walker, PLANNER_CONFIG_NO_TOPN)); - connections.put(PLANNER_CONFIG_SELECT_PAGING, connectJdbc(walker, PLANNER_CONFIG_SELECT_PAGING)); - connections.put(PLANNER_CONFIG_FALLBACK, connectJdbc(walker, PLANNER_CONFIG_FALLBACK)); - connections.put(PLANNER_CONFIG_SINGLE_NESTING_ONLY, connectJdbc(walker, PLANNER_CONFIG_SINGLE_NESTING_ONLY)); - connections.put(PLANNER_CONFIG_NO_SUBQUERIES, connectJdbc(walker, PLANNER_CONFIG_NO_SUBQUERIES)); - - unhook = Hook.QUERY_PLAN.add( - new Function() - { - @Override - public Object apply(Object input) - { - try { - log.info("Issued query: %s", OBJECT_MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(input)); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - recordedQueries.add((Query) input); - return null; - } - } - ); + Calcites.setSystemProperties(); + walker = CalciteTests.createMockWalker(temporaryFolder.newFolder()); } @After public void tearDown() throws Exception { - if (unhook != null) { - unhook.close(); - } walker.close(); walker = null; - for (CalciteConnection connection : connections.values()) { - connection.close(); - } - connections.clear(); - } - - private static CalciteConnection connectJdbc( - final QuerySegmentWalker walker, - final PlannerConfig plannerConfig - ) throws SQLException - { - return Calcites.jdbc(CalciteTests.createMockSchema(walker, plannerConfig), plannerConfig); } @Test @@ -238,31 +195,79 @@ public void testExplainSelectConstantExpression() throws Exception "EXPLAIN PLAN FOR SELECT 1 + 1", ImmutableList.of(), ImmutableList.of( - new Object[]{"EnumerableValues(tuples=[[{ 2 }]])\n"} + new Object[]{"BindableValues(tuples=[[{ 2 }]])\n"} ) ); } @Test - public void testMetadata() throws Exception + public void testInformationSchemaSchemata() throws Exception { - final String varcharDescription = "VARCHAR(1) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\" NOT NULL"; + testQuery( + "SELECT DISTINCT SCHEMA_NAME FROM INFORMATION_SCHEMA.SCHEMATA", + ImmutableList.of(), + ImmutableList.of( + new Object[]{"druid"}, + new Object[]{"INFORMATION_SCHEMA"} + ) + ); + } - // Fallback is necessary since without it, we don't have the Enumerable operators necessary to do this query. + @Test + public void testInformationSchemaTables() throws Exception + { testQuery( - PLANNER_CONFIG_FALLBACK, - "SELECT columnName, dataType, typeName FROM metadata.COLUMNS WHERE tableName = 'foo'", + "SELECT TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE\n" + + "FROM INFORMATION_SCHEMA.TABLES\n" + + "WHERE TABLE_TYPE IN ('SYSTEM_TABLE', 'TABLE')", ImmutableList.of(), ImmutableList.of( - new Object[]{"__time", 93, "TIMESTAMP(0) NOT NULL"}, - new Object[]{"cnt", -5, "BIGINT NOT NULL"}, - new Object[]{"dim1", 12, varcharDescription}, - new Object[]{"dim2", 12, varcharDescription}, - new Object[]{"m1", 6, "FLOAT NOT NULL"} + new Object[]{"druid", "foo", "TABLE"}, + new Object[]{"druid", "foo2", "TABLE"}, + new Object[]{"INFORMATION_SCHEMA", "COLUMNS", "SYSTEM_TABLE"}, + new Object[]{"INFORMATION_SCHEMA", "SCHEMATA", "SYSTEM_TABLE"}, + new Object[]{"INFORMATION_SCHEMA", "TABLES", "SYSTEM_TABLE"} ) ); } + @Test + public void testInformationSchemaColumns() throws Exception + { + testQuery( + "SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n" + + "FROM INFORMATION_SCHEMA.COLUMNS\n" + + "WHERE TABLE_SCHEMA = 'druid' AND TABLE_NAME = 'foo'", + ImmutableList.of(), + ImmutableList.of( + new Object[]{"__time", "TIMESTAMP", "NO"}, + new Object[]{"cnt", "BIGINT", "NO"}, + new Object[]{"dim1", "VARCHAR", "NO"}, + new Object[]{"dim2", "VARCHAR", "NO"}, + new Object[]{"m1", "FLOAT", "NO"}, + new Object[]{"unique_dim1", "OTHER", "NO"} + ) + ); + } + + @Test + public void testExplainInformationSchemaColumns() throws Exception + { + testQuery( + "EXPLAIN PLAN FOR\n" + + "SELECT COLUMN_NAME, DATA_TYPE\n" + + "FROM INFORMATION_SCHEMA.COLUMNS\n" + + "WHERE TABLE_SCHEMA = 'druid' AND TABLE_NAME = 'foo'", + ImmutableList.of(), + ImmutableList.of( + new Object[]{ + "BindableProject(COLUMN_NAME=[$3], DATA_TYPE=[$7])\n" + + " BindableFilter(condition=[AND(=(CAST($1):VARCHAR(5) CHARACTER SET \"UTF-16LE\" COLLATE \"UTF-16LE$en_US$primary\" NOT NULL, 'druid'), =(CAST($2):VARCHAR(3) CHARACTER SET \"UTF-16LE\" COLLATE \"UTF-16LE$en_US$primary\" NOT NULL, 'foo'))])\n" + + " BindableTableScan(table=[[INFORMATION_SCHEMA, COLUMNS]])\n" + } + ) + ); + } @Test public void testSelectStar() throws Exception @@ -271,13 +276,13 @@ public void testSelectStar() throws Exception "SELECT * FROM druid.foo", ImmutableList.of( Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .pagingSpec(FIRST_PAGING_SPEC) .build(), Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .pagingSpec( @@ -290,12 +295,32 @@ public void testSelectStar() throws Exception .build() ), ImmutableList.of( - new Object[]{T("2000-01-01"), 1L, "", "a", 1.0}, - new Object[]{T("2000-01-02"), 1L, "10.1", "", 2.0}, - new Object[]{T("2000-01-03"), 1L, "2", "", 3.0}, - new Object[]{T("2001-01-01"), 1L, "1", "a", 4.0}, - new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0}, - new Object[]{T("2001-01-03"), 1L, "abc", "", 6.0} + new Object[]{T("2000-01-01"), 1L, "", "a", 1.0, HLLCV1.class.getName()}, + new Object[]{T("2000-01-02"), 1L, "10.1", "", 2.0, HLLCV1.class.getName()}, + new Object[]{T("2000-01-03"), 1L, "2", "", 3.0, HLLCV1.class.getName()}, + new Object[]{T("2001-01-01"), 1L, "1", "a", 4.0, HLLCV1.class.getName()}, + new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0, HLLCV1.class.getName()}, + new Object[]{T("2001-01-03"), 1L, "abc", "", 6.0, HLLCV1.class.getName()} + ) + ); + } + + @Test + public void testUnqualifiedTableName() throws Exception + { + testQuery( + "SELECT COUNT(*) FROM foo", + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(QSS(Filtration.eternity())) + .granularity(QueryGranularities.ALL) + .aggregators(AGGS(new CountAggregatorFactory("a0"))) + .context(TIMESERIES_CONTEXT) + .build() + ), + ImmutableList.of( + new Object[]{6L} ) ); } @@ -308,8 +333,7 @@ public void testExplainSelectStar() throws Exception ImmutableList.of(), ImmutableList.of( new Object[]{ - "EnumerableInterpreter\n" - + " DruidQueryRel(dataSource=[foo])\n" + "DruidQueryRel(dataSource=[foo])\n" } ) ); @@ -322,15 +346,15 @@ public void testSelectStarWithLimit() throws Exception "SELECT * FROM druid.foo LIMIT 2", ImmutableList.of( Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .pagingSpec(FIRST_PAGING_SPEC) .build() ), ImmutableList.of( - new Object[]{T("2000-01-01"), 1L, "", "a", 1.0}, - new Object[]{T("2000-01-02"), 1L, "10.1", "", 2.0} + new Object[]{T("2000-01-01"), 1L, "", "a", 1.0, HLLCV1.class.getName()}, + new Object[]{T("2000-01-02"), 1L, "10.1", "", 2.0, HLLCV1.class.getName()} ) ); } @@ -342,7 +366,7 @@ public void testSelectStarWithLimitDescending() throws Exception "SELECT * FROM druid.foo ORDER BY __time DESC LIMIT 2", ImmutableList.of( Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .descending(true) @@ -350,8 +374,8 @@ public void testSelectStarWithLimitDescending() throws Exception .build() ), ImmutableList.of( - new Object[]{T("2001-01-03"), 1L, "abc", "", 6.0}, - new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0} + new Object[]{T("2001-01-03"), 1L, "abc", "", 6.0, HLLCV1.class.getName()}, + new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0, HLLCV1.class.getName()} ) ); } @@ -363,7 +387,7 @@ public void testSelectSingleColumnTwice() throws Exception "SELECT dim2 x, dim2 y FROM druid.foo LIMIT 2", ImmutableList.of( Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .dimensionSpecs(DIMS( new DefaultDimensionSpec("dim2", "d1"), @@ -388,7 +412,7 @@ public void testSelectSingleColumnWithLimitDescending() throws Exception "SELECT dim1 FROM druid.foo ORDER BY __time DESC LIMIT 2", ImmutableList.of( Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .dimensionSpecs(DIMS(new DefaultDimensionSpec("dim1", "d1"))) .granularity(QueryGranularities.ALL) @@ -403,6 +427,43 @@ public void testSelectSingleColumnWithLimitDescending() throws Exception ); } + @Test + public void testGroupBySingleColumnDescending() throws Exception + { + testQuery( + PLANNER_CONFIG_NO_TOPN, + "SELECT dim1 FROM druid.foo GROUP BY dim1 ORDER BY dim1 DESC", + ImmutableList.of( + new GroupByQuery.Builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(QSS(Filtration.eternity())) + .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0"))) + .setGranularity(QueryGranularities.ALL) + .setLimitSpec( + new DefaultLimitSpec( + ImmutableList.of( + new OrderByColumnSpec( + "d0", + OrderByColumnSpec.Direction.DESCENDING, + StringComparators.LEXICOGRAPHIC + ) + ), + Integer.MAX_VALUE + ) + ) + .build() + ), + ImmutableList.of( + new Object[]{"def"}, + new Object[]{"abc"}, + new Object[]{"2"}, + new Object[]{"10.1"}, + new Object[]{"1"}, + new Object[]{""} + ) + ); + } + @Test public void testSelfJoinWithFallback() throws Exception { @@ -415,38 +476,38 @@ public void testSelfJoinWithFallback() throws Exception + " x.dim1 <> ''", ImmutableList.of( Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) - .filters(NOT(SELECTOR("dim1", "", null))) .pagingSpec(FIRST_PAGING_SPEC) .build(), Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) - .filters(NOT(SELECTOR("dim1", "", null))) .pagingSpec( new PagingSpec( - ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 4), + ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 5), 1000, true ) ) .build(), Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) + .filters(NOT(SELECTOR("dim1", "", null))) .pagingSpec(FIRST_PAGING_SPEC) .build(), Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) + .filters(NOT(SELECTOR("dim1", "", null))) .pagingSpec( new PagingSpec( - ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 5), + ImmutableMap.of("foo_1970-01-01T00:00:00.000Z_2001-01-03T00:00:00.001Z_1", 4), 1000, true ) @@ -473,12 +534,10 @@ public void testExplainSelfJoinWithFallback() throws Exception ImmutableList.of(), ImmutableList.of( new Object[]{ - "EnumerableCalc(expr#0..9=[{inputs}], dim1=[$t7], dim10=[$t2], dim2=[$t3])\n" - + " EnumerableJoin(condition=[=($3, $7)], joinType=[inner])\n" - + " EnumerableInterpreter\n" - + " DruidQueryRel(dataSource=[foo])\n" - + " EnumerableInterpreter\n" - + " DruidQueryRel(dataSource=[foo], filter=[!dim1 = ])\n" + "BindableProject(dim1=[$8], dim10=[$2], dim2=[$3])\n" + + " BindableJoin(condition=[=($8, $3)], joinType=[inner])\n" + + " DruidQueryRel(dataSource=[foo])\n" + + " DruidQueryRel(dataSource=[foo], filter=[!dim1 = ])\n" } ) ); @@ -508,7 +567,8 @@ public void testUnplannableQueries() throws Exception + "FROM druid.foo\n" + "GROUP BY (CAST(__time AS DATE) + EXTRACT(HOUR FROM __time) * INTERVAL '1' HOUR)", // Time arithmetic "SELECT dim1, SUM(m1) AS m1_sum FROM druid.foo GROUP BY dim1 HAVING SUM(m1) > 1", // HAVING on float - "SELECT columnName, typeName FROM metadata.COLUMNS WHERE tableName = 'foo'" // Metadata tables without fallback + "SELECT SUBSTRING(dim1, 2) FROM druid.foo GROUP BY dim1", // Project a dimension from GROUP BY + "SELECT dim1 FROM druid.foo GROUP BY dim1 ORDER BY SUBSTRING(dim1, 2)" // ORDER BY projection ); for (final String query : queries) { @@ -531,8 +591,8 @@ private void assertQueryIsUnplannable(final PlannerConfig plannerConfig, final S e = e1; } - if (!(e instanceof SQLException) || !(e.getCause() instanceof RelOptPlanner.CannotPlanException)) { - log.error(e, "Expected SQLException caused by CannotPlanException for query: %s", sql); + if (!(e instanceof RelOptPlanner.CannotPlanException)) { + log.error(e, "Expected CannotPlanException for query: %s", sql); Assert.fail(sql); } } @@ -544,7 +604,7 @@ public void testSelectStarWithDimFilter() throws Exception "SELECT * FROM druid.foo WHERE dim1 > 'd' OR dim2 = 'a'", ImmutableList.of( Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .pagingSpec(FIRST_PAGING_SPEC) @@ -556,7 +616,7 @@ public void testSelectStarWithDimFilter() throws Exception ) .build(), Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .pagingSpec( @@ -575,9 +635,9 @@ public void testSelectStarWithDimFilter() throws Exception .build() ), ImmutableList.of( - new Object[]{T("2000-01-01"), 1L, "", "a", 1.0}, - new Object[]{T("2001-01-01"), 1L, "1", "a", 4.0}, - new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0} + new Object[]{T("2000-01-01"), 1L, "", "a", 1.0, HLLCV1.class.getName()}, + new Object[]{T("2001-01-01"), 1L, "1", "a", 4.0, HLLCV1.class.getName()}, + new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0, HLLCV1.class.getName()} ) ); } @@ -590,7 +650,7 @@ public void testSelectStarWithDimFilterAndPaging() throws Exception "SELECT * FROM druid.foo WHERE dim1 > 'd' OR dim2 = 'a'", ImmutableList.of( Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .pagingSpec(new PagingSpec(null, 2, true)) @@ -602,7 +662,7 @@ public void testSelectStarWithDimFilterAndPaging() throws Exception ) .build(), Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .pagingSpec( @@ -620,7 +680,7 @@ public void testSelectStarWithDimFilterAndPaging() throws Exception ) .build(), Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .pagingSpec( @@ -639,9 +699,9 @@ public void testSelectStarWithDimFilterAndPaging() throws Exception .build() ), ImmutableList.of( - new Object[]{T("2000-01-01"), 1L, "", "a", 1.0}, - new Object[]{T("2001-01-01"), 1L, "1", "a", 4.0}, - new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0} + new Object[]{T("2000-01-01"), 1L, "", "a", 1.0, HLLCV1.class.getName()}, + new Object[]{T("2001-01-01"), 1L, "1", "a", 4.0, HLLCV1.class.getName()}, + new Object[]{T("2001-01-02"), 1L, "def", "abc", 5.0, HLLCV1.class.getName()} ) ); } @@ -678,7 +738,7 @@ public void testGroupByWithFilterMatchingNothing() throws Exception "SELECT COUNT(*), MAX(cnt) FROM druid.foo WHERE dim1 = 'foobar'", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .filters(SELECTOR("dim1", "foobar", null)) .granularity(QueryGranularities.ALL) @@ -700,7 +760,7 @@ public void testGroupByWithFilterMatchingNothingWithGroupByLiteral() throws Exce "SELECT COUNT(*), MAX(cnt) FROM druid.foo WHERE dim1 = 'foobar' GROUP BY 'dummy'", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .filters(SELECTOR("dim1", "foobar", null)) .granularity(QueryGranularities.ALL) @@ -722,7 +782,7 @@ public void testCountStar() throws Exception "SELECT COUNT(*) FROM druid.foo", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) @@ -742,7 +802,7 @@ public void testCountStarWithLikeFilter() throws Exception "SELECT COUNT(*) FROM druid.foo WHERE dim1 like 'a%' OR dim2 like '%xb%' escape 'x'", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .filters( @@ -768,7 +828,7 @@ public void testCountStarWithLongColumnFilters() throws Exception "SELECT COUNT(*) FROM druid.foo WHERE cnt >= 3 OR cnt = 1", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .filters( @@ -794,7 +854,7 @@ public void testCountStarWithLongColumnFiltersOnTwoPoints() throws Exception "SELECT COUNT(*) FROM druid.foo WHERE cnt = 1 OR cnt = 2", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .filters(IN("cnt", ImmutableList.of("1", "2"), null)) @@ -817,7 +877,7 @@ public void testFilterOnStringAsNumber() throws Exception + "(floor(CAST(dim1 AS float)) = 10.00 and CAST(dim1 AS float) > 9 and CAST(dim1 AS float) <= 10.5)", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0"))) @@ -845,7 +905,7 @@ public void testSimpleAggregations() throws Exception "SELECT COUNT(*), COUNT(cnt), COUNT(dim1), AVG(cnt), SUM(cnt), SUM(cnt) + MIN(cnt) + MAX(cnt) FROM druid.foo", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .aggregators( @@ -901,7 +961,7 @@ public void testGroupByWithSortOnPostAggregation() throws Exception "SELECT dim1, MIN(m1) + MAX(m1) AS x FROM druid.foo GROUP BY dim1 ORDER BY x LIMIT 3", ImmutableList.of( new TopNQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .dimension(new DefaultDimensionSpec("dim1", "d0")) @@ -941,7 +1001,7 @@ public void testGroupByWithSortOnPostAggregationNoTopNConfig() throws Exception "SELECT dim1, MIN(m1) + MAX(m1) AS x FROM druid.foo GROUP BY dim1 ORDER BY x LIMIT 3", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0"))) @@ -1001,7 +1061,7 @@ public void testFilteredAggregations() throws Exception + "FROM druid.foo", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .aggregators(AGGS( @@ -1057,7 +1117,7 @@ public void testExpressionAggregations() throws Exception "SELECT SUM(cnt * 3), LN(SUM(cnt) + SUM(m1)) FROM druid.foo", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .aggregators(AGGS( @@ -1084,7 +1144,7 @@ public void testInFilter() throws Exception "SELECT dim1, COUNT(*) FROM druid.foo WHERE dim1 IN ('abc', 'def', 'ghi') GROUP BY dim1", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0"))) @@ -1110,7 +1170,7 @@ public void testCountStarWithDegenerateFilter() throws Exception "SELECT COUNT(*) FROM druid.foo WHERE dim2 = 'a' and (dim1 > 'a' OR dim1 < 'b')", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .filters(SELECTOR("dim2", "a", null)) @@ -1131,7 +1191,7 @@ public void testCountStarWithNotOfDegenerateFilter() throws Exception "SELECT COUNT(*) FROM druid.foo WHERE dim2 = 'a' and not (dim1 > 'a' OR dim1 < 'b')", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS()) .granularity(QueryGranularities.ALL) .filters(null) @@ -1150,7 +1210,7 @@ public void testCountStarWithBoundFilterSimplifyOr() throws Exception "SELECT COUNT(*) FROM druid.foo WHERE (dim1 >= 'a' and dim1 < 'b') OR dim1 = 'ab'", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .filters(BOUND("dim1", "a", "b", false, true, null, StringComparators.LEXICOGRAPHIC)) @@ -1171,7 +1231,7 @@ public void testCountStarWithBoundFilterSimplifyAnd() throws Exception "SELECT COUNT(*) FROM druid.foo WHERE (dim1 >= 'a' and dim1 < 'b') and dim1 = 'abc'", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .filters(SELECTOR("dim1", "abc", null)) @@ -1192,7 +1252,7 @@ public void testCountStarWithFilterOnCastedString() throws Exception "SELECT COUNT(*) FROM druid.foo WHERE CAST(dim1 AS bigint) = 2", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .filters(NUMERIC_SELECTOR("dim1", "2", null)) @@ -1214,7 +1274,7 @@ public void testCountStarWithTimeFilter() throws Exception + "WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2001-01-01 00:00:00'", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(new Interval("2000-01-01/2001-01-01"))) .granularity(QueryGranularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) @@ -1234,7 +1294,7 @@ public void testCountStarWithSinglePointInTime() throws Exception "SELECT COUNT(*) FROM druid.foo WHERE __time = TIMESTAMP '2000-01-01 00:00:00'", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(new Interval("2000-01-01/2000-01-01T00:00:00.001"))) .granularity(QueryGranularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) @@ -1255,7 +1315,7 @@ public void testCountStarWithTwoPointsInTime() throws Exception + "__time = TIMESTAMP '2000-01-01 00:00:00' OR __time = TIMESTAMP '2000-01-01 00:00:00' + INTERVAL '1' DAY", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals( QSS( new Interval("2000-01-01/2000-01-01T00:00:00.001"), @@ -1288,7 +1348,7 @@ public void testCountStarWithComplexDisjointTimeFilter() throws Exception + ")", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(new Interval("2000/2001"), new Interval("2002-05-01/2003-05-01"))) .granularity(QueryGranularities.ALL) .filters( @@ -1329,19 +1389,19 @@ public void testCountStarWithNotOfComplexDisjointTimeFilter() throws Exception + ")", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .filters( - OR( - NOT(SELECTOR("dim2", "a", null)), - AND( - NOT(TIME_BOUND("2000/2001")), - NOT(AND( + NOT(AND( + SELECTOR("dim2", "a", null), + OR( + TIME_BOUND("2000/2001"), + AND( SELECTOR("dim1", "abc", null), TIME_BOUND("2002-05-01/2003-05-01") - )) + ) ) - ) + )) ) .granularity(QueryGranularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) @@ -1364,7 +1424,7 @@ public void testCountStarWithNotTimeFilter() throws Exception + " OR (__time >= TIMESTAMP '2003-01-01 00:00:00' AND __time < TIMESTAMP '2004-01-01 00:00:00'))", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals( QSS( new Interval(Filtration.eternity().getStart(), new DateTime("2000")), @@ -1393,7 +1453,7 @@ public void testCountStarWithTimeAndDimFilter() throws Exception + "and __time BETWEEN TIMESTAMP '2000-01-01 00:00:00' AND TIMESTAMP '2000-12-31 23:59:59.999'", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(new Interval("2000-01-01/2001-01-01"))) .filters(NOT(SELECTOR("dim2", "a", null))) .granularity(QueryGranularities.ALL) @@ -1416,15 +1476,15 @@ public void testCountStarWithTimeOrDimFilter() throws Exception + "or __time BETWEEN TIMESTAMP '2000-01-01 00:00:00' AND TIMESTAMP '2000-12-31 23:59:59.999'", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .filters( OR( NOT(SELECTOR("dim2", "a", null)), BOUND( "__time", - String.valueOf(T("2000-01-01").getTime()), - String.valueOf(T("2000-12-31T23:59:59.999").getTime()), + String.valueOf(T("2000-01-01")), + String.valueOf(T("2000-12-31T23:59:59.999")), false, false, null, @@ -1452,7 +1512,7 @@ public void testCountStarWithTimeFilterOnLongColumn() throws Exception + "AND cnt < EXTRACT(EPOCH FROM TIMESTAMP '1970-01-02 00:00:00') * 1000", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .filters( @@ -1483,7 +1543,7 @@ public void testSelectDistinctWithCascadeExtractionFilter() throws Exception "SELECT distinct dim1 FROM druid.foo WHERE substring(substring(dim1, 2), 1, 1) = 'e' OR dim2 = 'a'", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0"))) @@ -1518,7 +1578,7 @@ public void testSelectDistinctWithStrlenFilter() throws Exception + "WHERE CHARACTER_LENGTH(dim1) = 3 OR CAST(CHARACTER_LENGTH(dim1) AS varchar) = 3", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0"))) @@ -1537,14 +1597,39 @@ public void testSelectDistinctWithStrlenFilter() throws Exception ); } + @Test + public void testSelectDistinctWithLimit() throws Exception + { + // Should use topN even if approximate topNs are off, because this query is exact. + + testQuery( + "SELECT DISTINCT dim2 FROM druid.foo LIMIT 10", + ImmutableList.of( + new TopNQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(QSS(Filtration.eternity())) + .granularity(QueryGranularities.ALL) + .dimension(new DefaultDimensionSpec("dim2", "d0")) + .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) + .threshold(10) + .build() + ), + ImmutableList.of( + new Object[]{""}, + new Object[]{"a"}, + new Object[]{"abc"} + ) + ); + } + @Test public void testCountDistinct() throws Exception { testQuery( - "SELECT SUM(cnt), COUNT(distinct dim2) FROM druid.foo", + "SELECT SUM(cnt), COUNT(distinct dim2), COUNT(distinct unique_dim1) FROM druid.foo", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .aggregators( @@ -1553,16 +1638,62 @@ public void testCountDistinct() throws Exception new CardinalityAggregatorFactory( "a1", null, - DIMS(new DefaultDimensionSpec("dim2", "A1:dimSpec")), + DIMS(new DefaultDimensionSpec("dim2", null)), false - ) + ), + new HyperUniquesAggregatorFactory("a2", "unique_dim1") ) ) .context(TIMESERIES_CONTEXT) .build() ), ImmutableList.of( - new Object[]{6L, 3L} + new Object[]{6L, 3L, 6L} + ) + ); + } + + @Test + public void testApproxCountDistinct() throws Exception + { + testQuery( + "SELECT\n" + + " SUM(cnt),\n" + + " APPROX_COUNT_DISTINCT(dim2),\n" // uppercase + + " approx_count_distinct(dim2) FILTER(WHERE dim2 <> ''),\n" // lowercase; also, filtered + + " approx_count_distinct(unique_dim1)\n" // on native hyperUnique column + + "FROM druid.foo", + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(QSS(Filtration.eternity())) + .granularity(QueryGranularities.ALL) + .aggregators( + AGGS( + new LongSumAggregatorFactory("a0", "cnt"), + new CardinalityAggregatorFactory( + "a1", + null, + DIMS(new DefaultDimensionSpec("dim2", "dim2")), + false + ), + new FilteredAggregatorFactory( + new CardinalityAggregatorFactory( + "a2", + null, + DIMS(new DefaultDimensionSpec("dim2", "dim2")), + false + ), + NOT(SELECTOR("dim2", "", null)) + ), + new HyperUniquesAggregatorFactory("a3", "unique_dim1") + ) + ) + .context(TIMESERIES_CONTEXT) + .build() + ), + ImmutableList.of( + new Object[]{6L, 3L, 2L, 6L} ) ); } @@ -1603,7 +1734,7 @@ public void testExactCountDistinctUsingSubquery() throws Exception .setDataSource( new QueryDataSource( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0"))) @@ -1625,6 +1756,116 @@ public void testExactCountDistinctUsingSubquery() throws Exception ); } + @Test + public void testTopNFilterJoin() throws Exception + { + // Filters on top N values of some dimension by using an inner join. + testQuery( + "SELECT t1.dim1, SUM(t1.cnt)\n" + + "FROM druid.foo t1\n" + + " INNER JOIN (\n" + + " SELECT\n" + + " SUM(cnt) AS sum_cnt,\n" + + " dim2\n" + + " FROM druid.foo\n" + + " GROUP BY dim2\n" + + " ORDER BY 1 DESC\n" + + " LIMIT 2\n" + + ") t2 ON (t1.dim2 = t2.dim2)\n" + + "GROUP BY t1.dim1\n" + + "ORDER BY 1\n", + ImmutableList.of( + new TopNQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(QSS(Filtration.eternity())) + .granularity(QueryGranularities.ALL) + .dimension(new DefaultDimensionSpec("dim2", "d0")) + .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) + .metric(new NumericTopNMetricSpec("a0")) + .threshold(2) + .build(), + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(QSS(Filtration.eternity())) + .setGranularity(QueryGranularities.ALL) + .setDimFilter(IN("dim2", ImmutableList.of("", "a"), null)) + .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0"))) + .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) + .setLimitSpec( + new DefaultLimitSpec( + ImmutableList.of( + new OrderByColumnSpec( + "d0", + OrderByColumnSpec.Direction.ASCENDING, + StringComparators.LEXICOGRAPHIC + ) + ), + Integer.MAX_VALUE + ) + ) + .build() + ), + ImmutableList.of( + new Object[]{"", 1L}, + new Object[]{"1", 1L}, + new Object[]{"10.1", 1L}, + new Object[]{"2", 1L}, + new Object[]{"abc", 1L} + ) + ); + } + + @Test + public void testRemovableLeftJoin() throws Exception + { + // LEFT JOIN where the right-hand side can be ignored. + + testQuery( + "SELECT t1.dim1, SUM(t1.cnt)\n" + + "FROM druid.foo t1\n" + + " LEFT JOIN (\n" + + " SELECT\n" + + " SUM(cnt) AS sum_cnt,\n" + + " dim2\n" + + " FROM druid.foo\n" + + " GROUP BY dim2\n" + + " ORDER BY 1 DESC\n" + + " LIMIT 2\n" + + ") t2 ON (t1.dim2 = t2.dim2)\n" + + "GROUP BY t1.dim1\n" + + "ORDER BY 1\n", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(QSS(Filtration.eternity())) + .setGranularity(QueryGranularities.ALL) + .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0"))) + .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) + .setLimitSpec( + new DefaultLimitSpec( + ImmutableList.of( + new OrderByColumnSpec( + "d0", + OrderByColumnSpec.Direction.ASCENDING, + StringComparators.LEXICOGRAPHIC + ) + ), + Integer.MAX_VALUE + ) + ) + .build() + ), + ImmutableList.of( + new Object[]{"", 1L}, + new Object[]{"1", 1L}, + new Object[]{"10.1", 1L}, + new Object[]{"2", 1L}, + new Object[]{"abc", 1L}, + new Object[]{"def", 1L} + ) + ); + } + @Test public void testExactCountDistinctOfSemiJoinResult() throws Exception { @@ -1639,13 +1880,13 @@ public void testExactCountDistinctOfSemiJoinResult() throws Exception + ")", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimFilter(NOT(SELECTOR("dim1", "", null))) .setDimensions(DIMS(new ExtractionDimensionSpec( "dim1", - "v0", + "d0", new SubstringDimExtractionFn(0, 1) ))) .build(), @@ -1653,7 +1894,7 @@ public void testExactCountDistinctOfSemiJoinResult() throws Exception .setDataSource( new QueryDataSource( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimFilter(IN( @@ -1693,7 +1934,7 @@ public void testExactCountDistinctUsingSubqueryWithWherePushDown() throws Except .setDataSource( new QueryDataSource( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setDimFilter(NOT(SELECTOR("dim2", "", null))) .setGranularity(QueryGranularities.ALL) @@ -1730,7 +1971,7 @@ public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter() throws E .setDataSource( new QueryDataSource( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0"))) @@ -1768,7 +2009,7 @@ public void testCompareExactAndApproximateCountDistinctUsingSubquery() throws Ex .setDataSource( new QueryDataSource( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimFilter(NOT(SELECTOR("dim1", "", null))) @@ -1782,7 +2023,7 @@ public void testCompareExactAndApproximateCountDistinctUsingSubquery() throws Ex new CountAggregatorFactory("a0"), new CardinalityAggregatorFactory( "a1", - DIMS(new DefaultDimensionSpec("d0", "A1:dimSpec")), + DIMS(new DefaultDimensionSpec("d0", null)), false ) )) @@ -1843,7 +2084,7 @@ public void testHistogramUsingSubquery() throws Exception .setDataSource( new QueryDataSource( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0"))) @@ -1884,7 +2125,7 @@ public void testHistogramUsingSubqueryWithSort() throws Exception .setDataSource( new QueryDataSource( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0"))) @@ -1934,7 +2175,7 @@ public void testCountDistinctArithmetic() throws Exception + "FROM druid.foo", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .aggregators( @@ -1943,7 +2184,7 @@ public void testCountDistinctArithmetic() throws Exception new CardinalityAggregatorFactory( "a1", null, - DIMS(new DefaultDimensionSpec("dim2", "A1:dimSpec")), + DIMS(new DefaultDimensionSpec("dim2", null)), false ) ) @@ -1985,7 +2226,7 @@ public void testCountDistinctOfSubstring() throws Exception "SELECT COUNT(distinct substring(dim1, 1, 1)) FROM druid.foo WHERE dim1 <> ''", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .filters(NOT(SELECTOR("dim1", "", null))) .granularity(QueryGranularities.ALL) @@ -1996,7 +2237,7 @@ public void testCountDistinctOfSubstring() throws Exception DIMS( new ExtractionDimensionSpec( "dim1", - "A0:dimSpec", + null, new SubstringDimExtractionFn(0, 1) ) ), @@ -2020,7 +2261,7 @@ public void testGroupBySortPushDown() throws Exception "SELECT dim1, dim2, SUM(cnt) FROM druid.foo GROUP BY dim1, dim2 ORDER BY dim2 LIMIT 4", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions( @@ -2065,7 +2306,7 @@ public void testGroupByLimitPushDownWithHavingOnLong() throws Exception + "limit 4", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions( @@ -2107,7 +2348,7 @@ public void testFilterOnTimeFloor() throws Exception + "WHERE floor(__time TO month) = TIMESTAMP '2000-01-01 00:00:00'", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(new Interval("2000/P1M"))) .granularity(QueryGranularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) @@ -2128,7 +2369,7 @@ public void testFilterOnTimeFloorMisaligned() throws Exception + "WHERE floor(__time TO month) = TIMESTAMP '2000-01-01 00:00:01'", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS()) .granularity(QueryGranularities.ALL) .aggregators(AGGS(new CountAggregatorFactory("a0"))) @@ -2147,7 +2388,7 @@ public void testGroupByFloor() throws Exception "SELECT floor(CAST(dim1 AS float)), COUNT(*) FROM druid.foo GROUP BY floor(CAST(dim1 AS float))", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions(DIMS( @@ -2172,7 +2413,7 @@ public void testGroupByFloorWithOrderBy() throws Exception "SELECT floor(CAST(dim1 AS float)) AS fl, COUNT(*) FROM druid.foo GROUP BY floor(CAST(dim1 AS float)) ORDER BY fl DESC", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions( @@ -2214,7 +2455,7 @@ public void testGroupByFloorTimeAndOneOtherDimensionWithOrderBy() throws Excepti + " ORDER BY floor(__time TO year), dim2, COUNT(*) DESC", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions( @@ -2273,7 +2514,7 @@ public void testGroupByStringLength() throws Exception "SELECT CHARACTER_LENGTH(dim1), COUNT(*) FROM druid.foo GROUP BY CHARACTER_LENGTH(dim1)", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions( @@ -2313,7 +2554,7 @@ public void testTimeseries() throws Exception + "ORDER BY gran", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.MONTH) .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) @@ -2339,7 +2580,7 @@ public void testTimeseriesDescending() throws Exception + "ORDER BY gran DESC", ImmutableList.of( Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.MONTH) .aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt"))) @@ -2366,7 +2607,7 @@ public void testGroupByExtractYear() throws Exception + "ORDER BY 1", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions( @@ -2410,7 +2651,7 @@ public void testExtractFloorTime() throws Exception + "GROUP BY EXTRACT(YEAR FROM FLOOR(__time TO YEAR))", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions( @@ -2450,7 +2691,7 @@ public void testTimeseriesWithLimitNoTopN() throws Exception + "LIMIT 1", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions( @@ -2497,7 +2738,7 @@ public void testTimeseriesWithLimit() throws Exception + "LIMIT 1", ImmutableList.of( new TopNQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .granularity(QueryGranularities.ALL) .dimension( @@ -2528,7 +2769,7 @@ public void testGroupByTimeAndOtherDimension() throws Exception + "ORDER BY dim2, gran", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimensions( @@ -2578,14 +2819,14 @@ public void testUsingSubqueryAsFilter() throws Exception + "group by dim1, dim2 ORDER BY dim2", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimFilter(NOT(SELECTOR("dim1", "", null))) - .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "v2"))) + .setDimensions(DIMS(new DefaultDimensionSpec("dim1", "d0"))) .build(), GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimFilter( @@ -2643,7 +2884,7 @@ public void testUsingSubqueryAsFilterOnTwoColumns() throws Exception + " )", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimFilter(SELECTOR("dim2", "abc", null)) @@ -2655,7 +2896,7 @@ public void testUsingSubqueryAsFilterOnTwoColumns() throws Exception .setHavingSpec(new DimFilterHavingSpec(NUMERIC_SELECTOR("a0", "1", null))) .build(), Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .dimensionSpecs(DIMS( new DefaultDimensionSpec("dim1", "d1"), new DefaultDimensionSpec("dim2", "d2") @@ -2667,7 +2908,7 @@ public void testUsingSubqueryAsFilterOnTwoColumns() throws Exception .pagingSpec(FIRST_PAGING_SPEC) .build(), Druids.newSelectQueryBuilder() - .dataSource(CalciteTests.DATASOURCE) + .dataSource(CalciteTests.DATASOURCE1) .dimensionSpecs(DIMS( new DefaultDimensionSpec("dim1", "d1"), new DefaultDimensionSpec("dim2", "d2") @@ -2700,16 +2941,16 @@ public void testUsingSubqueryWithExtractionFns() throws Exception + "group by dim2", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimFilter(NOT(SELECTOR("dim1", "", null))) .setDimensions( - DIMS(new ExtractionDimensionSpec("dim1", "v0", new SubstringDimExtractionFn(0, 1))) + DIMS(new ExtractionDimensionSpec("dim1", "d0", new SubstringDimExtractionFn(0, 1))) ) .build(), GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE) + .setDataSource(CalciteTests.DATASOURCE1) .setInterval(QSS(Filtration.eternity())) .setGranularity(QueryGranularities.ALL) .setDimFilter( @@ -2730,6 +2971,42 @@ public void testUsingSubqueryWithExtractionFns() throws Exception ); } + @Test + public void testUnicodeFilterAndGroupBy() throws Exception + { + testQuery( + "SELECT\n" + + " dim1,\n" + + " dim2,\n" + + " COUNT(*)\n" + + "FROM foo2\n" + + "WHERE\n" + + " dim1 LIKE U&'\u05D3\\05E8%'\n" // First char is actually in the string; second is a SQL U& escape + + " OR dim1 = 'друид'\n" + + "GROUP BY dim1, dim2", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE2) + .setInterval(QSS(Filtration.eternity())) + .setGranularity(QueryGranularities.ALL) + .setDimFilter(OR( + new LikeDimFilter("dim1", "דר%", null, null), + new SelectorDimFilter("dim1", "друид", null) + )) + .setDimensions(DIMS( + new DefaultDimensionSpec("dim1", "d0"), + new DefaultDimensionSpec("dim2", "d1") + )) + .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0"))) + .build() + ), + ImmutableList.of( + new Object[]{"друид", "ru", 1L}, + new Object[]{"דרואיד", "he", 1L} + ) + ); + } + private void testQuery( final String sql, final List expectedQueries, @@ -2746,41 +3023,68 @@ private void testQuery( final List expectedResults ) throws Exception { - recordedQueries.clear(); - log.info("SQL: %s", sql); + queryLogHook.clearRecordedQueries(); + final List plannerResults = getResults(plannerConfig, sql); + verifyResults(sql, expectedQueries, expectedResults, plannerResults); + } - final Connection theConnection = connections.get(plannerConfig); - final ResultSet resultSet = theConnection.createStatement().executeQuery(sql); - final ResultSetMetaData metaData = resultSet.getMetaData(); - final List results = Lists.newArrayList(); + private List getResults( + final PlannerConfig plannerConfig, + final String sql + ) throws Exception + { + final SchemaPlus rootSchema = Calcites.createRootSchema(CalciteTests.createMockSchema(walker, plannerConfig)); + final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); - while (resultSet.next()) { - final Object[] row = new Object[metaData.getColumnCount()]; - for (int i = 0; i < row.length; i++) { - row[i] = resultSet.getObject(i + 1); - } - log.info("Result row: %s", Arrays.toString(row)); - results.add(row); + try (Planner planner = new PlannerFactory(rootSchema, operatorTable, plannerConfig).createPlanner()) { + final PlannerResult plan = Calcites.plan(planner, sql); + return Sequences.toList(plan.run(), Lists.newArrayList()); } + } - Assert.assertEquals("result count", expectedResults.size(), results.size()); + private void verifyResults( + final String sql, + final List expectedQueries, + final List expectedResults, + final List results + ) + { for (int i = 0; i < results.size(); i++) { - Assert.assertArrayEquals("result #" + (i + 1), expectedResults.get(i), results.get(i)); + log.info("row #%d: %s", i, Arrays.toString(results.get(i))); + } + + Assert.assertEquals(String.format("result count: %s", sql), expectedResults.size(), results.size()); + for (int i = 0; i < results.size(); i++) { + Assert.assertArrayEquals( + String.format("result #%d: %s", i + 1, sql), + expectedResults.get(i), + results.get(i) + ); } if (expectedQueries != null) { - Assert.assertEquals("query count", expectedQueries.size(), recordedQueries.size()); + final List recordedQueries = queryLogHook.getRecordedQueries(); + + Assert.assertEquals( + String.format("query count: %s", sql), + expectedQueries.size(), + recordedQueries.size() + ); for (int i = 0; i < expectedQueries.size(); i++) { - Assert.assertEquals("query #" + (i + 1), expectedQueries.get(i), recordedQueries.get(i)); + Assert.assertEquals( + String.format("query #%d: %s", i + 1, sql), + expectedQueries.get(i), + recordedQueries.get(i) + ); } } } - // Generate java.util.Date, for expected results - private static Date T(final String timeString) + // Generate timestamps for expected results + private static long T(final String timeString) { - return new Date(new DateTime(timeString).getMillis()); + return new DateTime(timeString).getMillis(); } private static QuerySegmentSpec QSS(final Interval... intervals) diff --git a/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java b/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java index 12a963945964..2ab2590fa5ce 100644 --- a/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java @@ -26,12 +26,13 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.query.QueryInterruptedException; import io.druid.sql.calcite.planner.Calcites; +import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; +import io.druid.sql.calcite.planner.PlannerFactory; import io.druid.sql.calcite.util.CalciteTests; import io.druid.sql.http.SqlQuery; import io.druid.sql.http.SqlResource; -import org.apache.calcite.jdbc.CalciteConnection; -import org.junit.After; +import org.apache.calcite.schema.SchemaPlus; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -55,28 +56,21 @@ public class SqlResourceTest @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private CalciteConnection connection; private SqlResource resource; @Before public void setUp() throws Exception { + Calcites.setSystemProperties(); final PlannerConfig plannerConfig = new PlannerConfig(); - connection = Calcites.jdbc( + final SchemaPlus rootSchema = Calcites.createRootSchema( CalciteTests.createMockSchema( - CalciteTests.createWalker(temporaryFolder.newFolder()), + CalciteTests.createMockWalker(temporaryFolder.newFolder()), plannerConfig - ), - plannerConfig + ) ); - resource = new SqlResource(JSON_MAPPER, connection); - } - - @After - public void tearDown() throws Exception - { - connection.close(); - connection = null; + final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); + resource = new SqlResource(JSON_MAPPER, new PlannerFactory(rootSchema, operatorTable, plannerConfig)); } @Test @@ -152,8 +146,7 @@ public void testExplainCountStar() throws Exception ImmutableList.of( ImmutableMap.of( "PLAN", - "EnumerableInterpreter\n" - + " DruidQueryRel(dataSource=[foo], dimensions=[[]], aggregations=[[Aggregation{aggregatorFactories=[CountAggregatorFactory{name='a0'}], postAggregator=null, finalizingPostAggregatorFactory=null}]])\n" + "DruidQueryRel(dataSource=[foo], dimensions=[[]], aggregations=[[Aggregation{aggregatorFactories=[CountAggregatorFactory{name='a0'}], postAggregator=null, finalizingPostAggregatorFactory=null}]])\n" ) ), rows @@ -161,7 +154,7 @@ public void testExplainCountStar() throws Exception } @Test - public void testCannotPlan() throws Exception + public void testCannotValidate() throws Exception { expectedException.expect(QueryInterruptedException.class); expectedException.expectMessage("Column 'dim3' not found in any table"); @@ -173,6 +166,18 @@ public void testCannotPlan() throws Exception Assert.fail(); } + @Test + public void testCannotConvert() throws Exception + { + expectedException.expect(QueryInterruptedException.class); + expectedException.expectMessage("Cannot build plan for query: SELECT TRIM(dim1) FROM druid.foo"); + + // TRIM unsupported + doPost(new SqlQuery("SELECT TRIM(dim1) FROM druid.foo")); + + Assert.fail(); + } + private List> doPost(final SqlQuery query) throws Exception { final Response response = resource.doPost(query); diff --git a/sql/src/test/java/io/druid/sql/calcite/planner/CalcitesTest.java b/sql/src/test/java/io/druid/sql/calcite/planner/CalcitesTest.java new file mode 100644 index 000000000000..6afc5ea27d7e --- /dev/null +++ b/sql/src/test/java/io/druid/sql/calcite/planner/CalcitesTest.java @@ -0,0 +1,40 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.sql.calcite.planner; + +import org.junit.Assert; +import org.junit.Test; + +public class CalcitesTest +{ + @Test + public void testEscapeStringLiteral() + { + Assert.assertEquals("''", Calcites.escapeStringLiteral(null)); + Assert.assertEquals("''", Calcites.escapeStringLiteral("")); + Assert.assertEquals("'foo'", Calcites.escapeStringLiteral("foo")); + Assert.assertEquals("'foo bar'", Calcites.escapeStringLiteral("foo bar")); + Assert.assertEquals("U&'foö bar'", Calcites.escapeStringLiteral("foö bar")); + Assert.assertEquals("U&'foo \\0026\\0026 bar'", Calcites.escapeStringLiteral("foo && bar")); + Assert.assertEquals("U&'foo \\005C bar'", Calcites.escapeStringLiteral("foo \\ bar")); + Assert.assertEquals("U&'foo\\0027s bar'", Calcites.escapeStringLiteral("foo's bar")); + Assert.assertEquals("U&'друид'", Calcites.escapeStringLiteral("друид")); + } +} diff --git a/sql/src/test/java/io/druid/sql/calcite/DruidSchemaTest.java b/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java similarity index 82% rename from sql/src/test/java/io/druid/sql/calcite/DruidSchemaTest.java rename to sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java index 35d1e94acb29..093b0c7d3099 100644 --- a/sql/src/test/java/io/druid/sql/calcite/DruidSchemaTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java @@ -17,10 +17,10 @@ * under the License. */ -package io.druid.sql.calcite; +package io.druid.sql.calcite.schema; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; +import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.table.DruidTable; import io.druid.sql.calcite.util.CalciteTests; @@ -48,20 +48,6 @@ public class DruidSchemaTest { private static final PlannerConfig PLANNER_CONFIG_DEFAULT = new PlannerConfig(); - private static final PlannerConfig PLANNER_CONFIG_NO_TOPN = new PlannerConfig() - { - @Override - public int getMaxTopNLimit() - { - return 0; - } - - @Override - public boolean isUseApproximateTopN() - { - return false; - } - }; @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -73,7 +59,8 @@ public boolean isUseApproximateTopN() @Before public void setUp() throws Exception { - walker = CalciteTests.createWalker(temporaryFolder.newFolder()); + Calcites.setSystemProperties(); + walker = CalciteTests.createMockWalker(temporaryFolder.newFolder()); Properties props = new Properties(); props.setProperty("caseSensitive", "true"); @@ -103,17 +90,16 @@ public void tearDown() throws Exception @Test public void testGetTableMap() { - Assert.assertEquals(ImmutableSet.of("foo"), schema.getTableNames()); + Assert.assertEquals(ImmutableSet.of("foo", "foo2"), schema.getTableNames()); final Map tableMap = schema.getTableMap(); - Assert.assertEquals(1, tableMap.size()); - Assert.assertEquals("foo", Iterables.getOnlyElement(tableMap.keySet())); + Assert.assertEquals(ImmutableSet.of("foo", "foo2"), tableMap.keySet()); - final DruidTable druidTable = (DruidTable) Iterables.getOnlyElement(tableMap.values()); - final RelDataType rowType = druidTable.getRowType(new JavaTypeFactoryImpl()); + final DruidTable fooTable = (DruidTable) tableMap.get("foo"); + final RelDataType rowType = fooTable.getRowType(new JavaTypeFactoryImpl()); final List fields = rowType.getFieldList(); - Assert.assertEquals(5, fields.size()); + Assert.assertEquals(6, fields.size()); Assert.assertEquals("__time", fields.get(0).getName()); Assert.assertEquals(SqlTypeName.TIMESTAMP, fields.get(0).getType().getSqlTypeName()); @@ -129,5 +115,8 @@ public void testGetTableMap() Assert.assertEquals("m1", fields.get(4).getName()); Assert.assertEquals(SqlTypeName.FLOAT, fields.get(4).getType().getSqlTypeName()); + + Assert.assertEquals("unique_dim1", fields.get(5).getName()); + Assert.assertEquals(SqlTypeName.OTHER, fields.get(5).getType().getSqlTypeName()); } } diff --git a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java index 157d1e870a1a..1d3611ceed26 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java @@ -22,6 +22,7 @@ import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import io.druid.collections.StupidPool; import io.druid.data.input.InputRow; import io.druid.data.input.impl.DimensionsSpec; @@ -66,6 +67,9 @@ import io.druid.segment.TestHelper; import io.druid.segment.column.ValueType; import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator; +import io.druid.sql.calcite.aggregation.SqlAggregator; +import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.rel.QueryMaker; import io.druid.sql.calcite.table.DruidTable; @@ -74,6 +78,7 @@ import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.Table; import org.apache.calcite.schema.impl.AbstractSchema; +import org.joda.time.DateTime; import java.io.File; import java.nio.ByteBuffer; @@ -85,9 +90,102 @@ */ public class CalciteTests { - public static final String DATASOURCE = "foo"; + public static final String DATASOURCE1 = "foo"; + public static final String DATASOURCE2 = "foo2"; private static final String TIMESTAMP_COLUMN = "t"; + + private static final QueryRunnerFactoryConglomerate CONGLOMERATE = new DefaultQueryRunnerFactoryConglomerate( + ImmutableMap., QueryRunnerFactory>builder() + .put( + SegmentMetadataQuery.class, + new SegmentMetadataQueryRunnerFactory( + new SegmentMetadataQueryQueryToolChest( + new SegmentMetadataQueryConfig("P1W") + ), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + SelectQuery.class, + new SelectQueryRunnerFactory( + new SelectQueryQueryToolChest( + TestHelper.getObjectMapper(), + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), + new SelectQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + TimeseriesQuery.class, + new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest( + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + TopNQuery.class, + new TopNQueryRunnerFactory( + new StupidPool<>( + "TopNQueryRunnerFactory-bufferPool", + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(10 * 1024 * 1024); + } + } + ), + new TopNQueryQueryToolChest( + new TopNQueryConfig(), + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + GroupByQuery.class, + GroupByQueryRunnerTest.makeQueryRunnerFactory( + GroupByQueryRunnerTest.DEFAULT_MAPPER, + new GroupByQueryConfig() + { + @Override + public String getDefaultStrategy() + { + return GroupByStrategySelector.STRATEGY_V2; + } + }, + new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return null; + } + + @Override + public int intermediateComputeSizeBytes() + { + return 10 * 1024 * 1024; + } + + @Override + public int getNumMergeBuffers() + { + // Need 2 buffers for CalciteQueryTest.testDoubleNestedGroupby. + return 2; + } + } + ) + ) + .build() + ); + private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( new TimestampSpec(TIMESTAMP_COLUMN, "iso", null), @@ -98,163 +196,101 @@ public class CalciteTests ) ) ); - private static final List ROWS = ImmutableList.of( - ROW(ImmutableMap.of("t", "2000-01-01", "m1", "1.0", "dim1", "", "dim2", ImmutableList.of("a"))), - ROW(ImmutableMap.of("t", "2000-01-02", "m1", "2.0", "dim1", "10.1", "dim2", ImmutableList.of())), - ROW(ImmutableMap.of("t", "2000-01-03", "m1", "3.0", "dim1", "2", "dim2", ImmutableList.of(""))), - ROW(ImmutableMap.of("t", "2001-01-01", "m1", "4.0", "dim1", "1", "dim2", ImmutableList.of("a"))), - ROW(ImmutableMap.of("t", "2001-01-02", "m1", "5.0", "dim1", "def", "dim2", ImmutableList.of("abc"))), - ROW(ImmutableMap.of("t", "2001-01-03", "m1", "6.0", "dim1", "abc")) + + private static final IncrementalIndexSchema INDEX_SCHEMA = new IncrementalIndexSchema.Builder() + .withMetrics( + new AggregatorFactory[]{ + new CountAggregatorFactory("cnt"), + new DoubleSumAggregatorFactory("m1", "m1"), + new HyperUniquesAggregatorFactory("unique_dim1", "dim1") + } + ) + .withRollup(false) + .build(); + + private static final List ROWS1 = ImmutableList.of( + createRow(ImmutableMap.of("t", "2000-01-01", "m1", "1.0", "dim1", "", "dim2", ImmutableList.of("a"))), + createRow(ImmutableMap.of("t", "2000-01-02", "m1", "2.0", "dim1", "10.1", "dim2", ImmutableList.of())), + createRow(ImmutableMap.of("t", "2000-01-03", "m1", "3.0", "dim1", "2", "dim2", ImmutableList.of(""))), + createRow(ImmutableMap.of("t", "2001-01-01", "m1", "4.0", "dim1", "1", "dim2", ImmutableList.of("a"))), + createRow(ImmutableMap.of("t", "2001-01-02", "m1", "5.0", "dim1", "def", "dim2", ImmutableList.of("abc"))), + createRow(ImmutableMap.of("t", "2001-01-03", "m1", "6.0", "dim1", "abc")) ); - private static final Map COLUMN_TYPES = ImmutableMap.of( - "__time", ValueType.LONG, - "cnt", ValueType.LONG, - "dim1", ValueType.STRING, - "dim2", ValueType.STRING, - "m1", ValueType.FLOAT + + private static final List ROWS2 = ImmutableList.of( + createRow("2000-01-01", "דרואיד", "he", 1.0), + createRow("2000-01-01", "druid", "en", 1.0), + createRow("2000-01-01", "друид", "ru", 1.0) ); + private static final Map COLUMN_TYPES = ImmutableMap.builder() + .put("__time", ValueType.LONG) + .put("cnt", ValueType.LONG) + .put("dim1", ValueType.STRING) + .put("dim2", ValueType.STRING) + .put("m1", ValueType.FLOAT) + .put("unique_dim1", ValueType.COMPLEX) + .build(); + private CalciteTests() { // No instantiation. } - public static SpecificSegmentsQuerySegmentWalker createWalker(final File tmpDir) + public static QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate() { - return createWalker(tmpDir, ROWS); + return CONGLOMERATE; } - public static SpecificSegmentsQuerySegmentWalker createWalker(final File tmpDir, final List rows) + public static SpecificSegmentsQuerySegmentWalker createMockWalker(final File tmpDir) { - final QueryableIndex index = IndexBuilder.create() - .tmpDir(tmpDir) - .indexMerger(TestHelper.getTestIndexMergerV9()) - .schema( - new IncrementalIndexSchema.Builder() - .withMetrics( - new AggregatorFactory[]{ - new CountAggregatorFactory("cnt"), - new DoubleSumAggregatorFactory("m1", "m1"), - new HyperUniquesAggregatorFactory("unique_dim1", "dim1") - } - ) - .withRollup(false) - .build() - ) - .rows(rows) - .buildMMappedIndex(); - - final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate( - ImmutableMap., QueryRunnerFactory>builder() - .put( - SegmentMetadataQuery.class, - new SegmentMetadataQueryRunnerFactory( - new SegmentMetadataQueryQueryToolChest( - new SegmentMetadataQueryConfig("P1W") - ), - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) - ) - .put( - SelectQuery.class, - new SelectQueryRunnerFactory( - new SelectQueryQueryToolChest( - TestHelper.getObjectMapper(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ), - new SelectQueryEngine(), - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) - ) - .put( - TimeseriesQuery.class, - new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ), - new TimeseriesQueryEngine(), - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) - ) - .put( - TopNQuery.class, - new TopNQueryRunnerFactory( - new StupidPool<>( - "TopNQueryRunnerFactory-bufferPool", - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(10 * 1024 * 1024); - } - } - ), - new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ), - QueryRunnerTestHelper.NOOP_QUERYWATCHER - ) - ) - .put( - GroupByQuery.class, - GroupByQueryRunnerTest.makeQueryRunnerFactory( - GroupByQueryRunnerTest.DEFAULT_MAPPER, - new GroupByQueryConfig() - { - @Override - public String getDefaultStrategy() - { - return GroupByStrategySelector.STRATEGY_V2; - } - }, - new DruidProcessingConfig() - { - @Override - public String getFormatString() - { - return null; - } - - @Override - public int intermediateComputeSizeBytes() - { - return 10 * 1024 * 1024; - } + final QueryableIndex index1 = IndexBuilder.create() + .tmpDir(new File(tmpDir, "1")) + .indexMerger(TestHelper.getTestIndexMergerV9()) + .schema(INDEX_SCHEMA) + .rows(ROWS1) + .buildMMappedIndex(); - @Override - public int getNumMergeBuffers() - { - // Need 2 buffers for CalciteQueryTest.testDoubleNestedGroupby. - return 2; - } - } - ) - ) - .build() - ); + final QueryableIndex index2 = IndexBuilder.create() + .tmpDir(new File(tmpDir, "2")) + .indexMerger(TestHelper.getTestIndexMergerV9()) + .schema(INDEX_SCHEMA) + .rows(ROWS2) + .buildMMappedIndex(); - return new SpecificSegmentsQuerySegmentWalker(conglomerate).add( + return new SpecificSegmentsQuerySegmentWalker(queryRunnerFactoryConglomerate()).add( DataSegment.builder() - .dataSource(DATASOURCE) - .interval(index.getDataInterval()) + .dataSource(DATASOURCE1) + .interval(index1.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) .build(), - index + index1 + ).add( + DataSegment.builder() + .dataSource(DATASOURCE2) + .interval(index2.getDataInterval()) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .build(), + index2 ); } - public static DruidTable createDruidTable(final QuerySegmentWalker walker, final PlannerConfig plannerConfig) + public static DruidOperatorTable createOperatorTable() { - return new DruidTable(new QueryMaker(walker, plannerConfig), new TableDataSource(DATASOURCE), COLUMN_TYPES); + return new DruidOperatorTable(ImmutableSet.of(new ApproxCountDistinctSqlAggregator())); } public static Schema createMockSchema(final QuerySegmentWalker walker, final PlannerConfig plannerConfig) { - final DruidTable druidTable = createDruidTable(walker, plannerConfig); - final Map tableMap = ImmutableMap.of(DATASOURCE, druidTable); + final QueryMaker queryMaker = new QueryMaker(walker, plannerConfig); + final DruidTable druidTable1 = new DruidTable(queryMaker, new TableDataSource(DATASOURCE1), COLUMN_TYPES); + final DruidTable druidTable2 = new DruidTable(queryMaker, new TableDataSource(DATASOURCE2), COLUMN_TYPES); + final Map tableMap = ImmutableMap.of( + DATASOURCE1, druidTable1, + DATASOURCE2, druidTable2 + ); return new AbstractSchema() { @Override @@ -265,8 +301,20 @@ protected Map getTableMap() }; } - private static InputRow ROW(final ImmutableMap map) + public static InputRow createRow(final ImmutableMap map) { return PARSER.parse((Map) map); } + + public static InputRow createRow(final Object t, final String dim1, final String dim2, final double m1) + { + return PARSER.parse( + ImmutableMap.of( + "t", new DateTime(t).getMillis(), + "dim1", dim1, + "dim2", dim2, + "m1", m1 + ) + ); + } } diff --git a/sql/src/test/java/io/druid/sql/calcite/util/QueryLogHook.java b/sql/src/test/java/io/druid/sql/calcite/util/QueryLogHook.java new file mode 100644 index 000000000000..fb65bfe80e75 --- /dev/null +++ b/sql/src/test/java/io/druid/sql/calcite/util/QueryLogHook.java @@ -0,0 +1,106 @@ +/* + * 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.sql.calcite.util; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.logger.Logger; +import io.druid.query.Query; +import org.apache.calcite.runtime.Hook; +import org.junit.rules.TestRule; +import org.junit.runner.Description; +import org.junit.runners.model.Statement; + +import java.util.List; + +/** + * JUnit Rule that adds a Calcite hook to log and remember Druid queries. + */ +public class QueryLogHook implements TestRule +{ + private static final Logger log = new Logger(QueryLogHook.class); + + private final ObjectMapper objectMapper; + private final List recordedQueries = Lists.newCopyOnWriteArrayList(); + + public QueryLogHook(final ObjectMapper objectMapper) + { + this.objectMapper = objectMapper; + } + + public static QueryLogHook create() + { + return new QueryLogHook(new DefaultObjectMapper()); + } + + public static QueryLogHook create(final ObjectMapper objectMapper) + { + return new QueryLogHook(objectMapper); + } + + public void clearRecordedQueries() + { + recordedQueries.clear(); + } + + public List getRecordedQueries() + { + return ImmutableList.copyOf(recordedQueries); + } + + @Override + public Statement apply(final Statement base, final Description description) + { + return new Statement() + { + @Override + public void evaluate() throws Throwable + { + clearRecordedQueries(); + + final Function function = new Function() + { + @Override + public Object apply(final Object query) + { + try { + recordedQueries.add((Query) query); + log.info( + "Issued query: %s", + objectMapper.writerWithDefaultPrettyPrinter().writeValueAsString(query) + ); + } + catch (Exception e) { + log.warn(e, "Failed to serialize query: %s", query); + } + return null; + } + }; + + try (final Hook.Closeable unhook = Hook.QUERY_PLAN.add(function)) { + base.evaluate(); + } + } + }; + } +} diff --git a/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index 5c80e1a6e5fe..c9b88af530a2 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -81,6 +81,7 @@ public SpecificSegmentsQuerySegmentWalker add( final VersionedIntervalTimeline timeline = timelines.get(descriptor.getDataSource()); timeline.add(descriptor.getInterval(), descriptor.getVersion(), descriptor.getShardSpec().createChunk(segment)); segments.add(descriptor); + closeables.add(index); return this; } From e731c50ed1709fc49a3e007375a8ae0d8b42773b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 24 Jan 2017 11:48:47 +0900 Subject: [PATCH 05/10] Fix compilation failure --- .../java/util/common/guava/Sequences.java | 2 +- .../druid/sql/calcite/CalciteQueryTest.java | 46 +------------------ 2 files changed, 2 insertions(+), 46 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java index 1146c947f73b..a9da58ce25df 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java @@ -120,7 +120,7 @@ public static Sequence wrap(Sequence seq, SequenceWrapper wrapper) return new WrappingSequence<>(seq, wrapper); } - public static Sequence withEffect(final Sequence seq, final Runnable effect, final Executor exec) + public static Sequence withEffect(final Sequence seq, final Runnable effect, final Executor exec) { // Uses YieldingSequenceBase to be able to execute the effect if all elements of the wrapped seq are processed // (i. e. it "is done"), but the yielder of the underlying seq throws some exception from close(). This logic could diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java index 8c8647e855fe..b1ce7dedf562 100644 --- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java @@ -1698,51 +1698,6 @@ public void testApproxCountDistinct() throws Exception ); } - @Test - public void testApproxCountDistinct() throws Exception - { - testQuery( - "SELECT\n" - + " SUM(cnt),\n" - + " APPROX_COUNT_DISTINCT(dim2),\n" // uppercase - + " approx_count_distinct(dim2) FILTER(WHERE dim2 <> ''),\n" // lowercase; also, filtered - + " approx_count_distinct(unique_dim1)\n" // on native hyperUnique column - + "FROM druid.foo", - ImmutableList.of( - Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(QSS(Filtration.eternity())) - .granularity(QueryGranularities.ALL) - .aggregators( - AGGS( - new LongSumAggregatorFactory("a0", "cnt"), - new CardinalityAggregatorFactory( - "a1", - null, - DIMS(new DefaultDimensionSpec("dim2", "dim2")), - false - ), - new FilteredAggregatorFactory( - new CardinalityAggregatorFactory( - "a2", - null, - DIMS(new DefaultDimensionSpec("dim2", "dim2")), - false - ), - NOT(SELECTOR("dim2", "", null)) - ), - new HyperUniquesAggregatorFactory("a3", "unique_dim1") - ) - ) - .context(TIMESERIES_CONTEXT) - .build() - ), - ImmutableList.of( - new Object[]{6L, 3L, 2L, 6L} - ) - ); - } - @Test public void testDoubleNestedGroupBy() throws Exception { @@ -3097,6 +3052,7 @@ private void verifyResults( { for (int i = 0; i < results.size(); i++) { log.info("row #%d: %s", i, Arrays.toString(results.get(i))); + } Assert.assertEquals(String.format("result count: %s", sql), expectedResults.size(), results.size()); for (int i = 0; i < results.size(); i++) { From 4f114144f6e43e7ac51eb817086acfddc25497e9 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 31 Jan 2017 17:45:42 +0900 Subject: [PATCH 06/10] Address comments --- .../collections/OrderedMergeSequence.java | 8 +- .../druid/common/guava/CombiningSequence.java | 10 +- docs/content/querying/groupbyquery.md | 8 +- docs/content/querying/sql.md | 12 +- .../java/util/common/guava/BaseSequence.java | 8 +- .../util/common/guava/ConcatSequence.java | 8 +- .../util/common/guava/FilteredSequence.java | 8 +- .../java/util/common/guava/LazySequence.java | 8 +- .../util/common/guava/LimitedSequence.java | 4 +- .../util/common/guava/MappedSequence.java | 8 +- .../java/util/common/guava/MergeSequence.java | 4 +- .../java/util/common/guava/Sequences.java | 16 +- .../common/guava/YieldingSequenceBase.java | 4 +- .../java/util/common/guava/TestSequence.java | 8 +- .../common/guava/UnsupportedSequence.java | 4 +- .../java/io/druid/query/RetryQueryRunner.java | 4 +- .../spec/SpecificSegmentQueryRunner.java | 12 +- .../NestedGroupByQueryRunnerFailureTest.java | 145 ++++++++++++++++++ .../spec/SpecificSegmentQueryRunnerTest.java | 4 +- .../coordination/ServerManagerTest.java | 4 +- .../druid/sql/calcite/rule/GroupByRules.java | 4 +- .../druid/sql/calcite/CalciteQueryTest.java | 31 +++- 22 files changed, 247 insertions(+), 75 deletions(-) create mode 100644 processing/src/test/java/io/druid/query/groupby/NestedGroupByQueryRunnerFailureTest.java diff --git a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java index 9c88be318ed0..43313fdb3434 100644 --- a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java +++ b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java @@ -71,12 +71,12 @@ public OutType accumulate(OutType initValue, Accumulator a @Override public OutType accumulate( - Supplier initValue, Accumulator accumulator + Supplier initValSupplier, Accumulator accumulator ) { Yielder yielder = null; try { - yielder = toYielder(initValue, YieldingAccumulators.fromAccumulator(accumulator)); + yielder = toYielder(initValSupplier, YieldingAccumulators.fromAccumulator(accumulator)); return yielder.get(); } finally { @@ -92,13 +92,13 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { final PriorityQueue> pQueue = makePriorityQueue(); final Yielder> oldDudeAtCrosswalk = makeOldDudeAtCrosswalk(); - return makeYielder(pQueue, oldDudeAtCrosswalk, initValue.get(), accumulator); + return makeYielder(pQueue, oldDudeAtCrosswalk, initValSupplier.get(), accumulator); } private PriorityQueue> makePriorityQueue() diff --git a/common/src/main/java/io/druid/common/guava/CombiningSequence.java b/common/src/main/java/io/druid/common/guava/CombiningSequence.java index 4bf95d8b8932..2334d023ea13 100644 --- a/common/src/main/java/io/druid/common/guava/CombiningSequence.java +++ b/common/src/main/java/io/druid/common/guava/CombiningSequence.java @@ -67,15 +67,15 @@ public OutType accumulate(OutType initValue, final Accumulator OutType accumulate( - Supplier initValue, Accumulator accumulator + Supplier initValSupplier, Accumulator accumulator ) { - final CombiningAccumulator combiningAccumulator = new CombiningAccumulator<>(initValue, accumulator); + final CombiningAccumulator combiningAccumulator = new CombiningAccumulator<>(initValSupplier, accumulator); T lastValue = baseSequence.accumulate((T) null, combiningAccumulator); if (combiningAccumulator.accumulatedSomething()) { return accumulator.accumulate(combiningAccumulator.retVal, lastValue); } else { - return initValue.get(); + return initValSupplier.get(); } } @@ -87,14 +87,14 @@ public Yielder toYielder(OutType initValue, final YieldingAcc @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { final CombiningYieldingAccumulator combiningAccumulator = new CombiningYieldingAccumulator<>( ordering, mergeFn, accumulator ); - combiningAccumulator.setInitValSupplier(initValue); + combiningAccumulator.setInitValSupplier(initValSupplier); Yielder baseYielder = baseSequence.toYielder((T) null, combiningAccumulator); return makeYielder(baseYielder, combiningAccumulator, false); diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index a33da89f3219..734c456c0327 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -157,10 +157,10 @@ inner query's results stream with off-heap fact map and on-heap string dictionar strategy perform the outer query on the broker in a single-threaded fashion. Note that groupBys require a separate merge buffer on the broker for each layer beyond the first layer of the groupBy. -With the v2 groupBy strategy, this can potentially lead to deadlocks for groupBys nested beyond two layers, since the -merge buffers are limited in number and are acquired one-by-one and not as a complete set. At this time we recommend -that you avoid deeply-nested groupBys with the v2 strategy. Doubly-nested groupBys (groupBy -> groupBy -> table) are -safe and do not suffer from this issue. +This merge buffer is immediately released once they are not used anymore during the query processing, +but two or more concurrent nested groupBys can potentially lead to deadlocks since the merge buffers are limited in number +and are acquired one-by-one instead of a complete set. At this time we recommend that you avoid too many concurrent +execution of groupBys with the v2 strategy. #### Server configuration diff --git a/docs/content/querying/sql.md b/docs/content/querying/sql.md index b9b94ecf3321..b1903b6424bc 100644 --- a/docs/content/querying/sql.md +++ b/docs/content/querying/sql.md @@ -116,12 +116,12 @@ exact distinct count using a nested groupBy. SELECT COUNT(*) FROM (SELECT DISTINCT col FROM data_source) ``` -Note that groupBys require a separate merge buffer on the broker for each layer beyond the first layer of the groupBy. -With the v2 groupBy strategy, this can potentially lead to deadlocks for groupBys nested beyond two layers, since the -merge buffers are limited in number and are acquired one-by-one and not as a complete set. At this time we recommend -that you avoid deeply-nested groupBys with the v2 strategy. Doubly-nested groupBys (groupBy -> groupBy -> table) are -safe and do not suffer from this issue. If you like, you can forbid deeper nesting by setting -`druid.sql.planner.maxQueryCount = 2`. +For executing nested groupBys with the v2 groupBy strategy, you need to set `druid.processing.numMergeBuffers` to at least 2. +This is because groupBys require a separate merge buffer on the broker for each layer beyond the first layer of the groupBy. +This merge buffer is immediately released once they are not used anymore during the query processing, +but two or more concurrent nested groupBys can potentially lead to deadlocks since the merge buffers are limited in number +and are acquired one-by-one instead of a complete set. At this time we recommend that you avoid too many concurrent +execution of groupBys with the v2 strategy. #### Semi-joins diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java index 8d45083b1556..fd596f36550b 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java @@ -47,12 +47,12 @@ public OutType accumulate(OutType initValue, final Accumulator OutType accumulate(Supplier initValue, final Accumulator fn) + public OutType accumulate(Supplier initValSupplier, final Accumulator fn) { final IterType iterator = maker.make(); // initValue.get() is called here to guarantee some kind of initialization for initValue is executed // after making the iterator - final OutType retVal = initValue.get(); + final OutType retVal = initValSupplier.get(); return accumulate(iterator, retVal, fn); } @@ -97,7 +97,7 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { final IterType iterator = maker.make(); @@ -105,7 +105,7 @@ public Yielder toYielder( try { // initValue.get() is called here to guarantee some kind of initialization for initValue is executed // after making the iterator - return makeYielder(initValue.get(), accumulator, iterator); + return makeYielder(initValSupplier.get(), accumulator, iterator); } catch (Throwable t) { try { diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java index 7ef71c87d917..a82cd8568003 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java @@ -56,11 +56,11 @@ public OutType accumulate(OutType accumulated, Sequence in) @Override public OutType accumulate( - final Supplier initValue, final Accumulator accumulator + final Supplier initValSupplier, final Accumulator accumulator ) { return baseSequences.accumulate( - initValue, new Accumulator>() + initValSupplier, new Accumulator>() { @Override public OutType accumulate(OutType accumulated, Sequence in) @@ -82,7 +82,7 @@ public Yielder toYielder( @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { Yielder> yielderYielder = baseSequences.toYielder( @@ -99,7 +99,7 @@ public Sequence accumulate(Sequence accumulated, Sequence in) ); try { - return makeYielder(yielderYielder, initValue, accumulator); + return makeYielder(yielderYielder, initValSupplier, accumulator); } catch (Throwable t) { try { diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/FilteredSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/FilteredSequence.java index fe805f497d77..5369c599e200 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/FilteredSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/FilteredSequence.java @@ -48,10 +48,10 @@ public OutType accumulate(OutType initValue, Accumulator a @Override public OutType accumulate( - Supplier initValue, Accumulator accumulator + Supplier initValSupplier, Accumulator accumulator ) { - return baseSequence.accumulate(initValue, new FilteringAccumulator<>(pred, accumulator)); + return baseSequence.accumulate(initValSupplier, new FilteringAccumulator<>(pred, accumulator)); } @Override @@ -66,14 +66,14 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { final FilteringYieldingAccumulator filteringAccumulator = new FilteringYieldingAccumulator<>( pred, accumulator ); - return wrapYielder(baseSequence.toYielder(initValue, filteringAccumulator), filteringAccumulator); + return wrapYielder(baseSequence.toYielder(initValSupplier, filteringAccumulator), filteringAccumulator); } private Yielder wrapYielder( diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/LazySequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/LazySequence.java index 75b7d56a927c..108f1751170c 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/LazySequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/LazySequence.java @@ -42,10 +42,10 @@ public OutType accumulate(OutType initValue, Accumulator a @Override public OutType accumulate( - Supplier initValue, Accumulator accumulator + Supplier initValSupplier, Accumulator accumulator ) { - return provider.get().accumulate(initValue, accumulator); + return provider.get().accumulate(initValSupplier, accumulator); } @Override @@ -56,9 +56,9 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { - return provider.get().toYielder(initValue, accumulator); + return provider.get().toYielder(initValSupplier, accumulator); } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java index 5706c6f053be..f0f1bdef3776 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java @@ -57,13 +57,13 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { final LimitedYieldingAccumulator limitedAccumulator = new LimitedYieldingAccumulator<>( accumulator ); - final Yielder subYielder = baseSequence.toYielder(initValue, limitedAccumulator); + final Yielder subYielder = baseSequence.toYielder(initValSupplier, limitedAccumulator); return new LimitedYielder<>(subYielder, limitedAccumulator); } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MappedSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/MappedSequence.java index f9c3a42e8a0e..356ece5d3211 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MappedSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MappedSequence.java @@ -46,10 +46,10 @@ public OutType accumulate(OutType initValue, Accumulator @Override public OutType accumulate( - Supplier initValue, Accumulator accumulator + Supplier initValSupplier, Accumulator accumulator ) { - return baseSequence.accumulate(initValue, new MappingAccumulator<>(fn, accumulator)); + return baseSequence.accumulate(initValSupplier, new MappingAccumulator<>(fn, accumulator)); } @Override @@ -60,9 +60,9 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { - return baseSequence.toYielder(initValue, new MappingYieldingAccumulator<>(fn, accumulator)); + return baseSequence.toYielder(initValSupplier, new MappingYieldingAccumulator<>(fn, accumulator)); } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java index 36b66e454804..1dd3430de8b2 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java @@ -54,12 +54,12 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { final PriorityQueue> pQueue = makePriorityQueue(); - return makeYielder(pQueue, initValue.get(), accumulator); + return makeYielder(pQueue, initValSupplier.get(), accumulator); } private PriorityQueue> makePriorityQueue() diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java index a9da58ce25df..5ae5c7bf4f28 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java @@ -131,17 +131,17 @@ public static Sequence withEffect(final Sequence seq, final Runnable e return new YieldingSequenceBase() { @Override - public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) + public Yielder toYielder(OutType initValSupplier, YieldingAccumulator accumulator) { - return new ExecuteWhenDoneYielder<>(seq.toYielder(initValue, accumulator), effect, exec); + return new ExecuteWhenDoneYielder<>(seq.toYielder(initValSupplier, accumulator), effect, exec); } @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { - return new ExecuteWhenDoneYielder<>(seq.toYielder(initValue, accumulator), effect, exec); + return new ExecuteWhenDoneYielder<>(seq.toYielder(initValSupplier, accumulator), effect, exec); } }; } @@ -169,10 +169,10 @@ public OutType accumulate(OutType initValue, Accumulator OutType accumulate( - Supplier initValue, Accumulator accumulator + Supplier initValSupplier, Accumulator accumulator ) { - return initValue.get(); + return initValSupplier.get(); } @Override @@ -183,10 +183,10 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { - return Yielders.done(initValue.get(), null); + return Yielders.done(initValSupplier.get(), null); } } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java b/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java index c02c692e505e..7171b03c16b5 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java @@ -36,9 +36,9 @@ public OutType accumulate(OutType initValue, Accumulator a } @Override - public OutType accumulate(Supplier initValue, Accumulator accumulator) + public OutType accumulate(Supplier initValSupplier, Accumulator accumulator) { - Yielder yielder = toYielder(initValue, YieldingAccumulators.fromAccumulator(accumulator)); + Yielder yielder = toYielder(initValSupplier, YieldingAccumulators.fromAccumulator(accumulator)); try { return yielder.get(); diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java b/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java index 09a0761e341f..964516bbf340 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java @@ -65,10 +65,10 @@ public OutType accumulate(OutType initValue, Accumulator a @Override public OutType accumulate( - Supplier initValue, Accumulator accumulator + Supplier initValSupplier, Accumulator accumulator ) { - return base.accumulate(initValue, accumulator); + return base.accumulate(initValSupplier, accumulator); } @Override @@ -79,10 +79,10 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { - return base.toYielder(initValue, accumulator); + return base.toYielder(initValSupplier, accumulator); } public boolean isClosed() diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/UnsupportedSequence.java b/java-util/src/test/java/io/druid/java/util/common/guava/UnsupportedSequence.java index c56f367c244d..1c84d6579f34 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/UnsupportedSequence.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/UnsupportedSequence.java @@ -35,7 +35,7 @@ public OutType accumulate( @Override public OutType accumulate( - Supplier initValue, Accumulator accumulator + Supplier initValSupplier, Accumulator accumulator ) { throw new UnsupportedOperationException(); @@ -51,7 +51,7 @@ public Yielder toYielder( @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { throw new UnsupportedOperationException(); diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java index 2359e640acdd..f63a81d1eac3 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -79,10 +79,10 @@ public Yielder toYielder( @Override public Yielder toYielder( - Supplier initValue, YieldingAccumulator accumulator + Supplier initValSupplier, YieldingAccumulator accumulator ) { - return makeSequence().toYielder(initValue, accumulator); + return makeSequence().toYielder(initValSupplier, accumulator); } private Sequence makeSequence() diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java index ca48564da0b1..50ef93012254 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -85,15 +85,15 @@ public OutType accumulate(final OutType initValue, final Accumulator OutType accumulate( - final Supplier initValue, final Accumulator accumulator + final Supplier initValSupplier, final Accumulator accumulator ) { try { - return baseSequence.accumulate(initValue, accumulator); + return baseSequence.accumulate(initValSupplier, accumulator); } catch (SegmentMissingException e) { appendMissingSegment(responseContext); - return initValue.get(); + return initValSupplier.get(); } } @@ -108,15 +108,15 @@ public Yielder toYielder( @Override public Yielder toYielder( - final Supplier initValue, final YieldingAccumulator accumulator + final Supplier initValSupplier, final YieldingAccumulator accumulator ) { try { - return makeYielder(baseSequence.toYielder(initValue, accumulator)); + return makeYielder(baseSequence.toYielder(initValSupplier, accumulator)); } catch (SegmentMissingException e) { appendMissingSegment(responseContext); - return Yielders.done(initValue.get(), null); + return Yielders.done(initValSupplier.get(), null); } } diff --git a/processing/src/test/java/io/druid/query/groupby/NestedGroupByQueryRunnerFailureTest.java b/processing/src/test/java/io/druid/query/groupby/NestedGroupByQueryRunnerFailureTest.java new file mode 100644 index 000000000000..ce18248d2aa6 --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/NestedGroupByQueryRunnerFailureTest.java @@ -0,0 +1,145 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.groupby; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.MoreExecutors; +import io.druid.data.input.Row; +import io.druid.granularity.QueryGranularities; +import io.druid.query.DruidProcessingConfig; +import io.druid.query.QueryContextKeys; +import io.druid.query.QueryDataSource; +import io.druid.query.QueryInterruptedException; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.DimensionSpec; +import org.bouncycastle.util.Integers; +import org.hamcrest.CoreMatchers; +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.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.TimeoutException; + +@RunWith(Parameterized.class) +public class NestedGroupByQueryRunnerFailureTest +{ + public static final DruidProcessingConfig DEFAULT_PROCESSING_CONFIG = new DruidProcessingConfig() + { + + @Override + public String getFormatString() + { + return null; + } + + @Override + public int intermediateComputeSizeBytes() + { + return 10 * 1024 * 1024; + } + + @Override + public int getNumMergeBuffers() + { + return 1; + } + + @Override + public int getNumThreads() + { + return 2; + } + }; + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private static final GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory( + GroupByQueryRunnerTest.DEFAULT_MAPPER, + new GroupByQueryConfig() + { + public String getDefaultStrategy() + { + return "v2"; + } + }, + DEFAULT_PROCESSING_CONFIG + ); + + private QueryRunner runner; + + @Parameters(name = "{0}") + public static Collection constructorFeeder() throws IOException + { + final List args = Lists.newArrayList(); + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { + args.add(new Object[]{runner}); + } + return args; + } + + public NestedGroupByQueryRunnerFailureTest(QueryRunner runner) + { + this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.>of(runner)); + } + + @Test(timeout = 10000) + public void testLackOfMergeBuffers() throws IOException + { + expectedException.expect(QueryInterruptedException.class); + expectedException.expectCause(CoreMatchers.instanceOf(TimeoutException.class)); + + final GroupByQuery query = GroupByQuery + .builder() + .setDataSource( + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval(QueryRunnerTestHelper.firstToThird) + .setGranularity(QueryGranularities.ALL) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec( + "quality", + "alias" + ))) + .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) + .build() + ) + ) + .setGranularity(QueryGranularities.ALL) + .setInterval(QueryRunnerTestHelper.firstToThird) + .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows"))) + .setContext(ImmutableMap.of(QueryContextKeys.TIMEOUT, Integers.valueOf(500))) + .build(); + + GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + } +} diff --git a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java index ac55da67317f..6b847e6a26c9 100644 --- a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java @@ -75,7 +75,7 @@ public Sequence run(Query query, Map responseContext) { @Override public Object accumulate( - Supplier initValue, Accumulator accumulator + Supplier initValSupplier, Accumulator accumulator ) { throw new SegmentMissingException("FAILSAUCE"); @@ -96,7 +96,7 @@ public Yielder toYielder( } @Override - public Yielder toYielder(Supplier initValue, YieldingAccumulator accumulator) + public Yielder toYielder(Supplier initValSupplier, YieldingAccumulator accumulator) { throw new SegmentMissingException("FAILSAUCE"); } diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 82dbbece5c38..6a6a4b657fed 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -719,7 +719,7 @@ public Yielder toYielder( @Override public Yielder toYielder( - final Supplier initValue, final YieldingAccumulator accumulator + final Supplier initValSupplier, final YieldingAccumulator accumulator ) { notifyLatch.countDown(); @@ -731,7 +731,7 @@ public Yielder toYielder( throw Throwables.propagate(e); } - final Yielder baseYielder = baseSequence.toYielder(initValue, accumulator); + final Yielder baseYielder = baseSequence.toYielder(initValSupplier, accumulator); return new Yielder() { @Override diff --git a/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java b/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java index f724b6177f5e..885296da1108 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java +++ b/sql/src/main/java/io/druid/sql/calcite/rule/GroupByRules.java @@ -476,15 +476,13 @@ private static DruidRel applyAggregate( if (isNestedQuery) { // Nested groupBy. - final DruidNestedGroupBy retVal = DruidNestedGroupBy.from( + return DruidNestedGroupBy.from( druidRel, filter, Grouping.create(dimensions, aggregations), aggregate.getRowType(), rowOrder ); - - return retVal; } else { // groupBy on a base dataSource. return druidRel.withQueryBuilder( diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java index b1ce7dedf562..3c184da997e9 100644 --- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java @@ -1713,7 +1713,36 @@ public void testDoubleNestedGroupBy() throws Exception + " ) t1\n" + " GROUP BY dim2\n" + ") t2", - null, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + GroupByQuery.builder() + .setDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(QSS(Filtration.eternity())) + .setGranularity(QueryGranularities.ALL) + .setDimensions(DIMS( + new DefaultDimensionSpec("dim1", "d0"), + new DefaultDimensionSpec("dim2", "d1") + )) + .setAggregatorSpecs(AGGS(new CountAggregatorFactory("a0"))) + .build() + ) + .setInterval(QSS(Filtration.eternity())) + .setGranularity(QueryGranularities.ALL) + .setDimensions(DIMS(new DefaultDimensionSpec("d1", "d0"))) + .setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "a0"))) + .build() + ) + .setInterval(QSS(Filtration.eternity())) + .setGranularity(QueryGranularities.ALL) + .setAggregatorSpecs(AGGS( + new LongSumAggregatorFactory("a0", "a0"), + new CountAggregatorFactory("a1") + )) + .build() + ), ImmutableList.of( new Object[]{6L, 3L} ) From 8a7970034160c091c6dc8dedf57fb97e5a79bd10 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 8 Feb 2017 15:46:58 +0900 Subject: [PATCH 07/10] Address comments - Revert Sequence - Add isInitialized() to Grouper - Initialize the grouper in RowBasedGrouperHelper.Accumulator - Simple refactoring RowBasedGrouperHelper.Accumulator - Add tests for checking the number of used merge buffers - Improve docs --- .../io/druid/collections/BlockingPool.java | 2 +- .../collections/OrderedMergeSequence.java | 65 ++-- .../druid/common/guava/CombiningSequence.java | 53 +-- docs/content/querying/groupbyquery.md | 11 +- docs/content/querying/sql.md | 11 +- .../java/util/common/guava/BaseSequence.java | 40 -- .../util/common/guava/ConcatSequence.java | 80 +--- .../util/common/guava/FilteredSequence.java | 21 -- .../java/util/common/guava/LazySequence.java | 16 - .../util/common/guava/LimitedSequence.java | 13 - .../util/common/guava/MappedSequence.java | 17 - .../java/util/common/guava/MergeSequence.java | 24 +- .../java/util/common/guava/Sequence.java | 30 +- .../java/util/common/guava/Sequences.java | 31 +- .../util/common/guava/WrappingSequence.java | 21 +- .../common/guava/YieldingSequenceBase.java | 11 +- .../java/util/common/guava/TestSequence.java | 18 - .../common/guava/UnsupportedSequence.java | 18 - .../java/io/druid/query/RetryQueryRunner.java | 22 +- .../groupby/epinephelinae/BufferGrouper.java | 17 +- .../epinephelinae/ConcurrentGrouper.java | 70 ++-- .../epinephelinae/GroupByRowProcessor.java | 10 +- .../query/groupby/epinephelinae/Grouper.java | 7 + .../epinephelinae/RowBasedGrouperHelper.java | 168 +++++++-- .../epinephelinae/SpillingGrouper.java | 6 + .../spec/SpecificSegmentQueryRunner.java | 25 +- .../groupby/GroupByQueryMergeBufferTest.java | 345 ++++++++++++++++++ ...ava => GroupByQueryRunnerFailureTest.java} | 9 +- .../epinephelinae/ConcurrentGrouperTest.java | 212 +++++++++++ .../spec/SpecificSegmentQueryRunnerTest.java | 17 +- .../coordination/ServerManagerTest.java | 12 +- 31 files changed, 859 insertions(+), 543 deletions(-) create mode 100644 processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java rename processing/src/test/java/io/druid/query/groupby/{NestedGroupByQueryRunnerFailureTest.java => GroupByQueryRunnerFailureTest.java} (94%) create mode 100644 processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java diff --git a/common/src/main/java/io/druid/collections/BlockingPool.java b/common/src/main/java/io/druid/collections/BlockingPool.java index ebe834890348..9afc03b42732 100644 --- a/common/src/main/java/io/druid/collections/BlockingPool.java +++ b/common/src/main/java/io/druid/collections/BlockingPool.java @@ -37,7 +37,7 @@ public class BlockingPool { private static final Logger log = new Logger(BlockingPool.class); - private final BlockingQueue objects; + protected final BlockingQueue objects; public BlockingPool( Supplier generator, diff --git a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java index 43313fdb3434..9bd32c9dc0c7 100644 --- a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java +++ b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java @@ -20,10 +20,9 @@ package io.druid.collections; import com.google.common.base.Function; -import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import com.google.common.collect.Ordering; + import com.google.common.io.Closer; import io.druid.java.util.common.guava.Accumulator; import io.druid.java.util.common.guava.CloseQuietly; @@ -39,13 +38,13 @@ /** * An OrderedMergeIterator is an iterator that merges together multiple sorted iterators. It is written assuming * that the input Iterators are provided in order. That is, it places an extra restriction in the input iterators. - *

+ * * Normally a merge operation could operate with the actual input iterators in any order as long as the actual values * in the iterators are sorted. This requires that not only the individual values be sorted, but that the iterators * be provided in the order of the first element of each iterator. - *

+ * * If this doesn't make sense, check out OrderedMergeSequenceTest.testScrewsUpOnOutOfOrderBeginningOfList() - *

+ * * It places this extra restriction on the input data in order to implement an optimization that allows it to * remain as lazy as possible in the face of a common case where the iterators are just appended one after the other. */ @@ -65,18 +64,10 @@ public OrderedMergeSequence( @Override public OutType accumulate(OutType initValue, Accumulator accumulator) - { - return accumulate(Suppliers.ofInstance(initValue), accumulator); - } - - @Override - public OutType accumulate( - Supplier initValSupplier, Accumulator accumulator - ) { Yielder yielder = null; try { - yielder = toYielder(initValSupplier, YieldingAccumulators.fromAccumulator(accumulator)); + yielder = toYielder(initValue, YieldingAccumulators.fromAccumulator(accumulator)); return yielder.get(); } finally { @@ -87,23 +78,7 @@ public OutType accumulate( @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { - return toYielder(Suppliers.ofInstance(initValue), accumulator); - } - - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) - { - final PriorityQueue> pQueue = makePriorityQueue(); - final Yielder> oldDudeAtCrosswalk = makeOldDudeAtCrosswalk(); - - return makeYielder(pQueue, oldDudeAtCrosswalk, initValSupplier.get(), accumulator); - } - - private PriorityQueue> makePriorityQueue() - { - return new PriorityQueue>( + PriorityQueue> pQueue = new PriorityQueue>( 32, ordering.onResultOf( new Function, T>() @@ -116,19 +91,16 @@ public T apply(Yielder input) } ) ); - } - private Yielder> makeOldDudeAtCrosswalk() - { - return sequences.toYielder( - (Yielder) null, + Yielder> oldDudeAtCrosswalk = sequences.toYielder( + null, new YieldingAccumulator, Sequence>() { @Override public Yielder accumulate(Yielder accumulated, Sequence in) { final Yielder retVal = in.toYielder( - (T) null, + null, new YieldingAccumulator() { @Override @@ -148,7 +120,8 @@ public T accumulate(T accumulated, T in) throw Throwables.propagate(e); } return null; - } else { + } + else { yield(); } @@ -156,6 +129,8 @@ public T accumulate(T accumulated, T in) } } ); + + return makeYielder(pQueue, oldDudeAtCrosswalk, initValue, accumulator); } private Yielder makeYielder( @@ -170,16 +145,19 @@ private Yielder makeYielder( Yielder yielder; if (oldDudeAtCrosswalk.isDone()) { yielder = pQueue.remove(); - } else if (pQueue.isEmpty()) { + } + else if (pQueue.isEmpty()) { yielder = oldDudeAtCrosswalk.get(); oldDudeAtCrosswalk = oldDudeAtCrosswalk.next(null); - } else { + } + else { Yielder queueYielder = pQueue.peek(); Yielder iterYielder = oldDudeAtCrosswalk.get(); if (ordering.compare(queueYielder.get(), iterYielder.get()) <= 0) { yielder = pQueue.remove(); - } else { + } + else { yielder = oldDudeAtCrosswalk.get(); oldDudeAtCrosswalk = oldDudeAtCrosswalk.next(null); } @@ -194,7 +172,8 @@ private Yielder makeYielder( catch (IOException e) { throw Throwables.propagate(e); } - } else { + } + else { pQueue.add(yielder); } } @@ -230,7 +209,7 @@ public boolean isDone() public void close() throws IOException { Closer closer = Closer.create(); - while (!pQueue.isEmpty()) { + while(!pQueue.isEmpty()) { closer.register(pQueue.remove()); } closer.close(); diff --git a/common/src/main/java/io/druid/common/guava/CombiningSequence.java b/common/src/main/java/io/druid/common/guava/CombiningSequence.java index 2334d023ea13..e3ccc40453a4 100644 --- a/common/src/main/java/io/druid/common/guava/CombiningSequence.java +++ b/common/src/main/java/io/druid/common/guava/CombiningSequence.java @@ -19,8 +19,6 @@ package io.druid.common.guava; -import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; import com.google.common.collect.Ordering; import io.druid.java.util.common.guava.Accumulator; import io.druid.java.util.common.guava.Sequence; @@ -62,40 +60,24 @@ public CombiningSequence( @Override public OutType accumulate(OutType initValue, final Accumulator accumulator) { - return accumulate(Suppliers.ofInstance(initValue), accumulator); - } - - @Override - public OutType accumulate( - Supplier initValSupplier, Accumulator accumulator - ) - { - final CombiningAccumulator combiningAccumulator = new CombiningAccumulator<>(initValSupplier, accumulator); - T lastValue = baseSequence.accumulate((T) null, combiningAccumulator); + final CombiningAccumulator combiningAccumulator = new CombiningAccumulator<>(initValue, accumulator); + T lastValue = baseSequence.accumulate(null, combiningAccumulator); if (combiningAccumulator.accumulatedSomething()) { return accumulator.accumulate(combiningAccumulator.retVal, lastValue); } else { - return initValSupplier.get(); + return initValue; } } @Override public Yielder toYielder(OutType initValue, final YieldingAccumulator accumulator) - { - return toYielder(Suppliers.ofInstance(initValue), accumulator); - } - - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) { final CombiningYieldingAccumulator combiningAccumulator = new CombiningYieldingAccumulator<>( ordering, mergeFn, accumulator ); - combiningAccumulator.setInitValSupplier(initValSupplier); - Yielder baseYielder = baseSequence.toYielder((T) null, combiningAccumulator); + combiningAccumulator.setRetVal(initValue); + Yielder baseYielder = baseSequence.toYielder(null, combiningAccumulator); return makeYielder(baseYielder, combiningAccumulator, false); } @@ -110,22 +92,23 @@ private Yielder makeYielder( final OutType retVal; final boolean finalFinalValue; - if (!yielder.isDone()) { + if(!yielder.isDone()) { retVal = combiningAccumulator.getRetVal(); finalYielder = null; finalFinalValue = false; } else { - if (!finalValue && combiningAccumulator.accumulatedSomething()) { + if(!finalValue && combiningAccumulator.accumulatedSomething()) { combiningAccumulator.accumulateLastValue(); retVal = combiningAccumulator.getRetVal(); finalFinalValue = true; - if (!combiningAccumulator.yielded()) { + if(!combiningAccumulator.yielded()) { return Yielders.done(retVal, yielder); } else { finalYielder = Yielders.done(null, yielder); } - } else { + } + else { return Yielders.done(combiningAccumulator.getRetVal(), yielder); } } @@ -171,7 +154,6 @@ private static class CombiningYieldingAccumulator extends YieldingAc private final YieldingAccumulator accumulator; private OutType retVal; - private Supplier initValSupplier; private T lastMergedVal; private boolean accumulatedSomething = false; @@ -188,12 +170,12 @@ private static class CombiningYieldingAccumulator extends YieldingAc public OutType getRetVal() { - return accumulatedSomething ? retVal : initValSupplier.get(); + return retVal; } - public void setInitValSupplier(Supplier initValSupplier) + public void setRetVal(OutType retVal) { - this.initValSupplier = initValSupplier; + this.retVal = retVal; } public YieldingAccumulator getAccumulator() @@ -221,7 +203,6 @@ public T accumulate(T prevValue, T t) { if (!accumulatedSomething) { accumulatedSomething = true; - retVal = initValSupplier.get(); } if (prevValue == null) { @@ -241,7 +222,7 @@ public T accumulate(T prevValue, T t) void accumulateLastValue() { - retVal = accumulator.accumulate(getRetVal(), lastMergedVal); + retVal = accumulator.accumulate(retVal, lastMergedVal); } boolean accumulatedSomething() @@ -253,14 +234,13 @@ boolean accumulatedSomething() private class CombiningAccumulator implements Accumulator { private OutType retVal; - private Supplier initValSupplier; private final Accumulator accumulator; private volatile boolean accumulatedSomething = false; - CombiningAccumulator(Supplier initValSupplier, Accumulator accumulator) + CombiningAccumulator(OutType retVal, Accumulator accumulator) { - this.initValSupplier = initValSupplier; + this.retVal = retVal; this.accumulator = accumulator; } @@ -274,7 +254,6 @@ public T accumulate(T prevValue, T t) { if (!accumulatedSomething) { accumulatedSomething = true; - retVal = initValSupplier.get(); } if (prevValue == null) { diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index 734c456c0327..5f30d5695add 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -157,10 +157,10 @@ inner query's results stream with off-heap fact map and on-heap string dictionar strategy perform the outer query on the broker in a single-threaded fashion. Note that groupBys require a separate merge buffer on the broker for each layer beyond the first layer of the groupBy. -This merge buffer is immediately released once they are not used anymore during the query processing, -but two or more concurrent nested groupBys can potentially lead to deadlocks since the merge buffers are limited in number -and are acquired one-by-one instead of a complete set. At this time we recommend that you avoid too many concurrent -execution of groupBys with the v2 strategy. +This merge buffer is immediately released once they are not used anymore during the query processing. However, deeply +nested groupBys (there are two or more groupBy layers beyond the first one) can potentially lead to deadlocks since the +merge buffers are limited in number and are acquired one-by-one instead of a complete set. At this time, we recommend +that you avoid too many concurrent execution of deeply nested groupBys with the v2 strategy. #### Server configuration @@ -185,7 +185,8 @@ When using the "v2" strategy, the following runtime properties apply: Additionally, the "v2" strategy uses merging buffers for merging. It is currently the only query implementation that does so. By default, Druid is configured without any merging buffer pool, so to use the "v2" strategy you must also -set `druid.processing.numMergeBuffers` to some non-zero number. +set `druid.processing.numMergeBuffers` to some non-zero number. Furthermore, if you want to execute deeply nested gropuBys, +you must set `druid.processing.numMergeBuffers` to at least 2. This may require allocating more direct memory. The amount of direct memory needed by Druid is at least `druid.processing.buffer.sizeBytes * (druid.processing.numMergeBuffers + druid.processing.numThreads + 1)`. You can diff --git a/docs/content/querying/sql.md b/docs/content/querying/sql.md index b1903b6424bc..d1987f09a8ba 100644 --- a/docs/content/querying/sql.md +++ b/docs/content/querying/sql.md @@ -116,12 +116,11 @@ exact distinct count using a nested groupBy. SELECT COUNT(*) FROM (SELECT DISTINCT col FROM data_source) ``` -For executing nested groupBys with the v2 groupBy strategy, you need to set `druid.processing.numMergeBuffers` to at least 2. -This is because groupBys require a separate merge buffer on the broker for each layer beyond the first layer of the groupBy. -This merge buffer is immediately released once they are not used anymore during the query processing, -but two or more concurrent nested groupBys can potentially lead to deadlocks since the merge buffers are limited in number -and are acquired one-by-one instead of a complete set. At this time we recommend that you avoid too many concurrent -execution of groupBys with the v2 strategy. +Note that groupBys require a separate merge buffer on the broker for each layer beyond the first layer of the groupBy. +This merge buffer is immediately released once they are not used anymore during the query processing. However, deeply +nested groupBys (there are two or more groupBy layers beyond the first one) can potentially lead to deadlocks since the +merge buffers are limited in number and are acquired one-by-one instead of a complete set. At this time, we recommend +that you avoid too many concurrent execution of deeply nested groupBys with the v2 strategy. #### Semi-joins diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java index fd596f36550b..e715a3d21908 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/BaseSequence.java @@ -19,8 +19,6 @@ package io.druid.java.util.common.guava; -import com.google.common.base.Supplier; - import java.io.Closeable; import java.io.IOException; import java.util.Iterator; @@ -43,21 +41,6 @@ public BaseSequence( public OutType accumulate(OutType initValue, final Accumulator fn) { IterType iterator = maker.make(); - return accumulate(iterator, initValue, fn); - } - - @Override - public OutType accumulate(Supplier initValSupplier, final Accumulator fn) - { - final IterType iterator = maker.make(); - // initValue.get() is called here to guarantee some kind of initialization for initValue is executed - // after making the iterator - final OutType retVal = initValSupplier.get(); - return accumulate(iterator, retVal, fn); - } - - private OutType accumulate(IterType iterator, OutType initValue, final Accumulator fn) - { try { while (iterator.hasNext()) { initValue = fn.accumulate(initValue, iterator.next()); @@ -95,29 +78,6 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat } } - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) - { - final IterType iterator = maker.make(); - - try { - // initValue.get() is called here to guarantee some kind of initialization for initValue is executed - // after making the iterator - return makeYielder(initValSupplier.get(), accumulator, iterator); - } - catch (Throwable t) { - try { - maker.cleanup(iterator); - } - catch (Exception e) { - t.addSuppressed(e); - } - throw t; - } - } - private Yielder makeYielder( OutType initValue, final YieldingAccumulator accumulator, diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java index a82cd8568003..7479b7c0a952 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/ConcatSequence.java @@ -19,8 +19,6 @@ package io.druid.java.util.common.guava; -import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import java.io.Closeable; @@ -44,30 +42,13 @@ public OutType accumulate(OutType initValue, final Accumulator>() - { - @Override - public OutType accumulate(OutType accumulated, Sequence in) - { - return in.accumulate(accumulated, accumulator); - } - } - ); - } - - @Override - public OutType accumulate( - final Supplier initValSupplier, final Accumulator accumulator - ) - { - return baseSequences.accumulate( - initValSupplier, new Accumulator>() - { - @Override - public OutType accumulate(OutType accumulated, Sequence in) - { - return in.accumulate(accumulated, accumulator); - } - } + { + @Override + public OutType accumulate(OutType accumulated, Sequence in) + { + return in.accumulate(accumulated, accumulator); + } + } ); } @@ -76,17 +57,9 @@ public Yielder toYielder( final OutType initValue, final YieldingAccumulator accumulator ) - { - return toYielder(Suppliers.ofInstance(initValue), accumulator); - } - - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) { Yielder> yielderYielder = baseSequences.toYielder( - (Sequence) null, + null, new YieldingAccumulator, Sequence>() { @Override @@ -99,7 +72,7 @@ public Sequence accumulate(Sequence accumulated, Sequence in) ); try { - return makeYielder(yielderYielder, initValSupplier, accumulator); + return makeYielder(yielderYielder, initValue, accumulator); } catch (Throwable t) { try { @@ -114,21 +87,17 @@ public Sequence accumulate(Sequence accumulated, Sequence in) public Yielder makeYielder( Yielder> yielderYielder, - Supplier initValSupplier, + OutType initValue, YieldingAccumulator accumulator ) { - if (yielderYielder.isDone()) { - return Yielders.done(initValSupplier.get(), yielderYielder); - } else { - // pass the supplier to get the first Yielder - Yielder yielder = yielderYielder.get().toYielder(initValSupplier, accumulator); - + while (!yielderYielder.isDone()) { + Yielder yielder = yielderYielder.get().toYielder(initValue, accumulator); if (accumulator.yielded()) { return wrapYielder(yielder, yielderYielder, accumulator); } - OutType initVal = yielder.get(); + initValue = yielder.get(); try { yielder.close(); } @@ -137,26 +106,9 @@ public Yielder makeYielder( } yielderYielder = yielderYielder.next(null); - - while (!yielderYielder.isDone()) { - yielder = yielderYielder.get().toYielder(initVal, accumulator); - if (accumulator.yielded()) { - return wrapYielder(yielder, yielderYielder, accumulator); - } - - initVal = yielder.get(); - try { - yielder.close(); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - - yielderYielder = yielderYielder.next(null); - } - - return Yielders.done(initVal, yielderYielder); } + + return Yielders.done(initValue, yielderYielder); } private Yielder wrapYielder( @@ -174,7 +126,7 @@ private Yielder wrapYielder( throw Throwables.propagate(e); } - return makeYielder(yielderYielder.next(null), Suppliers.ofInstance(nextInit), accumulator); + return makeYielder(yielderYielder.next(null), nextInit, accumulator); } return new Yielder() diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/FilteredSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/FilteredSequence.java index 5369c599e200..8fc52344403b 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/FilteredSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/FilteredSequence.java @@ -20,7 +20,6 @@ package io.druid.java.util.common.guava; import com.google.common.base.Predicate; -import com.google.common.base.Supplier; import java.io.IOException; @@ -46,14 +45,6 @@ public OutType accumulate(OutType initValue, Accumulator a return baseSequence.accumulate(initValue, new FilteringAccumulator<>(pred, accumulator)); } - @Override - public OutType accumulate( - Supplier initValSupplier, Accumulator accumulator - ) - { - return baseSequence.accumulate(initValSupplier, new FilteringAccumulator<>(pred, accumulator)); - } - @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { @@ -64,18 +55,6 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat return wrapYielder(baseSequence.toYielder(initValue, filteringAccumulator), filteringAccumulator); } - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) - { - final FilteringYieldingAccumulator filteringAccumulator = new FilteringYieldingAccumulator<>( - pred, accumulator - ); - - return wrapYielder(baseSequence.toYielder(initValSupplier, filteringAccumulator), filteringAccumulator); - } - private Yielder wrapYielder( final Yielder yielder, final FilteringYieldingAccumulator accumulator ) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/LazySequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/LazySequence.java index 108f1751170c..d832a6a95bb8 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/LazySequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/LazySequence.java @@ -40,25 +40,9 @@ public OutType accumulate(OutType initValue, Accumulator a return provider.get().accumulate(initValue, accumulator); } - @Override - public OutType accumulate( - Supplier initValSupplier, Accumulator accumulator - ) - { - return provider.get().accumulate(initValSupplier, accumulator); - } - @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { return provider.get().toYielder(initValue, accumulator); } - - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) - { - return provider.get().toYielder(initValSupplier, accumulator); - } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java index f0f1bdef3776..5e7f005a4b80 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/LimitedSequence.java @@ -20,7 +20,6 @@ package io.druid.java.util.common.guava; import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; import java.io.IOException; @@ -55,18 +54,6 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat return new LimitedYielder<>(subYielder, limitedAccumulator); } - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) - { - final LimitedYieldingAccumulator limitedAccumulator = new LimitedYieldingAccumulator<>( - accumulator - ); - final Yielder subYielder = baseSequence.toYielder(initValSupplier, limitedAccumulator); - return new LimitedYielder<>(subYielder, limitedAccumulator); - } - private class LimitedYielder implements Yielder { private final Yielder subYielder; diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MappedSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/MappedSequence.java index 356ece5d3211..31d189b72312 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MappedSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MappedSequence.java @@ -20,7 +20,6 @@ package io.druid.java.util.common.guava; import com.google.common.base.Function; -import com.google.common.base.Supplier; /** */ @@ -44,25 +43,9 @@ public OutType accumulate(OutType initValue, Accumulator return baseSequence.accumulate(initValue, new MappingAccumulator<>(fn, accumulator)); } - @Override - public OutType accumulate( - Supplier initValSupplier, Accumulator accumulator - ) - { - return baseSequence.accumulate(initValSupplier, new MappingAccumulator<>(fn, accumulator)); - } - @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { return baseSequence.toYielder(initValue, new MappingYieldingAccumulator<>(fn, accumulator)); } - - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) - { - return baseSequence.toYielder(initValSupplier, new MappingYieldingAccumulator<>(fn, accumulator)); - } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java index 1dd3430de8b2..0293927992da 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java @@ -20,7 +20,6 @@ package io.druid.java.util.common.guava; import com.google.common.base.Function; -import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Ordering; import com.google.common.io.Closer; @@ -46,23 +45,6 @@ public MergeSequence( @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) - { - final PriorityQueue> pQueue = makePriorityQueue(); - - return makeYielder(pQueue, initValue, accumulator); - } - - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) - { - final PriorityQueue> pQueue = makePriorityQueue(); - - return makeYielder(pQueue, initValSupplier.get(), accumulator); - } - - private PriorityQueue> makePriorityQueue() { PriorityQueue> pQueue = new PriorityQueue<>( 32, @@ -78,7 +60,7 @@ public T apply(Yielder input) ) ); - return baseSequences.accumulate( + pQueue = baseSequences.accumulate( pQueue, new Accumulator>, Sequence>() { @@ -86,7 +68,7 @@ public T apply(Yielder input) public PriorityQueue> accumulate(PriorityQueue> queue, Sequence in) { final Yielder yielder = in.toYielder( - (T) null, + null, new YieldingAccumulator() { @Override @@ -113,6 +95,8 @@ public T accumulate(T accumulated, T in) } } ); + + return makeYielder(pQueue, initValue, accumulator); } private Yielder makeYielder( diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java index 2ec1456b5095..4cd28d4d6c32 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequence.java @@ -19,8 +19,6 @@ package io.druid.java.util.common.guava; -import com.google.common.base.Supplier; - /** * A Sequence represents an iterable sequence of elements. Unlike normal Iterators however, it doesn't expose * a way for you to extract values from it, instead you provide it with a worker (an Accumulator) and that defines @@ -47,19 +45,7 @@ public interface Sequence */ OutType accumulate(OutType initValue, Accumulator accumulator); - /** - * Accumulate this sequence using the given accumulator. - * The {@code initValSupplier} provides an way for lazy evaluation of the initial value. - * - * @param initValSupplier the supplier which returns an initial value to pass along to start the accumulation. - * @param accumulator the accumulator which is responsible for accumulating input values. - * @param the type of accumulated value. - * - * @return accumulated value. - */ - OutType accumulate(Supplier initValSupplier, Accumulator accumulator); - - /** + /** * Return an Yielder for accumulated sequence. * * @param initValue the initial value to pass along to start the accumulation. @@ -71,18 +57,4 @@ public interface Sequence * @see Yielder */ Yielder toYielder(OutType initValue, YieldingAccumulator accumulator); - - /** - * Return an Yielder for accumulated sequence. - * The {@code initValSupplier} provides an way for lazy evaluation of the initial value. - * - * @param initValSupplier the supplier which returns an initial value to pass along to start the accumulation. - * @param accumulator the accumulator which is responsible for accumulating input values. - * @param the type of accumulated value. - * - * @return an Yielder for accumulated sequence. - * - * @see Yielder - */ - Yielder toYielder(Supplier initValSupplier, YieldingAccumulator accumulator); } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java index 5ae5c7bf4f28..c132ce99d2c2 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java @@ -22,7 +22,6 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; -import com.google.common.base.Supplier; import com.google.common.collect.Lists; import java.io.Closeable; @@ -120,7 +119,7 @@ public static Sequence wrap(Sequence seq, SequenceWrapper wrapper) return new WrappingSequence<>(seq, wrapper); } - public static Sequence withEffect(final Sequence seq, final Runnable effect, final Executor exec) + public static Sequence withEffect(final Sequence seq, final Runnable effect, final Executor exec) { // Uses YieldingSequenceBase to be able to execute the effect if all elements of the wrapped seq are processed // (i. e. it "is done"), but the yielder of the underlying seq throws some exception from close(). This logic could @@ -131,17 +130,9 @@ public static Sequence withEffect(final Sequence seq, final Runnable e return new YieldingSequenceBase() { @Override - public Yielder toYielder(OutType initValSupplier, YieldingAccumulator accumulator) + public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { - return new ExecuteWhenDoneYielder<>(seq.toYielder(initValSupplier, accumulator), effect, exec); - } - - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) - { - return new ExecuteWhenDoneYielder<>(seq.toYielder(initValSupplier, accumulator), effect, exec); + return new ExecuteWhenDoneYielder<>(seq.toYielder(initValue, accumulator), effect, exec); } }; } @@ -167,26 +158,10 @@ public OutType accumulate(OutType initValue, Accumulator OutType accumulate( - Supplier initValSupplier, Accumulator accumulator - ) - { - return initValSupplier.get(); - } - @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { return Yielders.done(initValue, null); } - - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) - { - return Yielders.done(initValSupplier.get(), null); - } } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/WrappingSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/WrappingSequence.java index 810c525e8cfd..a9276acaa4c5 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/WrappingSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/WrappingSequence.java @@ -21,7 +21,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; import com.google.common.base.Throwables; /** @@ -39,14 +38,6 @@ final class WrappingSequence implements Sequence @Override public OutType accumulate(final OutType outType, final Accumulator accumulator) - { - return accumulate(Suppliers.ofInstance(outType), accumulator); - } - - @Override - public OutType accumulate( - final Supplier initValSupplier, final Accumulator accumulator - ) { OutType result; try { @@ -56,7 +47,7 @@ public OutType accumulate( @Override public OutType get() { - return baseSequence.accumulate(initValSupplier, accumulator); + return baseSequence.accumulate(outType, accumulator); } }); } @@ -85,14 +76,6 @@ public Yielder toYielder( final OutType initValue, final YieldingAccumulator accumulator ) - { - return toYielder(Suppliers.ofInstance(initValue), accumulator); - } - - @Override - public Yielder toYielder( - final Supplier initValSupplier, final YieldingAccumulator accumulator - ) { try { wrapper.before(); @@ -101,7 +84,7 @@ public Yielder toYielder( @Override public Yielder get() { - return new WrappingYielder<>(baseSequence.toYielder(initValSupplier, accumulator), wrapper); + return new WrappingYielder<>(baseSequence.toYielder(initValue, accumulator), wrapper); } }); } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java b/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java index 7171b03c16b5..c3150c0a818c 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/YieldingSequenceBase.java @@ -19,9 +19,6 @@ package io.druid.java.util.common.guava; -import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; - /** * A Sequence that is based entirely on the Yielder implementation. *

@@ -32,13 +29,7 @@ public abstract class YieldingSequenceBase implements Sequence @Override public OutType accumulate(OutType initValue, Accumulator accumulator) { - return accumulate(Suppliers.ofInstance(initValue), accumulator); - } - - @Override - public OutType accumulate(Supplier initValSupplier, Accumulator accumulator) - { - Yielder yielder = toYielder(initValSupplier, YieldingAccumulators.fromAccumulator(accumulator)); + Yielder yielder = toYielder(initValue, YieldingAccumulators.fromAccumulator(accumulator)); try { return yielder.get(); diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java b/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java index 964516bbf340..950b5046925e 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/TestSequence.java @@ -19,8 +19,6 @@ package io.druid.java.util.common.guava; -import com.google.common.base.Supplier; - import java.io.Closeable; import java.util.Arrays; import java.util.concurrent.atomic.AtomicBoolean; @@ -63,28 +61,12 @@ public OutType accumulate(OutType initValue, Accumulator a return base.accumulate(initValue, accumulator); } - @Override - public OutType accumulate( - Supplier initValSupplier, Accumulator accumulator - ) - { - return base.accumulate(initValSupplier, accumulator); - } - @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { return base.toYielder(initValue, accumulator); } - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) - { - return base.toYielder(initValSupplier, accumulator); - } - public boolean isClosed() { return closed.get(); diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/UnsupportedSequence.java b/java-util/src/test/java/io/druid/java/util/common/guava/UnsupportedSequence.java index 1c84d6579f34..f410fbefdc1e 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/UnsupportedSequence.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/UnsupportedSequence.java @@ -19,8 +19,6 @@ package io.druid.java.util.common.guava; -import com.google.common.base.Supplier; - /** */ public class UnsupportedSequence implements Sequence @@ -33,14 +31,6 @@ public OutType accumulate( throw new UnsupportedOperationException(); } - @Override - public OutType accumulate( - Supplier initValSupplier, Accumulator accumulator - ) - { - throw new UnsupportedOperationException(); - } - @Override public Yielder toYielder( OutType initValue, YieldingAccumulator accumulator @@ -48,12 +38,4 @@ public Yielder toYielder( { throw new UnsupportedOperationException(); } - - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) - { - throw new UnsupportedOperationException(); - } } diff --git a/processing/src/main/java/io/druid/query/RetryQueryRunner.java b/processing/src/main/java/io/druid/query/RetryQueryRunner.java index f63a81d1eac3..57a5cde5124b 100644 --- a/processing/src/main/java/io/druid/query/RetryQueryRunner.java +++ b/processing/src/main/java/io/druid/query/RetryQueryRunner.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Supplier; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -73,19 +72,6 @@ public Sequence run(final Query query, final Map context) public Yielder toYielder( OutType initValue, YieldingAccumulator accumulator ) - { - return makeSequence().toYielder(initValue, accumulator); - } - - @Override - public Yielder toYielder( - Supplier initValSupplier, YieldingAccumulator accumulator - ) - { - return makeSequence().toYielder(initValSupplier, accumulator); - } - - private Sequence makeSequence() { List missingSegments = getMissingSegments(context); @@ -114,10 +100,12 @@ private Sequence makeSequence() return new MergeSequence<>( query.getResultOrdering(), - Sequences.simple(listOfSequences) + Sequences.simple(listOfSequences)).toYielder( + initValue, accumulator ); - } else { - return Iterables.getOnlyElement(listOfSequences); + } + else { + return Iterables.getOnlyElement(listOfSequences).toYielder(initValue, accumulator); } } }; diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java index e356867e3941..4987d34b6b14 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/BufferGrouper.java @@ -93,6 +93,8 @@ public class BufferGrouper implements Grouper // Maximum number of elements in the table before it must be resized private int maxSize; + private boolean initialized = false; + public BufferGrouper( final Supplier bufferSupplier, final KeySerde keySerde, @@ -129,9 +131,18 @@ public BufferGrouper( @Override public void init() { - this.buffer = bufferSupplier.get(); - this.tableArenaSize = (buffer.capacity() / (bucketSize + Ints.BYTES)) * bucketSize; - reset(); + if (!initialized) { + this.buffer = bufferSupplier.get(); + this.tableArenaSize = (buffer.capacity() / (bucketSize + Ints.BYTES)) * bucketSize; + reset(); + initialized = true; + } + } + + @Override + public boolean isInitialized() + { + return initialized; } @Override diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java index d086155f8234..27b1016d0a63 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouper.java @@ -63,6 +63,8 @@ public class ConcurrentGrouper implements Grouper private final int concurrencyHint; private final KeySerdeFactory keySerdeFactory; + private volatile boolean initialized = false; + public ConcurrentGrouper( final Supplier bufferSupplier, final KeySerdeFactory keySerdeFactory, @@ -104,33 +106,51 @@ protected SpillingGrouper initialValue() @Override public void init() { - final ByteBuffer buffer = bufferSupplier.get(); - final int sliceSize = (buffer.capacity() / concurrencyHint); - - for (int i = 0; i < concurrencyHint; i++) { - final ByteBuffer slice = buffer.duplicate(); - slice.position(sliceSize * i); - slice.limit(slice.position() + sliceSize); - final SpillingGrouper grouper = new SpillingGrouper<>( - Suppliers.ofInstance(slice.slice()), - keySerdeFactory, - columnSelectorFactory, - aggregatorFactories, - bufferGrouperMaxSize, - bufferGrouperMaxLoadFactor, - bufferGrouperInitialBuckets, - temporaryStorage, - spillMapper, - false - ); - grouper.init(); - groupers.add(grouper); + if (!initialized) { + synchronized (bufferSupplier) { + if (!initialized) { + final ByteBuffer buffer = bufferSupplier.get(); + final int sliceSize = (buffer.capacity() / concurrencyHint); + + for (int i = 0; i < concurrencyHint; i++) { + final ByteBuffer slice = buffer.duplicate(); + slice.position(sliceSize * i); + slice.limit(slice.position() + sliceSize); + final SpillingGrouper grouper = new SpillingGrouper<>( + Suppliers.ofInstance(slice.slice()), + keySerdeFactory, + columnSelectorFactory, + aggregatorFactories, + bufferGrouperMaxSize, + bufferGrouperMaxLoadFactor, + bufferGrouperInitialBuckets, + temporaryStorage, + spillMapper, + false + ); + grouper.init(); + groupers.add(grouper); + } + + initialized = true; + } + } } } + @Override + public boolean isInitialized() + { + return initialized; + } + @Override public boolean aggregate(KeyType key, int keyHash) { + if (!initialized) { + throw new ISE("Grouper is not initialized"); + } + if (closed) { throw new ISE("Grouper is closed"); } @@ -167,6 +187,10 @@ public boolean aggregate(KeyType key) @Override public void reset() { + if (!initialized) { + throw new ISE("Grouper is not initialized"); + } + if (closed) { throw new ISE("Grouper is closed"); } @@ -181,6 +205,10 @@ public void reset() @Override public Iterator> iterator(final boolean sorted) { + if (!initialized) { + throw new ISE("Grouper is not initialized"); + } + if (closed) { throw new ISE("Grouper is closed"); } diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java index 1e37270b9b9e..232776f70f99 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/GroupByRowProcessor.java @@ -177,15 +177,7 @@ public ByteBuffer get() closeOnFailure.add(grouper); final Grouper retVal = filteredSequence.accumulate( - new Supplier>() - { - @Override - public Grouper get() - { - grouper.init(); - return grouper; - } - }, + grouper, accumulator ); if (retVal != grouper) { diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java index 3680a11e42d2..2417e8961bc6 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/Grouper.java @@ -46,6 +46,13 @@ public interface Grouper extends Closeable */ void init(); + /** + * Check this grouper is initialized or not. + * + * @return true if the grouper is already initialized, otherwise false. + */ + boolean isInitialized(); + /** * Aggregate the current row with the provided key. Some implementations are thread-safe and * some are not. diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 5135b4b741bb..7a1598f0a96b 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -125,6 +125,14 @@ public static Pair, Accumulator, Row>> dimensionSelectors = null; } + final int keySize = includeTimestamp ? query.getDimensions().size() + 1 : query.getDimensions().size(); + final ValueExtractFunction valueExtractFn = makeValueExtractFunction( + query, + isInputRaw, + includeTimestamp, + dimensionSelectors + ); + final Accumulator, Row> accumulator = new Accumulator, Row>() { @Override @@ -142,42 +150,14 @@ public Grouper accumulate( return null; } - columnSelectorRow.set(row); - - final int dimStart; - final Comparable[] key; - - if (includeTimestamp) { - key = new Comparable[query.getDimensions().size() + 1]; - - final long timestamp; - if (isInputRaw) { - if (query.getGranularity() instanceof AllGranularity) { - timestamp = query.getIntervals().get(0).getStartMillis(); - } else { - timestamp = query.getGranularity().truncate(row.getTimestampFromEpoch()); - } - } else { - timestamp = row.getTimestampFromEpoch(); - } - - key[0] = timestamp; - dimStart = 1; - } else { - key = new Comparable[query.getDimensions().size()]; - dimStart = 0; + if (!theGrouper.isInitialized()) { + theGrouper.init(); } - for (int i = dimStart; i < key.length; i++) { - final String value; - if (isInputRaw) { - IndexedInts index = dimensionSelectors[i - dimStart].getRow(); - value = index.size() == 0 ? "" : dimensionSelectors[i - dimStart].lookupName(index.get(0)); - } else { - value = (String) row.getRaw(query.getDimensions().get(i - dimStart).getOutputName()); - } - key[i] = Strings.nullToEmpty(value); - } + columnSelectorRow.set(row); + + final Comparable[] key = new Comparable[keySize]; + valueExtractFn.apply(row, key); final boolean didAggregate = theGrouper.aggregate(new RowBasedKey(key)); if (!didAggregate) { @@ -193,6 +173,126 @@ public Grouper accumulate( return new Pair<>(grouper, accumulator); } + private interface TimestampExtractFunction + { + long apply(Row row); + } + + private static TimestampExtractFunction makeTimestampExtractFunction( + final GroupByQuery query, + final boolean isInputRaw + ) + { + if (isInputRaw) { + if (query.getGranularity() instanceof AllGranularity) { + return new TimestampExtractFunction() + { + @Override + public long apply(Row row) + { + return query.getIntervals().get(0).getStartMillis(); + } + }; + } else { + return new TimestampExtractFunction() + { + @Override + public long apply(Row row) + { + return query.getGranularity().truncate(row.getTimestampFromEpoch()); + } + }; + } + } else { + return new TimestampExtractFunction() + { + @Override + public long apply(Row row) + { + return row.getTimestampFromEpoch(); + } + }; + } + } + + private interface ValueExtractFunction + { + Comparable[] apply(Row row, Comparable[] key); + } + + private static ValueExtractFunction makeValueExtractFunction( + final GroupByQuery query, + final boolean isInputRaw, + final boolean includeTimestamp, + final DimensionSelector[] dimensionSelectors + ) + { + final TimestampExtractFunction timestampExtractFn = includeTimestamp ? + makeTimestampExtractFunction(query, isInputRaw) : + null; + if (isInputRaw) { + if (includeTimestamp) { + return new ValueExtractFunction() + { + @Override + public Comparable[] apply(Row row, Comparable[] key) + { + key[0] = timestampExtractFn.apply(row); + for (int i = 1; i < key.length; i++) { + final IndexedInts index = dimensionSelectors[i - 1].getRow(); + final String value = index.size() == 0 ? "" : dimensionSelectors[i - 1].lookupName(index.get(0)); + key[i] = Strings.nullToEmpty(value); + } + return key; + } + }; + } else { + return new ValueExtractFunction() + { + @Override + public Comparable[] apply(Row row, Comparable[] key) + { + for (int i = 0; i < key.length; i++) { + final IndexedInts index = dimensionSelectors[i].getRow(); + final String value = index.size() == 0 ? "" : dimensionSelectors[i].lookupName(index.get(0)); + key[i] = Strings.nullToEmpty(value); + } + return key; + } + }; + } + } else { + if (includeTimestamp) { + return new ValueExtractFunction() + { + @Override + public Comparable[] apply(Row row, Comparable[] key) + { + key[0] = timestampExtractFn.apply(row); + for (int i = 1; i < key.length; i++) { + final String value = (String) row.getRaw(query.getDimensions().get(i - 1).getOutputName()); + key[i] = Strings.nullToEmpty(value); + } + return key; + } + }; + } else { + return new ValueExtractFunction() + { + @Override + public Comparable[] apply(Row row, Comparable[] key) + { + for (int i = 0; i < key.length; i++) { + final String value = (String) row.getRaw(query.getDimensions().get(i).getOutputName()); + key[i] = Strings.nullToEmpty(value); + } + return key; + } + }; + } + } + } + public static CloseableGrouperIterator makeGrouperIterator( final Grouper grouper, final GroupByQuery query, diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java index f2561d7b20e9..c5a4317d58f9 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/SpillingGrouper.java @@ -99,6 +99,12 @@ public void init() grouper.init(); } + @Override + public boolean isInitialized() + { + return grouper.isInitialized(); + } + @Override public boolean aggregate(KeyType key, int keyHash) { diff --git a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java index 50ef93012254..c331ef7556d2 100644 --- a/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -20,7 +20,6 @@ package io.druid.query.spec; import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; import com.google.common.collect.Lists; import io.druid.java.util.common.guava.Accumulator; import io.druid.java.util.common.guava.Sequence; @@ -79,21 +78,13 @@ public Sequence get() { @Override public OutType accumulate(final OutType initValue, final Accumulator accumulator) - { - return accumulate(Suppliers.ofInstance(initValue), accumulator); - } - - @Override - public OutType accumulate( - final Supplier initValSupplier, final Accumulator accumulator - ) { try { - return baseSequence.accumulate(initValSupplier, accumulator); + return baseSequence.accumulate(initValue, accumulator); } catch (SegmentMissingException e) { appendMissingSegment(responseContext); - return initValSupplier.get(); + return initValue; } } @@ -102,21 +93,13 @@ public Yielder toYielder( final OutType initValue, final YieldingAccumulator accumulator ) - { - return toYielder(Suppliers.ofInstance(initValue), accumulator); - } - - @Override - public Yielder toYielder( - final Supplier initValSupplier, final YieldingAccumulator accumulator - ) { try { - return makeYielder(baseSequence.toYielder(initValSupplier, accumulator)); + return makeYielder(baseSequence.toYielder(initValue, accumulator)); } catch (SegmentMissingException e) { appendMissingSegment(responseContext); - return Yielders.done(initValSupplier.get(), null); + return Yielders.done(initValue, null); } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java new file mode 100644 index 000000000000..0e2da35bd3fa --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -0,0 +1,345 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.groupby; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; +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.ReferenceCountingResourceHolder; +import io.druid.collections.StupidPool; +import io.druid.data.input.Row; +import io.druid.granularity.QueryGranularities; +import io.druid.query.DruidProcessingConfig; +import io.druid.query.QueryContextKeys; +import io.druid.query.QueryDataSource; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.groupby.strategy.GroupByStrategySelector; +import io.druid.query.groupby.strategy.GroupByStrategyV1; +import io.druid.query.groupby.strategy.GroupByStrategyV2; +import org.bouncycastle.util.Integers; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +@RunWith(Parameterized.class) +public class GroupByQueryMergeBufferTest +{ + private static class TestBlockingPool extends BlockingPool + { + private int minRemainBufferNum; + + public TestBlockingPool(Supplier generator, int limit) + { + super(generator, limit); + minRemainBufferNum = limit; + } + + @Override + public ReferenceCountingResourceHolder take(final long timeout) throws InterruptedException + { + final ReferenceCountingResourceHolder holder = super.take(timeout); + if (minRemainBufferNum > objects.size()) { + minRemainBufferNum = objects.size(); + } + return holder; + } + + public void resetMinRemainBufferNum() + { + minRemainBufferNum = PROCESSING_CONFIG.getNumMergeBuffers(); + } + + public int getMinRemainBufferNum() + { + return minRemainBufferNum; + } + } + + public static final DruidProcessingConfig PROCESSING_CONFIG = new DruidProcessingConfig() + { + + @Override + public String getFormatString() + { + return null; + } + + @Override + public int intermediateComputeSizeBytes() + { + return 10 * 1024 * 1024; + } + + @Override + public int getNumMergeBuffers() + { + return 3; + } + + @Override + public int getNumThreads() + { + return 1; + } + }; + + private static GroupByQueryRunnerFactory makeQueryRunnerFactory( + final ObjectMapper mapper, + final GroupByQueryConfig config + ) + { + final Supplier configSupplier = Suppliers.ofInstance(config); + final StupidPool bufferPool = new StupidPool<>( + "GroupByQueryEngine-bufferPool", + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocateDirect(PROCESSING_CONFIG.intermediateComputeSizeBytes()); + } + } + ); + final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + configSupplier, + new GroupByStrategyV1( + configSupplier, + new GroupByQueryEngine(configSupplier, bufferPool), + QueryRunnerTestHelper.NOOP_QUERYWATCHER, + bufferPool + ), + new GroupByStrategyV2( + PROCESSING_CONFIG, + configSupplier, + bufferPool, + mergeBufferPool, + mapper, + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( + configSupplier, + strategySelector, + bufferPool, + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ); + return new GroupByQueryRunnerFactory( + strategySelector, + toolChest + ); + } + + private final static TestBlockingPool mergeBufferPool = new TestBlockingPool( + new Supplier() + { + @Override + public ByteBuffer get () + { + return ByteBuffer.allocateDirect(PROCESSING_CONFIG.intermediateComputeSizeBytes()); + } + }, + PROCESSING_CONFIG.getNumMergeBuffers() + ); + + private static final GroupByQueryRunnerFactory factory = makeQueryRunnerFactory( + GroupByQueryRunnerTest.DEFAULT_MAPPER, + new GroupByQueryConfig() + { + public String getDefaultStrategy() + { + return "v2"; + } + } + ); + + private QueryRunner runner; + + @Parameters(name = "{0}") + public static Collection constructorFeeder() throws IOException + { + final List args = Lists.newArrayList(); + for (QueryRunner runner : QueryRunnerTestHelper.makeQueryRunners(factory)) { + args.add(new Object[]{runner}); + } + return args; + } + + public GroupByQueryMergeBufferTest(QueryRunner runner) + { + this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.>of(runner)); + } + + @Before + public void setup() + { + mergeBufferPool.resetMinRemainBufferNum(); + } + + @Test + public void testSimpleGroupBy() + { + final GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setGranularity(QueryGranularities.ALL) + .setInterval(QueryRunnerTestHelper.firstToThird) + .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows"))) + .setContext(ImmutableMap.of(QueryContextKeys.TIMEOUT, Integers.valueOf(500))) + .build(); + + GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + + assertEquals(2, mergeBufferPool.getMinRemainBufferNum()); + } + + @Test + public void testNestedGroupBy() + { + final GroupByQuery query = GroupByQuery + .builder() + .setDataSource( + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval(QueryRunnerTestHelper.firstToThird) + .setGranularity(QueryGranularities.ALL) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) + .build() + ) + ) + .setGranularity(QueryGranularities.ALL) + .setInterval(QueryRunnerTestHelper.firstToThird) + .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows"))) + .setContext(ImmutableMap.of(QueryContextKeys.TIMEOUT, Integers.valueOf(500))) + .build(); + + GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + + assertEquals(1, mergeBufferPool.getMinRemainBufferNum()); + } + + @Test + public void testDoubleNestedGroupBy() + { + final GroupByQuery query = GroupByQuery + .builder() + .setDataSource( + new QueryDataSource( + GroupByQuery.builder() + .setDataSource( + GroupByQuery.builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval(QueryRunnerTestHelper.firstToThird) + .setGranularity(QueryGranularities.ALL) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("quality", "alias"), + new DefaultDimensionSpec("market", null) + )) + .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) + .build() + ) + .setInterval(QueryRunnerTestHelper.firstToThird) + .setGranularity(QueryGranularities.ALL) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) + .build() + ) + ) + .setGranularity(QueryGranularities.ALL) + .setInterval(QueryRunnerTestHelper.firstToThird) + .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows"))) + .setContext(ImmutableMap.of(QueryContextKeys.TIMEOUT, Integers.valueOf(500))) + .build(); + + GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + + assertEquals(1, mergeBufferPool.getMinRemainBufferNum()); + } + + @Test + public void testTripleNestedGroupBy() + { + final GroupByQuery query = GroupByQuery + .builder() + .setDataSource( + new QueryDataSource( + GroupByQuery.builder() + .setDataSource( + GroupByQuery.builder() + .setDataSource( + GroupByQuery.builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval(QueryRunnerTestHelper.firstToThird) + .setGranularity(QueryGranularities.ALL) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("quality", "alias"), + new DefaultDimensionSpec("market", null), + new DefaultDimensionSpec("placement", null) + )) + .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) + .build() + ) + .setInterval(QueryRunnerTestHelper.firstToThird) + .setGranularity(QueryGranularities.ALL) + .setDimensions(Lists.newArrayList( + new DefaultDimensionSpec("quality", "alias"), + new DefaultDimensionSpec("market", null) + )) + .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) + .build() + ) + .setInterval(QueryRunnerTestHelper.firstToThird) + .setGranularity(QueryGranularities.ALL) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) + .build() + ) + ) + .setGranularity(QueryGranularities.ALL) + .setInterval(QueryRunnerTestHelper.firstToThird) + .setAggregatorSpecs(Lists.newArrayList(new LongSumAggregatorFactory("rows", "rows"))) + .setContext(ImmutableMap.of(QueryContextKeys.TIMEOUT, Integers.valueOf(500))) + .build(); + + GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + + assertEquals(1, mergeBufferPool.getMinRemainBufferNum()); + } +} diff --git a/processing/src/test/java/io/druid/query/groupby/NestedGroupByQueryRunnerFailureTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java similarity index 94% rename from processing/src/test/java/io/druid/query/groupby/NestedGroupByQueryRunnerFailureTest.java rename to processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java index ce18248d2aa6..2489318d3a84 100644 --- a/processing/src/test/java/io/druid/query/groupby/NestedGroupByQueryRunnerFailureTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFailureTest.java @@ -50,7 +50,7 @@ import java.util.concurrent.TimeoutException; @RunWith(Parameterized.class) -public class NestedGroupByQueryRunnerFailureTest +public class GroupByQueryRunnerFailureTest { public static final DruidProcessingConfig DEFAULT_PROCESSING_CONFIG = new DruidProcessingConfig() { @@ -107,7 +107,7 @@ public static Collection constructorFeeder() throws IOException return args; } - public NestedGroupByQueryRunnerFailureTest(QueryRunner runner) + public GroupByQueryRunnerFailureTest(QueryRunner runner) { this.runner = factory.mergeRunners(MoreExecutors.sameThreadExecutor(), ImmutableList.>of(runner)); } @@ -126,10 +126,7 @@ public void testLackOfMergeBuffers() throws IOException .setDataSource(QueryRunnerTestHelper.dataSource) .setInterval(QueryRunnerTestHelper.firstToThird) .setGranularity(QueryGranularities.ALL) - .setDimensions(Lists.newArrayList(new DefaultDimensionSpec( - "quality", - "alias" - ))) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) .setAggregatorSpecs(Lists.newArrayList(QueryRunnerTestHelper.rowsCount)) .build() ) diff --git a/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java new file mode 100644 index 000000000000..c3440095a013 --- /dev/null +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/ConcurrentGrouperTest.java @@ -0,0 +1,212 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.groupby.epinephelinae; + +import com.google.common.base.Supplier; +import com.google.common.primitives.Longs; +import io.druid.java.util.common.IAE; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.groupby.epinephelinae.Grouper.KeyComparator; +import io.druid.query.groupby.epinephelinae.Grouper.KeySerde; +import io.druid.query.groupby.epinephelinae.Grouper.KeySerdeFactory; +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.DimensionSelector; +import io.druid.segment.FloatColumnSelector; +import io.druid.segment.LongColumnSelector; +import io.druid.segment.ObjectColumnSelector; +import io.druid.segment.column.ColumnCapabilities; +import org.junit.AfterClass; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; + +public class ConcurrentGrouperTest +{ + private static final ExecutorService service = Executors.newFixedThreadPool(8); + + @AfterClass + public static void teardown() + { + service.shutdown(); + } + + private static final Supplier bufferSupplier = new Supplier() + { + private final AtomicBoolean called = new AtomicBoolean(false); + + @Override + public ByteBuffer get() + { + if (called.compareAndSet(false, true)) { + return ByteBuffer.allocate(192); + } else { + throw new IAE("Cannot get twice"); + } + } + }; + + private static final KeySerdeFactory keySerdeFactory = new KeySerdeFactory() + { + @Override + public KeySerde factorize() + { + return new KeySerde() + { + final ByteBuffer buffer = ByteBuffer.allocate(8); + + @Override + public int keySize() + { + return 8; + } + + @Override + public Class keyClazz() + { + return Long.class; + } + + @Override + public ByteBuffer toByteBuffer(Long key) + { + buffer.rewind(); + buffer.putLong(key); + buffer.position(0); + return buffer; + } + + @Override + public Long fromByteBuffer(ByteBuffer buffer, int position) + { + return buffer.getLong(position); + } + + @Override + public KeyComparator bufferComparator() + { + return new KeyComparator() + { + @Override + public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition) + { + return Longs.compare(lhsBuffer.getLong(lhsPosition), rhsBuffer.getLong(rhsPosition)); + } + }; + } + + @Override + public void reset() {} + }; + } + + @Override + public Comparator objectComparator() + { + return new Comparator() + { + @Override + public int compare(Long o1, Long o2) + { + return o1.compareTo(o2); + } + }; + } + }; + + private static final ColumnSelectorFactory null_factory = new ColumnSelectorFactory() + { + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + return null; + } + + @Override + public FloatColumnSelector makeFloatColumnSelector(String columnName) + { + return null; + } + + @Override + public LongColumnSelector makeLongColumnSelector(String columnName) + { + return null; + } + + @Override + public ObjectColumnSelector makeObjectColumnSelector(String columnName) + { + return null; + } + + @Override + public ColumnCapabilities getColumnCapabilities(String columnName) + { + return null; + } + }; + + @Test + public void testAggregate() throws InterruptedException, ExecutionException + { + final ConcurrentGrouper grouper = new ConcurrentGrouper<>( + bufferSupplier, + keySerdeFactory, + null_factory, + new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, + 24, + 0.7f, + 1, + null, + null, + 8 + ); + + Future[] futures = new Future[8]; + + for (int i = 0; i < 8; i++) { + futures[i] = service.submit(new Runnable() + { + @Override + public void run() + { + grouper.init(); + for (long i = 0; i < 100; i++) { + grouper.aggregate(0L); + } + } + }); + } + + for (Future eachFuture : futures) { + eachFuture.get(); + } + + grouper.close(); + } +} diff --git a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java index 6b847e6a26c9..7c85101bba83 100644 --- a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java @@ -20,7 +20,6 @@ package io.druid.query.spec; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -73,14 +72,6 @@ public Sequence run(Query query, Map responseContext) { return new Sequence() { - @Override - public Object accumulate( - Supplier initValSupplier, Accumulator accumulator - ) - { - throw new SegmentMissingException("FAILSAUCE"); - } - @Override public Object accumulate(Object initValue, Accumulator accumulator) { @@ -94,12 +85,6 @@ public Yielder toYielder( { throw new SegmentMissingException("FAILSAUCE"); } - - @Override - public Yielder toYielder(Supplier initValSupplier, YieldingAccumulator accumulator) - { - throw new SegmentMissingException("FAILSAUCE"); - } }; } @@ -129,7 +114,7 @@ public Yielder toYielder(Supplier initValSupplier, YieldingAccumulator accumulat responseContext = Maps.newHashMap(); results = queryRunner.run(query, responseContext); results.toYielder( - (Object) null, new YieldingAccumulator() + null, new YieldingAccumulator() { final List lists = Lists.newArrayList(); @Override diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 6a6a4b657fed..b24ed2ba4287 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -22,8 +22,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -713,14 +711,6 @@ private BlockingSequence( public Yielder toYielder( final OutType initValue, final YieldingAccumulator accumulator ) - { - return toYielder(Suppliers.ofInstance(initValue), accumulator); - } - - @Override - public Yielder toYielder( - final Supplier initValSupplier, final YieldingAccumulator accumulator - ) { notifyLatch.countDown(); @@ -731,7 +721,7 @@ public Yielder toYielder( throw Throwables.propagate(e); } - final Yielder baseYielder = baseSequence.toYielder(initValSupplier, accumulator); + final Yielder baseYielder = baseSequence.toYielder(initValue, accumulator); return new Yielder() { @Override From 1ab663b39cab961975e1f6b01732af31ce6b9e18 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 10 Feb 2017 19:48:14 +0900 Subject: [PATCH 08/10] Revert unnecessary changes --- .../java/io/druid/common/guava/CombiningSequenceTest.java | 2 +- .../java/io/druid/common/guava/ComplexSequenceTest.java | 2 +- docs/content/querying/groupbyquery.md | 6 +++--- .../java/io/druid/query/scan/ScanQueryLimitRowIterator.java | 2 +- .../main/java/io/druid/java/util/common/guava/Yielders.java | 2 +- .../io/druid/java/util/common/guava/ConcatSequenceTest.java | 2 +- .../io/druid/query/aggregation/AggregationTestHelper.java | 2 +- services/src/main/java/io/druid/cli/DumpSegment.java | 2 +- .../main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java | 2 +- 9 files changed, 11 insertions(+), 11 deletions(-) diff --git a/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java b/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java index bf46b38f2868..040bac1739cb 100644 --- a/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java +++ b/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java @@ -258,7 +258,7 @@ public Pair apply( Assert.assertEquals(expected, merged); Yielder> yielder = seq.toYielder( - (Pair) null, + null, new YieldingAccumulator, Pair>() { int count = 0; diff --git a/common/src/test/java/io/druid/common/guava/ComplexSequenceTest.java b/common/src/test/java/io/druid/common/guava/ComplexSequenceTest.java index 528920042461..0e7db26ffb8e 100644 --- a/common/src/test/java/io/druid/common/guava/ComplexSequenceTest.java +++ b/common/src/test/java/io/druid/common/guava/ComplexSequenceTest.java @@ -54,7 +54,7 @@ private void check(String expected, Sequence complex) Assert.assertEquals(expected, combined.toString()); Yielder yielder = complex.toYielder( - (Integer) null, + null, new YieldingAccumulator() { @Override diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index 5f30d5695add..1a003c6edbd6 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -157,9 +157,9 @@ inner query's results stream with off-heap fact map and on-heap string dictionar strategy perform the outer query on the broker in a single-threaded fashion. Note that groupBys require a separate merge buffer on the broker for each layer beyond the first layer of the groupBy. -This merge buffer is immediately released once they are not used anymore during the query processing. However, deeply -nested groupBys (there are two or more groupBy layers beyond the first one) can potentially lead to deadlocks since the -merge buffers are limited in number and are acquired one-by-one instead of a complete set. At this time, we recommend +This merge buffer is immediately released once they are not used anymore during the query processing. However, deeply +nested groupBys (there are two or more groupBy layers beyond the first one) can potentially lead to deadlocks since the +merge buffers are limited in number and are acquired one-by-one instead of a complete set. At this time, we recommend that you avoid too many concurrent execution of deeply nested groupBys with the v2 strategy. #### Server configuration diff --git a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryLimitRowIterator.java b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryLimitRowIterator.java index caa406a14c09..78a11073f55e 100644 --- a/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/extensions-contrib/scan-query/src/main/java/io/druid/query/scan/ScanQueryLimitRowIterator.java @@ -44,7 +44,7 @@ public ScanQueryLimitRowIterator( limit = query.getLimit(); Sequence baseSequence = baseRunner.run(query, responseContext); yielder = baseSequence.toYielder( - (ScanResultValue) null, + null, new YieldingAccumulator() { @Override diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Yielders.java b/java-util/src/main/java/io/druid/java/util/common/guava/Yielders.java index e391b2186184..8e30d6fc0d43 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Yielders.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Yielders.java @@ -30,7 +30,7 @@ public class Yielders public static Yielder each(final Sequence sequence) { return sequence.toYielder( - (T) null, + null, new YieldingAccumulator() { @Override diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/ConcatSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/ConcatSequenceTest.java index b2af7bcae22b..bf2eb3761568 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/ConcatSequenceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/ConcatSequenceTest.java @@ -256,7 +256,7 @@ public void remove() ); Yielder yielder = seq.toYielder( - (Integer) null, + null, new YieldingAccumulator() { @Override diff --git a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java index 2dfcb94f45d0..fd1dbf351069 100644 --- a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java @@ -524,7 +524,7 @@ public Sequence run(Query query, Map map) try { Sequence resultSeq = baseRunner.run(query, Maps.newHashMap()); final Yielder yielder = resultSeq.toYielder( - (Object) null, + null, new YieldingAccumulator() { @Override diff --git a/services/src/main/java/io/druid/cli/DumpSegment.java b/services/src/main/java/io/druid/cli/DumpSegment.java index b9fdcdebaf6d..7f1d9f781fff 100644 --- a/services/src/main/java/io/druid/cli/DumpSegment.java +++ b/services/src/main/java/io/druid/cli/DumpSegment.java @@ -479,7 +479,7 @@ private static Sequence executeQuery(final Injector injector, final Query private static void evaluateSequenceForSideEffects(final Sequence sequence) { sequence.accumulate( - (Object) null, + null, new Accumulator() { @Override diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java index 090b677e85ec..26f8265ae54f 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java @@ -211,7 +211,7 @@ private DruidRel getLeftRelWithFilter() final Set> valuess = Sets.newHashSet(); final List filters = Lists.newArrayList(); right.runQuery().accumulate( - (Object) null, + null, new Accumulator() { @Override From 9c1ca81e398f633d9c2fa503528f47a4063ba18a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 11 Feb 2017 22:15:38 +0900 Subject: [PATCH 09/10] change to visible to testing --- .../src/main/java/io/druid/collections/BlockingPool.java | 9 ++++++++- .../druid/query/groupby/GroupByQueryMergeBufferTest.java | 5 +++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/common/src/main/java/io/druid/collections/BlockingPool.java b/common/src/main/java/io/druid/collections/BlockingPool.java index 9afc03b42732..81cedc4e1da8 100644 --- a/common/src/main/java/io/druid/collections/BlockingPool.java +++ b/common/src/main/java/io/druid/collections/BlockingPool.java @@ -19,6 +19,7 @@ package io.druid.collections; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; @@ -37,7 +38,7 @@ public class BlockingPool { private static final Logger log = new Logger(BlockingPool.class); - protected final BlockingQueue objects; + private final BlockingQueue objects; public BlockingPool( Supplier generator, @@ -78,4 +79,10 @@ public void close() throws IOException } ); } + + @VisibleForTesting + protected int getQueueSize() + { + return objects.size(); + } } 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 0e2da35bd3fa..5d04ab840fac 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -74,8 +74,9 @@ public TestBlockingPool(Supplier generator, int limit) public ReferenceCountingResourceHolder take(final long timeout) throws InterruptedException { final ReferenceCountingResourceHolder holder = super.take(timeout); - if (minRemainBufferNum > objects.size()) { - minRemainBufferNum = objects.size(); + final int queueSize = getQueueSize(); + if (minRemainBufferNum > queueSize) { + minRemainBufferNum = queueSize; } return holder; } From 54eefd3fcedccbb66ac5ae9052bd41183ccc13ed Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 14 Feb 2017 08:39:58 +0900 Subject: [PATCH 10/10] fix misspelling --- docs/content/querying/groupbyquery.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/querying/groupbyquery.md b/docs/content/querying/groupbyquery.md index 1a003c6edbd6..d455c27e3902 100644 --- a/docs/content/querying/groupbyquery.md +++ b/docs/content/querying/groupbyquery.md @@ -185,7 +185,7 @@ When using the "v2" strategy, the following runtime properties apply: Additionally, the "v2" strategy uses merging buffers for merging. It is currently the only query implementation that does so. By default, Druid is configured without any merging buffer pool, so to use the "v2" strategy you must also -set `druid.processing.numMergeBuffers` to some non-zero number. Furthermore, if you want to execute deeply nested gropuBys, +set `druid.processing.numMergeBuffers` to some non-zero number. Furthermore, if you want to execute deeply nested groupBys, you must set `druid.processing.numMergeBuffers` to at least 2. This may require allocating more direct memory. The amount of direct memory needed by Druid is at least