diff --git a/.github/workflows/reusable-standard-its.yml b/.github/workflows/reusable-standard-its.yml index b96bb4b4c495..d0adace22fbe 100644 --- a/.github/workflows/reusable-standard-its.yml +++ b/.github/workflows/reusable-standard-its.yml @@ -111,7 +111,7 @@ jobs: - name: Collect service logs on failure if: ${{ failure() && steps.run-it.conclusion == 'failure' }} run: | - tar cvzf ./service-logs.tgz ./shared/logs + tar cvzf ./service-logs.tgz ~/shared/logs - name: Upload Druid service logs to GitHub if: ${{ failure() && steps.run-it.conclusion == 'failure' }} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtilTest.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtilTest.java index a81e07cfde1f..f4b79108f407 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtilTest.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtilTest.java @@ -20,6 +20,7 @@ package org.apache.druid.benchmark; import org.apache.druid.java.util.common.parsers.Parser; +import org.apache.druid.utils.JvmUtils; import org.junit.Assert; import org.junit.Test; @@ -64,15 +65,31 @@ public void checkEvent1(Map event) Assert.assertEquals("129047958", event.get("e2.ad1[0]").toString()); Assert.assertEquals("1658972185", event.get("e2.ad1[1]").toString()); Assert.assertEquals("-997010830", event.get("e2.ad1[2]").toString()); - Assert.assertEquals("-5.8772014847368817E18", event.get("e3.m1").toString()); + + // Java 19 changes some floating point string representation + // https://bugs.openjdk.org/browse/JDK-8291475 + if (JvmUtils.majorVersion() < 19) { + Assert.assertEquals("-5.8772014847368817E18", event.get("e3.m1").toString()); + } else { + Assert.assertEquals("-5.877201484736882E18", event.get("e3.m1").toString()); + } + Assert.assertEquals("0.4375433369079904", event.get("e3.m2").toString()); Assert.assertEquals("0.8510482953607659", event.get("e3.m3").toString()); Assert.assertEquals("-2.3832626488759337E18", event.get("e3.m4").toString()); - Assert.assertEquals("7.9789762132607068E18", event.get("e3.am1[0]").toString()); - Assert.assertEquals("-7.8634787235005573E18", event.get("e3.am1[1]").toString()); - Assert.assertEquals("8.7372945568982446E18", event.get("e3.am1[2]").toString()); - Assert.assertEquals("3.1928124802414899E18", event.get("e3.am1[3]").toString()); - Assert.assertEquals("-3.9806631713718011E18", event.get("e4.e4.m4").toString()); + if (JvmUtils.majorVersion() < 19) { + Assert.assertEquals("7.9789762132607068E18", event.get("e3.am1[0]").toString()); + Assert.assertEquals("-7.8634787235005573E18", event.get("e3.am1[1]").toString()); + Assert.assertEquals("8.7372945568982446E18", event.get("e3.am1[2]").toString()); + Assert.assertEquals("3.1928124802414899E18", event.get("e3.am1[3]").toString()); + Assert.assertEquals("-3.9806631713718011E18", event.get("e4.e4.m4").toString()); + } else { + Assert.assertEquals("7.978976213260707E18", event.get("e3.am1[0]").toString()); + Assert.assertEquals("-7.863478723500557E18", event.get("e3.am1[1]").toString()); + Assert.assertEquals("8.737294556898245E18", event.get("e3.am1[2]").toString()); + Assert.assertEquals("3.19281248024149E18", event.get("e3.am1[3]").toString()); + Assert.assertEquals("-3.980663171371801E18", event.get("e4.e4.m4").toString()); + } Assert.assertEquals("-1915243040", event.get("ae1[0].d1").toString()); Assert.assertEquals("-2020543641", event.get("ae1[1].d1").toString()); Assert.assertEquals("1414285347", event.get("ae1[2].e1.d2").toString()); diff --git a/extensions-contrib/kubernetes-overlord-extensions/pom.xml b/extensions-contrib/kubernetes-overlord-extensions/pom.xml index eacd69ed6292..f231521dfa71 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/pom.xml +++ b/extensions-contrib/kubernetes-overlord-extensions/pom.xml @@ -214,7 +214,6 @@ org.mockito mockito-core - ${mockito.version} test diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java index 3b230d706bf2..36c43ef475d4 100644 --- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java +++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryQueryToolChestTest.java @@ -45,6 +45,7 @@ import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -54,7 +55,7 @@ import java.util.Map; import java.util.stream.Collectors; -public class MaterializedViewQueryQueryToolChestTest +public class MaterializedViewQueryQueryToolChestTest extends InitializedNullHandlingTest { static { NullHandling.initializeForTests(); diff --git a/extensions-core/azure-extensions/pom.xml b/extensions-core/azure-extensions/pom.xml index 2e91c4bd930f..362c9f43b6af 100644 --- a/extensions-core/azure-extensions/pom.xml +++ b/extensions-core/azure-extensions/pom.xml @@ -154,13 +154,14 @@ org.mockito - mockito-core + mockito-${mockito.inline.artifact} ${mockito.version} test + org.mockito - mockito-inline + mockito-core test diff --git a/extensions-core/druid-pac4j/pom.xml b/extensions-core/druid-pac4j/pom.xml index 65bb678596d4..a8cb8b3a08bf 100644 --- a/extensions-core/druid-pac4j/pom.xml +++ b/extensions-core/druid-pac4j/pom.xml @@ -64,6 +64,13 @@ org.pac4j pac4j-oidc ${pac4j.version} + + + + org.mockito + mockito-core + + diff --git a/extensions-core/lookups-cached-global/pom.xml b/extensions-core/lookups-cached-global/pom.xml index 092e8f87d96b..3aca23fc9ec6 100644 --- a/extensions-core/lookups-cached-global/pom.xml +++ b/extensions-core/lookups-cached-global/pom.xml @@ -140,14 +140,16 @@ easymock test + org.mockito - mockito-inline + mockito-core + ${mockito.version} test org.mockito - mockito-core + mockito-${mockito.inline.artifact} test diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 5cdd7b0ffa6e..c423b959eccf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -874,7 +874,7 @@ public void resultsComplete( try { convertedResultObject = context.jsonMapper().convertValue( resultObject, - queryKernel.getStageDefinition(stageId).getProcessorFactory().getAccumulatedResultTypeReference() + queryKernel.getStageDefinition(stageId).getProcessorFactory().getResultTypeReference() ); } catch (IllegalArgumentException e) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 95f656015ac3..d522c3a7f169 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -62,6 +62,8 @@ import org.apache.druid.frame.processor.PartitionedOutputChannel; import org.apache.druid.frame.processor.SuperSorter; import org.apache.druid.frame.processor.SuperSorterProgressTracker; +import org.apache.druid.frame.processor.manager.ProcessorManager; +import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.indexer.TaskStatus; @@ -71,8 +73,6 @@ import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.CounterNames; @@ -1100,11 +1100,12 @@ private void makeInputSliceReader() .put(ExternalInputSlice.class, new ExternalInputSliceReader(frameContext.tempDir())) .put(InlineInputSlice.class, new InlineInputSliceReader(frameContext.segmentWrangler())) .put(LookupInputSlice.class, new LookupInputSliceReader(frameContext.segmentWrangler())) - .put(SegmentsInputSlice.class, - new SegmentsInputSliceReader( - frameContext.dataSegmentProvider(), - MultiStageQueryContext.isReindex(QueryContext.of(task().getContext())) - ) + .put( + SegmentsInputSlice.class, + new SegmentsInputSliceReader( + frameContext.dataSegmentProvider(), + MultiStageQueryContext.isReindex(QueryContext.of(task().getContext())) + ) ) .build() ); @@ -1152,7 +1153,16 @@ private void makeShuffleOutputChannelFactory(boolean isFinalStage) ); } - private , I, WorkerClass extends FrameProcessor, T, R> void makeAndRunWorkProcessors() + /** + * Use {@link FrameProcessorFactory#makeProcessors} to create {@link ProcessorsAndChannels}. Executes the + * processors using {@link #exec} and sets the output channels in {@link #workResultAndOutputChannels}. + * + * @param type of {@link StageDefinition#getProcessorFactory()} + * @param return type of {@link FrameProcessor} created by the manager + * @param result type of {@link ProcessorManager#result()} + * @param type of {@link WorkOrder#getExtraInfo()} + */ + private , ProcessorReturnType, ManagerReturnType, ExtraInfoType> void makeAndRunWorkProcessors() throws IOException { if (workResultAndOutputChannels != null) { @@ -1163,13 +1173,13 @@ private , I, Wor final FactoryType processorFactory = (FactoryType) kernel.getStageDefinition().getProcessorFactory(); @SuppressWarnings("unchecked") - final ProcessorsAndChannels processors = + final ProcessorsAndChannels processors = processorFactory.makeProcessors( kernel.getStageDefinition(), kernel.getWorkOrder().getWorkerNumber(), kernel.getWorkOrder().getInputs(), inputSliceReader, - (I) kernel.getWorkOrder().getExtraInfo(), + (ExtraInfoType) kernel.getWorkOrder().getExtraInfo(), workOutputChannelFactory, frameContext, parallelism, @@ -1177,7 +1187,7 @@ private , I, Wor e -> warningPublisher.publishException(kernel.getStageDefinition().getStageNumber(), e) ); - final Sequence processorSequence = processors.processors(); + final ProcessorManager processorManager = processors.getProcessorManager(); final int maxOutstandingProcessors; @@ -1190,10 +1200,8 @@ private , I, Wor Math.max(1, Math.min(parallelism, processors.getOutputChannels().getAllChannels().size())); } - final ListenableFuture workResultFuture = exec.runAllFully( - processorSequence, - processorFactory.newAccumulatedResult(), - processorFactory::accumulateResult, + final ListenableFuture workResultFuture = exec.runAllFully( + processorManager, maxOutstandingProcessors, processorBouncer, cancellationId @@ -1716,11 +1724,13 @@ private ResultAndChannels gatherResultKeyStatistics(final OutputChannels chan final ListenableFuture clusterByStatisticsCollectorFuture = exec.runAllFully( - Sequences.simple(processors), - stageDefinition.createResultKeyStatisticsCollector( - frameContext.memoryParameters().getPartitionStatisticsMaxRetainedBytes() - ), - ClusterByStatisticsCollector::addAll, + ProcessorManagers.of(processors) + .withAccumulation( + stageDefinition.createResultKeyStatisticsCollector( + frameContext.memoryParameters().getPartitionStatisticsMaxRetainedBytes() + ), + ClusterByStatisticsCollector::addAll + ), // Run all processors simultaneously. They are lightweight and this keeps things moving. processors.size(), Bouncer.unlimited(), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java index 4bddb949f076..a09d0508485d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java @@ -32,6 +32,7 @@ import org.apache.druid.msq.input.InputSpecs; import org.apache.druid.msq.kernel.QueryDefinition; import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.querykit.BroadcastJoinSegmentMapFnProcessor; import org.apache.druid.msq.statistics.ClusterByStatisticsCollectorImpl; import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.query.lookup.LookupExtractorFactoryContainer; @@ -130,11 +131,11 @@ public class WorkerMemoryParameters private static final long SMALL_WORKER_CAPACITY_THRESHOLD_BYTES = 256_000_000; /** - * Fraction of free memory per bundle that can be used by {@link org.apache.druid.msq.querykit.BroadcastJoinHelper} - * to store broadcast data on-heap. This is used to limit the total size of input frames, which we expect to - * expand on-heap. Expansion can potentially be somewhat over 2x: for example, strings are UTF-8 in frames, but are - * UTF-16 on-heap, which is a 2x expansion, and object and index overhead must be considered on top of that. So - * we use a value somewhat lower than 0.5. + * Fraction of free memory per bundle that can be used by {@link BroadcastJoinSegmentMapFnProcessor} to store broadcast + * data on-heap. This is used to limit the total size of input frames, which we expect to expand on-heap. Expansion + * can potentially be somewhat over 2x: for example, strings are UTF-8 in frames, but are UTF-16 on-heap, which is + * a 2x expansion, and object and index overhead must be considered on top of that. So we use a value somewhat + * lower than 0.5. */ static final double BROADCAST_JOIN_MEMORY_FRACTION = 0.3; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java index 998e04295579..8af4862f2185 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessorFactory.java @@ -28,6 +28,7 @@ import com.google.common.collect.Iterables; import org.apache.druid.frame.processor.OutputChannelFactory; import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.java.util.common.Pair; @@ -75,7 +76,7 @@ @JsonTypeName("segmentGenerator") public class SegmentGeneratorFrameProcessorFactory - implements FrameProcessorFactory, SegmentGeneratorFrameProcessor, DataSegment, Set> + implements FrameProcessorFactory, List> { private final DataSchema dataSchema; private final ColumnMappings columnMappings; @@ -112,7 +113,7 @@ public MSQTuningConfig getTuningConfig() } @Override - public ProcessorsAndChannels makeProcessors( + public ProcessorsAndChannels> makeProcessors( StageDefinition stageDefinition, int workerNumber, List inputSlices, @@ -151,7 +152,8 @@ public Pair apply(ReadableInput readableInput) } )); final SegmentGenerationProgressCounter segmentGenerationProgressCounter = counters.segmentGenerationProgress(); - final SegmentGeneratorMetricsWrapper segmentGeneratorMetricsWrapper = new SegmentGeneratorMetricsWrapper(segmentGenerationProgressCounter); + final SegmentGeneratorMetricsWrapper segmentGeneratorMetricsWrapper = + new SegmentGeneratorMetricsWrapper(segmentGenerationProgressCounter); final Sequence workers = inputSequence.map( readableInputPair -> { @@ -196,32 +198,28 @@ public Pair apply(ReadableInput readableInput) } ); - return new ProcessorsAndChannels<>(workers, OutputChannels.none()); + return new ProcessorsAndChannels<>( + ProcessorManagers.of(workers) + .withAccumulation( + new HashSet<>(), + (acc, segment) -> { + if (segment != null) { + acc.add(segment); + } + + return acc; + } + ), + OutputChannels.none() + ); } @Override - public TypeReference> getAccumulatedResultTypeReference() + public TypeReference> getResultTypeReference() { return new TypeReference>() {}; } - @Override - public Set newAccumulatedResult() - { - return new HashSet<>(); - } - - @Nullable - @Override - public Set accumulateResult(Set accumulated, DataSegment current) - { - if (current != null) { - accumulated.add(current); - } - - return accumulated; - } - @Nullable @Override public Set mergeAccumulatedResult(Set accumulated, Set otherAccumulated) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameProcessorFactory.java index 4b76d8d281f1..87423664fbc5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameProcessorFactory.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.core.type.TypeReference; -import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.frame.processor.manager.ProcessorManager; import org.apache.druid.msq.counters.CounterTracker; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; @@ -36,18 +36,17 @@ * Property of {@link StageDefinition} that describes its computation logic. * * Workers call {@link #makeProcessors} to generate the processors that perform computations within that worker's - * {@link org.apache.druid.frame.processor.FrameProcessorExecutor}. Additionally, provides methods for accumulating - * the results of the processors: {@link #newAccumulatedResult()}, {@link #accumulateResult}, and - * {@link #mergeAccumulatedResult}. + * {@link org.apache.druid.frame.processor.FrameProcessorExecutor}. Additionally, provides + * {@link #mergeAccumulatedResult(Object, Object)} for merging results from {@link ProcessorManager#result()}. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -public interface FrameProcessorFactory, T, R> +public interface FrameProcessorFactory { /** * Create processors for a particular worker in a particular stage. The processors will be run on a thread pool, * with at most "maxOutstandingProcessors" number of processors outstanding at once. * - * The Sequence returned by {@link ProcessorsAndChannels#processors()} is passed directly to + * The Sequence returned by {@link ProcessorsAndChannels#getProcessorManager()} is passed directly to * {@link org.apache.druid.frame.processor.FrameProcessorExecutor#runAllFully}. * * @param stageDefinition stage definition @@ -65,7 +64,7 @@ public interface FrameProcessorFactory makeProcessors( + ProcessorsAndChannels makeProcessors( StageDefinition stageDefinition, int workerNumber, List inputSlices, @@ -78,18 +77,8 @@ ProcessorsAndChannels makeProcessors( Consumer warningPublisher ) throws IOException; - TypeReference getAccumulatedResultTypeReference(); - - /** - * Produces a "blank slate" result. - */ - R newAccumulatedResult(); - - /** - * Accumulates an additional result. May modify the left-hand side {@code accumulated}. Does not modify the - * right-hand side {@code current}. - */ - R accumulateResult(R accumulated, T current); + @Nullable + TypeReference getResultTypeReference(); /** * Merges two accumulated results. May modify the left-hand side {@code accumulated}. Does not modify the right-hand diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ProcessorsAndChannels.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ProcessorsAndChannels.java index 89d882d588a6..a90751ea14ab 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ProcessorsAndChannels.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/ProcessorsAndChannels.java @@ -19,32 +19,34 @@ package org.apache.druid.msq.kernel; -import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.OutputChannels; -import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.frame.processor.manager.ProcessorManager; /** * Returned from {@link FrameProcessorFactory#makeProcessors}. * * Includes a processor sequence and a list of output channels. + * + * @param return type of {@link org.apache.druid.frame.processor.FrameProcessor} from {@link #getProcessorManager()} + * @param result type of {@link ProcessorManager#result()} */ -public class ProcessorsAndChannels, T> +public class ProcessorsAndChannels { - private final Sequence workers; + private final ProcessorManager processorManager; private final OutputChannels outputChannels; public ProcessorsAndChannels( - final Sequence workers, + final ProcessorManager processorManager, final OutputChannels outputChannels ) { - this.workers = workers; + this.processorManager = processorManager; this.outputChannels = outputChannels; } - public Sequence processors() + public ProcessorManager getProcessorManager() { - return workers; + return processorManager; } public OutputChannels getOutputChannels() diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseFrameProcessorFactory.java index 576abd163159..33f702aca0cc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseFrameProcessorFactory.java @@ -20,7 +20,6 @@ package org.apache.druid.msq.querykit; import com.fasterxml.jackson.core.type.TypeReference; -import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.java.util.common.ISE; import org.apache.druid.msq.kernel.ExtraInfoHolder; import org.apache.druid.msq.kernel.FrameProcessorFactory; @@ -30,30 +29,17 @@ /** * Basic abstract {@link FrameProcessorFactory} that yields workers that do not require extra info and that - * always return Longs. This base class isn't used for every worker factory, but it is used for many of them. + * ignore the return values of their processors. This base class isn't used for every worker factory, but it is used + * for many of them. */ -public abstract class BaseFrameProcessorFactory - implements FrameProcessorFactory, Long, Long> +public abstract class BaseFrameProcessorFactory implements FrameProcessorFactory { @Override - public TypeReference getAccumulatedResultTypeReference() + public TypeReference getResultTypeReference() { return new TypeReference() {}; } - @Override - public Long newAccumulatedResult() - { - return 0L; - } - - @Nullable - @Override - public Long accumulateResult(Long accumulated, Long current) - { - return accumulated + current; - } - @Override public Long mergeAccumulatedResult(Long accumulated, Long otherAccumulated) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index d0a12b3ae753..3974b7e1e1d6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -19,9 +19,6 @@ package org.apache.druid.msq.querykit; -import it.unimi.dsi.fastutil.ints.Int2IntMap; -import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap; -import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.frame.channel.ReadableFrameChannel; @@ -31,127 +28,46 @@ import org.apache.druid.frame.processor.ReturnOrAwait; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.write.FrameWriterFactory; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.Unit; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.input.table.SegmentWithDescriptor; -import org.apache.druid.query.DataSource; -import org.apache.druid.query.FilteredDataSource; -import org.apache.druid.query.JoinDataSource; -import org.apache.druid.query.Query; -import org.apache.druid.query.UnnestDataSource; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; -public abstract class BaseLeafFrameProcessor implements FrameProcessor +public abstract class BaseLeafFrameProcessor implements FrameProcessor { - private final Query query; private final ReadableInput baseInput; - private final List inputChannels; private final ResourceHolder outputChannelHolder; private final ResourceHolder frameWriterFactoryHolder; - private final BroadcastJoinHelper broadcastJoinHelper; - - private Function segmentMapFn; + private final Function segmentMapFn; protected BaseLeafFrameProcessor( - final Query query, final ReadableInput baseInput, - final Int2ObjectMap sideChannels, + final Function segmentMapFn, final ResourceHolder outputChannelHolder, - final ResourceHolder frameWriterFactoryHolder, - final long memoryReservedForBroadcastJoin + final ResourceHolder frameWriterFactoryHolder ) { - this.query = query; this.baseInput = baseInput; this.outputChannelHolder = outputChannelHolder; this.frameWriterFactoryHolder = frameWriterFactoryHolder; - - final Pair, BroadcastJoinHelper> inputChannelsAndBroadcastJoinHelper = - makeInputChannelsAndBroadcastJoinHelper( - query.getDataSource(), - baseInput, - sideChannels, - memoryReservedForBroadcastJoin - ); - - this.inputChannels = inputChannelsAndBroadcastJoinHelper.lhs; - this.broadcastJoinHelper = inputChannelsAndBroadcastJoinHelper.rhs; - } - - /** - * Helper that enables implementations of {@link BaseLeafFrameProcessorFactory} to set up their primary and side channels. - */ - private static Pair, BroadcastJoinHelper> makeInputChannelsAndBroadcastJoinHelper( - final DataSource dataSource, - final ReadableInput baseInput, - final Int2ObjectMap sideChannels, - final long memoryReservedForBroadcastJoin - ) - { - // An UnnestDataSource or FilteredDataSource can have a join as a base - // In such a case a side channel is expected to be there - final DataSource baseDataSource; - if (dataSource instanceof UnnestDataSource) { - baseDataSource = ((UnnestDataSource) dataSource).getBase(); - } else if (dataSource instanceof FilteredDataSource) { - baseDataSource = ((FilteredDataSource) dataSource).getBase(); - } else { - baseDataSource = dataSource; - } - if (!(baseDataSource instanceof JoinDataSource) && !sideChannels.isEmpty()) { - throw new ISE("Did not expect side channels for dataSource [%s]", dataSource); - } - - final List inputChannels = new ArrayList<>(); - final BroadcastJoinHelper broadcastJoinHelper; - - if (baseInput.hasChannel()) { - inputChannels.add(baseInput.getChannel()); - } - - if (baseDataSource instanceof JoinDataSource) { - final Int2IntMap inputNumberToProcessorChannelMap = new Int2IntOpenHashMap(); - final List channelReaders = new ArrayList<>(); - - if (baseInput.hasChannel()) { - // BroadcastJoinHelper doesn't need to read the base channel, so stub in a null reader. - channelReaders.add(null); - } - - for (Int2ObjectMap.Entry sideChannelEntry : sideChannels.int2ObjectEntrySet()) { - final int inputNumber = sideChannelEntry.getIntKey(); - inputNumberToProcessorChannelMap.put(inputNumber, inputChannels.size()); - inputChannels.add(sideChannelEntry.getValue().getChannel()); - channelReaders.add(sideChannelEntry.getValue().getChannelFrameReader()); - } - - broadcastJoinHelper = new BroadcastJoinHelper( - inputNumberToProcessorChannelMap, - inputChannels, - channelReaders, - memoryReservedForBroadcastJoin - ); - } else { - broadcastJoinHelper = null; - } - - return Pair.of(inputChannels, broadcastJoinHelper); + this.segmentMapFn = segmentMapFn; } @Override public List inputChannels() { - return inputChannels; + if (baseInput.hasSegment()) { + return Collections.emptyList(); + } else { + return Collections.singletonList(baseInput.getChannel()); + } } @Override @@ -161,23 +77,25 @@ public List outputChannels() } @Override - public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException + public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException { - if (!initializeSegmentMapFn(readableInputs)) { - return ReturnOrAwait.awaitAll(broadcastJoinHelper.getSideChannelNumbers()); - } else if (readableInputs.size() != inputChannels.size()) { - return ReturnOrAwait.awaitAll(inputChannels.size()); - } else if (baseInput.hasSegment()) { - return runWithSegment(baseInput.getSegment()); + final ReturnOrAwait retVal; + + if (baseInput.hasSegment()) { + retVal = runWithSegment(baseInput.getSegment()); } else { - assert baseInput.hasChannel(); - return runWithInputChannel(baseInput.getChannel(), baseInput.getChannelFrameReader()); + retVal = runWithInputChannel(baseInput.getChannel(), baseInput.getChannelFrameReader()); } + + //noinspection rawtypes,unchecked + return (ReturnOrAwait) retVal; } @Override public void cleanup() throws IOException { + // Don't close the output channel, because multiple workers write to the same channel. + // The channel should be closed by the caller. FrameProcessors.closeAll(inputChannels(), Collections.emptyList(), outputChannelHolder, frameWriterFactoryHolder); } @@ -186,9 +104,9 @@ protected FrameWriterFactory getFrameWriterFactory() return frameWriterFactoryHolder.get(); } - protected abstract ReturnOrAwait runWithSegment(SegmentWithDescriptor segment) throws IOException; + protected abstract ReturnOrAwait runWithSegment(SegmentWithDescriptor segment) throws IOException; - protected abstract ReturnOrAwait runWithInputChannel( + protected abstract ReturnOrAwait runWithInputChannel( ReadableFrameChannel inputChannel, FrameReader inputFrameReader ) throws IOException; @@ -201,22 +119,4 @@ protected SegmentReference mapSegment(final Segment segment) { return segmentMapFn.apply(ReferenceCountingSegment.wrapRootGenerationSegment(segment)); } - - private boolean initializeSegmentMapFn(final IntSet readableInputs) - { - final AtomicLong cpuAccumulator = new AtomicLong(); - if (segmentMapFn != null) { - return true; - } else if (broadcastJoinHelper == null) { - segmentMapFn = query.getDataSource().createSegmentMapFunction(query, cpuAccumulator); - return true; - } else { - final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); - DataSource inlineChannelDataSource = broadcastJoinHelper.inlineChannelData(query.getDataSource()); - if (retVal) { - segmentMapFn = inlineChannelDataSource.createSegmentMapFunction(query, cpuAccumulator); - } - return retVal; - } - } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java index 7581da03b44f..041ac23c5b04 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorFactory.java @@ -19,6 +19,7 @@ package org.apache.druid.msq.querykit; +import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import it.unimi.dsi.fastutil.ints.Int2ObjectAVLTreeMap; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; @@ -30,10 +31,9 @@ import org.apache.druid.frame.processor.OutputChannel; import org.apache.druid.frame.processor.OutputChannelFactory; import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.frame.processor.manager.ProcessorManager; +import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.frame.write.FrameWriterFactory; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.CounterTracker; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; @@ -45,6 +45,8 @@ import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.kernel.ProcessorsAndChannels; import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.query.Query; +import org.apache.druid.segment.SegmentReference; import javax.annotation.Nullable; import java.io.IOException; @@ -52,15 +54,26 @@ import java.util.ArrayList; import java.util.List; import java.util.Queue; -import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; +import java.util.function.Function; +/** + * Base class of frame processors that can read regular Druid segments, external data, *or* channels from + * other stages. The term "leaf" represents the fact that they are capable of being leaves in the query tree. However, + * they do not *need* to be leaves. They can read from prior stages as well. + */ public abstract class BaseLeafFrameProcessorFactory extends BaseFrameProcessorFactory { - private static final Logger log = new Logger(BaseLeafFrameProcessorFactory.class); + private final Query query; + + protected BaseLeafFrameProcessorFactory(Query query) + { + this.query = query; + } @Override - public ProcessorsAndChannels, Long> makeProcessors( + public ProcessorsAndChannels makeProcessors( StageDefinition stageDefinition, int workerNumber, List inputSlices, @@ -85,7 +98,7 @@ public ProcessorsAndChannels, Long> makeProcessors( ); if (totalProcessors == 0) { - return new ProcessorsAndChannels<>(Sequences.empty(), OutputChannels.none()); + return new ProcessorsAndChannels<>(ProcessorManagers.none(), OutputChannels.none()); } final int outstandingProcessors; @@ -100,23 +113,20 @@ public ProcessorsAndChannels, Long> makeProcessors( outstandingProcessors = Math.min(totalProcessors, maxOutstandingProcessors); } - final AtomicReference> frameWriterFactoryQueueRef = - new AtomicReference<>(new ArrayDeque<>(outstandingProcessors)); - final AtomicReference> channelQueueRef = - new AtomicReference<>(new ArrayDeque<>(outstandingProcessors)); + final Queue frameWriterFactoryQueue = new ArrayDeque<>(outstandingProcessors); + final Queue channelQueue = new ArrayDeque<>(outstandingProcessors); final List outputChannels = new ArrayList<>(outstandingProcessors); for (int i = 0; i < outstandingProcessors; i++) { final OutputChannel outputChannel = outputChannelFactory.openChannel(0 /* Partition number doesn't matter */); outputChannels.add(outputChannel); - channelQueueRef.get().add(outputChannel.getWritableChannel()); - frameWriterFactoryQueueRef.get().add( - stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator()) + channelQueue.add(outputChannel.getWritableChannel()); + frameWriterFactoryQueue.add(stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator()) ); } // Read all base inputs in separate processors, one per processor. - final Sequence processorBaseInputs = readBaseInputs( + final Iterable processorBaseInputs = readBaseInputs( stageDefinition, inputSlices, inputSliceReader, @@ -124,65 +134,57 @@ public ProcessorsAndChannels, Long> makeProcessors( warningPublisher ); - final Sequence> processors = processorBaseInputs.map( - processorBaseInput -> { - // Read broadcast data from earlier stages. Note that for each processor, we are rebuilding the broadcast - // table from scratch, which is wasteful. This could be pushed up a level. - final Int2ObjectMap sideChannels = - readBroadcastInputsFromEarlierStages( - stageDefinition, - inputSlices, - inputSliceReader, - counters, - warningPublisher - ); + // SegmentMapFn processor, if needed. May be null. + final FrameProcessor> segmentMapFnProcessor = + makeSegmentMapFnProcessor( + stageDefinition, + inputSlices, + inputSliceReader, + frameContext, + counters, + warningPublisher + ); - return makeProcessor( - processorBaseInput, - sideChannels, - makeLazyResourceHolder( - channelQueueRef, - channel -> { - try { - channel.close(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - ), - makeLazyResourceHolder(frameWriterFactoryQueueRef, ignored -> {}), - frameContext - ); - } - ).withBaggage( - () -> { - final Queue channelQueue; - synchronized (channelQueueRef) { - // Set to null so any channels returned by outstanding workers are immediately closed. - channelQueue = channelQueueRef.getAndSet(null); - } + // Function to generate a processor manger for the regular processors, which run after the segmentMapFnProcessor. + final Function, ProcessorManager> processorManagerFn = + segmentMapFn -> + new BaseLeafFrameProcessorManager( + processorBaseInputs, + segmentMapFn, + frameWriterFactoryQueue, + channelQueue, + frameContext, + this + ); - WritableFrameChannel c; - while ((c = channelQueue.poll()) != null) { - try { - c.close(); - } - catch (Throwable e) { - log.warn(e, "Error encountered while closing channel for [%s]", this); - } - } - } - ); + //noinspection rawtypes + final ProcessorManager processorManager; + + if (segmentMapFnProcessor == null) { + final Function segmentMapFn = + query.getDataSource().createSegmentMapFunction(query, new AtomicLong()); + processorManager = processorManagerFn.apply(segmentMapFn); + } else { + processorManager = new ChainedProcessorManager<>(segmentMapFnProcessor, processorManagerFn); + } - return new ProcessorsAndChannels<>(processors, OutputChannels.wrapReadOnly(outputChannels)); + //noinspection unchecked,rawtypes + return new ProcessorsAndChannels<>(processorManager, OutputChannels.wrapReadOnly(outputChannels)); } + protected abstract FrameProcessor makeProcessor( + ReadableInput baseInput, + Function segmentMapFn, + ResourceHolder outputChannelHolder, + ResourceHolder frameWriterFactoryHolder, + FrameContext providerThingy + ); + /** * Read base inputs, where "base" is meant in the same sense as in * {@link org.apache.druid.query.planning.DataSourceAnalysis}: the primary datasource that drives query processing. */ - private static Sequence readBaseInputs( + private static Iterable readBaseInputs( final StageDefinition stageDef, final List inputSlices, final InputSliceReader inputSliceReader, @@ -190,18 +192,22 @@ private static Sequence readBaseInputs( final Consumer warningPublisher ) { - final List> sequences = new ArrayList<>(); + final List inputss = new ArrayList<>(); for (int inputNumber = 0; inputNumber < inputSlices.size(); inputNumber++) { if (!stageDef.getBroadcastInputNumbers().contains(inputNumber)) { - final int i = inputNumber; - final Sequence sequence = - Sequences.simple(inputSliceReader.attach(i, inputSlices.get(i), counters, warningPublisher)); - sequences.add(sequence); + final ReadableInputs inputs = + inputSliceReader.attach( + inputNumber, + inputSlices.get(inputNumber), + counters, + warningPublisher + ); + inputss.add(inputs); } } - return Sequences.concat(sequences); + return Iterables.concat(inputss); } /** @@ -251,52 +257,47 @@ private static Int2ObjectMap readBroadcastInputsFromEarlierStages } } - protected abstract FrameProcessor makeProcessor( - ReadableInput baseInput, - Int2ObjectMap sideChannels, - ResourceHolder outputChannel, - ResourceHolder frameWriterFactory, - FrameContext providerThingy - ); - - private static ResourceHolder makeLazyResourceHolder( - final AtomicReference> queueRef, - final Consumer backupCloser + /** + * Creates a processor that builds the segmentMapFn for all other processors. Must be run prior to all other + * processors being run. Returns null if a dedicated segmentMapFn processor is unnecessary. + */ + @Nullable + private FrameProcessor> makeSegmentMapFnProcessor( + StageDefinition stageDefinition, + List inputSlices, + InputSliceReader inputSliceReader, + FrameContext frameContext, + CounterTracker counters, + Consumer warningPublisher ) { - return new LazyResourceHolder<>( - () -> { - final T resource; + // Read broadcast data once, so it can be reused across all processors in the form of a segmentMapFn. + // No explicit cleanup: let the garbage collector handle it. + final Int2ObjectMap broadcastInputs = + readBroadcastInputsFromEarlierStages( + stageDefinition, + inputSlices, + inputSliceReader, + counters, + warningPublisher + ); - synchronized (queueRef) { - resource = queueRef.get().poll(); - } - - return new ResourceHolder() - { - @Override - public T get() - { - return resource; - } - - @Override - public void close() - { - synchronized (queueRef) { - final Queue queue = queueRef.get(); - if (queue != null) { - queue.add(resource); - return; - } - } - - // Queue was null - backupCloser.accept(resource); - } - }; - } - ); + if (broadcastInputs.isEmpty()) { + if (query.getDataSource().getAnalysis().isJoin()) { + // Joins may require significant computation to compute the segmentMapFn. Offload it to a processor. + return new SimpleSegmentMapFnProcessor(query); + } else { + // Non-joins are expected to have cheap-to-compute segmentMapFn. Do the computation in the factory thread, + // without offloading to a processor. + return null; + } + } else { + return BroadcastJoinSegmentMapFnProcessor.create( + query, + broadcastInputs, + frameContext.memoryParameters().getBroadcastJoinMemory() + ); + } } private static boolean hasParquet(final List slices) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorManager.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorManager.java new file mode 100644 index 000000000000..38ea840a5180 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessorManager.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.msq.querykit; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.manager.ProcessorAndCallback; +import org.apache.druid.frame.processor.manager.ProcessorManager; +import org.apache.druid.frame.write.FrameWriterFactory; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.segment.SegmentReference; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.Queue; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import java.util.function.Function; + +/** + * Manager for processors created by {@link BaseLeafFrameProcessorFactory}. + */ +public class BaseLeafFrameProcessorManager implements ProcessorManager +{ + private static final Logger log = new Logger(BaseLeafFrameProcessorManager.class); + + /** + * Base inputs, from {@link BaseLeafFrameProcessorFactory#readBaseInputs}. Set to null by {@link #next()} + * once exhausted. + */ + @Nullable + private Iterator baseInputIterator; + + /** + * Segment map function for this processor, from {@link BaseLeafFrameProcessorFactory#makeSegmentMapFnProcessor}. + */ + private final Function segmentMapFn; + + /** + * Frame writer factories. + * + * Sychronized by itself. Not marked with {@link com.google.errorprone.annotations.concurrent.GuardedBy} because + * errorprone has difficulty tracking synchronization through {@link #makeLazyResourceHolder}. + */ + private final AtomicReference> frameWriterFactoryQueueRef; + + /** + * Output channels. + * + * Sychronized by itself. Not marked with {@link com.google.errorprone.annotations.concurrent.GuardedBy} because + * errorprone has difficulty tracking synchronization through {@link #makeLazyResourceHolder}. + */ + private final AtomicReference> channelQueueRef; + + /** + * Frame context from our parent. + */ + private final FrameContext frameContext; + + /** + * Parent, used for {@link BaseLeafFrameProcessorFactory#makeProcessor}. + */ + private final BaseLeafFrameProcessorFactory parentFactory; + + BaseLeafFrameProcessorManager( + Iterable baseInputs, + Function segmentMapFn, + Queue frameWriterFactoryQueue, + Queue channelQueue, + FrameContext frameContext, + BaseLeafFrameProcessorFactory parentFactory + ) + { + this.baseInputIterator = baseInputs.iterator(); + this.segmentMapFn = segmentMapFn; + this.frameWriterFactoryQueueRef = new AtomicReference<>(frameWriterFactoryQueue); + this.channelQueueRef = new AtomicReference<>(channelQueue); + this.frameContext = frameContext; + this.parentFactory = parentFactory; + } + + @Override + public ListenableFuture>> next() + { + if (baseInputIterator == null) { + // Prior call would have returned empty Optional. + throw new NoSuchElementException(); + } else if (baseInputIterator.hasNext()) { + final ReadableInput baseInput = baseInputIterator.next(); + final FrameProcessor processor = parentFactory.makeProcessor( + baseInput, + segmentMapFn, + makeLazyResourceHolder( + channelQueueRef, + channel -> { + try { + channel.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ), + makeLazyResourceHolder(frameWriterFactoryQueueRef, ignored -> {}), + frameContext + ); + + return Futures.immediateFuture(Optional.of(new ProcessorAndCallback<>(processor, null))); + } else { + baseInputIterator = null; + return Futures.immediateFuture(Optional.empty()); + } + } + + @Override + public Long result() + { + // Return value isn't used for anything. Must be a Long for backwards-compatibility. + return 0L; + } + + @Override + public void close() + { + final Queue channelQueue; + synchronized (channelQueueRef) { + // Set to null so any channels returned by outstanding workers are immediately closed. + channelQueue = channelQueueRef.getAndSet(null); + } + + WritableFrameChannel c; + while ((c = channelQueue.poll()) != null) { + try { + c.close(); + } + catch (Throwable e) { + log.warn(e, "Error encountered while closing channel for [%s]", this); + } + } + } + + private static ResourceHolder makeLazyResourceHolder( + final AtomicReference> queueRef, + final Consumer backupCloser + ) + { + return new LazyResourceHolder<>( + () -> { + final T resource; + + synchronized (queueRef) { + resource = queueRef.get().poll(); + } + + return new ResourceHolder() + { + @Override + public T get() + { + return resource; + } + + @Override + public void close() + { + synchronized (queueRef) { + final Queue queue = queueRef.get(); + if (queue != null) { + queue.add(resource); + return; + } + } + + // Queue was null + backupCloser.accept(resource); + } + }; + } + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessor.java similarity index 62% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessor.java index d9e7bc6deec0..d03e56547e26 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessor.java @@ -20,27 +20,47 @@ package org.apache.druid.msq.querykit; import it.unimi.dsi.fastutil.ints.Int2IntMap; +import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntOpenHashSet; import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.druid.frame.Frame; import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.FrameProcessors; +import org.apache.druid.frame.processor.ReturnOrAwait; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault; import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.query.DataSource; import org.apache.druid.query.InlineDataSource; +import org.apache.druid.query.Query; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.SegmentReference; +import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; import java.util.stream.Collectors; -public class BroadcastJoinHelper +/** + * Processor that reads broadcast join data and creates a segment mapping function. The resulting segment + * mapping function embeds the joinable data within itself, and can be applied anywhere that would otherwise have used + * {@link org.apache.druid.query.JoinDataSource#createSegmentMapFunction(Query, AtomicLong)}. + * + * @see SimpleSegmentMapFnProcessor processor that creates a segment mapping function when there is no broadcast input + */ +public class BroadcastJoinSegmentMapFnProcessor implements FrameProcessor> { + private final Query query; private final Int2IntMap inputNumberToProcessorChannelMap; private final List channels; private final List channelReaders; @@ -51,23 +71,26 @@ public class BroadcastJoinHelper private long memoryUsed = 0L; /** - * Create a new broadcast join helper. Currently this builds the tables in channelData. Using - * {@link org.apache.druid.query.groupby.epinephelinae.collection.MemoryOpenHashTable} should be more appropriate for - * this purpose + * Create a new broadcast join data reader. Currently, this builds the tables as Object arrays + * in {@link #channelData}. Using {@link org.apache.druid.query.groupby.epinephelinae.collection.MemoryOpenHashTable} + * would likely be an improvement. * + * @param query original query * @param inputNumberToProcessorChannelMap map of input slice number -> channel position in the "channels" list * @param channels list of input channels * @param channelReaders list of input channel readers; corresponds one-to-one with "channels" * @param memoryReservedForBroadcastJoin total bytes of frames we are permitted to use; derived from * {@link WorkerMemoryParameters#getBroadcastJoinMemory()} */ - public BroadcastJoinHelper( + public BroadcastJoinSegmentMapFnProcessor( + final Query query, final Int2IntMap inputNumberToProcessorChannelMap, final List channels, final List channelReaders, final long memoryReservedForBroadcastJoin ) { + this.query = query; this.inputNumberToProcessorChannelMap = inputNumberToProcessorChannelMap; this.channels = channels; this.channelReaders = channelReaders; @@ -87,49 +110,90 @@ public BroadcastJoinHelper( } /** - * Reads up to one frame from each readable side channel, and uses them to incrementally build up joinable - * broadcast tables. - * - * @param readableInputs all readable input channel numbers, including non-side-channels - * - * @return whether side channels have been fully read + * Helper that enables implementations of {@link BaseLeafFrameProcessorFactory} to set up an instance of this class. */ - public boolean buildBroadcastTablesIncrementally(final IntSet readableInputs) + public static BroadcastJoinSegmentMapFnProcessor create( + final Query query, + final Int2ObjectMap sideChannels, + final long memoryReservedForBroadcastJoin + ) { - final IntIterator inputChannelIterator = readableInputs.iterator(); + final Int2IntMap inputNumberToProcessorChannelMap = new Int2IntOpenHashMap(); + final List inputChannels = new ArrayList<>(); + final List channelReaders = new ArrayList<>(); - while (inputChannelIterator.hasNext()) { - final int channelNumber = inputChannelIterator.nextInt(); - if (sideChannelNumbers.contains(channelNumber) && channels.get(channelNumber).canRead()) { - final Frame frame = channels.get(channelNumber).read(); + for (Int2ObjectMap.Entry sideChannelEntry : sideChannels.int2ObjectEntrySet()) { + final int inputNumber = sideChannelEntry.getIntKey(); + inputNumberToProcessorChannelMap.put(inputNumber, inputChannels.size()); + inputChannels.add(sideChannelEntry.getValue().getChannel()); + channelReaders.add(sideChannelEntry.getValue().getChannelFrameReader()); + } - memoryUsed += frame.numBytes(); + return new BroadcastJoinSegmentMapFnProcessor( + query, + inputNumberToProcessorChannelMap, + inputChannels, + channelReaders, + memoryReservedForBroadcastJoin + ); + } - if (memoryUsed > memoryReservedForBroadcastJoin) { - throw new MSQException(new BroadcastTablesTooLargeFault(memoryReservedForBroadcastJoin)); - } + @Override + public List inputChannels() + { + return channels; + } - addFrame(channelNumber, frame); - } - } + @Override + public List outputChannels() + { + return Collections.emptyList(); + } - for (int channelNumber : sideChannelNumbers) { - if (!channels.get(channelNumber).isFinished()) { - return false; - } + @Override + public ReturnOrAwait> runIncrementally(IntSet readableInputs) + { + if (buildBroadcastTablesIncrementally(readableInputs)) { + return ReturnOrAwait.returnObject(createSegmentMapFunction()); + } else { + return ReturnOrAwait.awaitAny(sideChannelNumbers); } - - return true; } - public IntSet getSideChannelNumbers() + @Override + public void cleanup() throws IOException { - return sideChannelNumbers; + FrameProcessors.closeAll(inputChannels(), outputChannels()); } + private void addFrame(final int channelNumber, final Frame frame) + { + final List data = channelData.get(channelNumber); + final FrameReader frameReader = channelReaders.get(channelNumber); + final Cursor cursor = FrameProcessors.makeCursor(frame, frameReader); + final List selectors = + frameReader.signature().getColumnNames().stream().map( + columnName -> + cursor.getColumnSelectorFactory().makeColumnValueSelector(columnName) + ).collect(Collectors.toList()); - public DataSource inlineChannelData(final DataSource originalDataSource) + while (!cursor.isDone()) { + final Object[] row = new Object[selectors.size()]; + for (int i = 0; i < row.length; i++) { + row[i] = selectors.get(i).getObject(); + } + data.add(row); + cursor.advance(); + } + } + + private Function createSegmentMapFunction() + { + return inlineChannelData(query.getDataSource()).createSegmentMapFunction(query, new AtomicLong()); + } + + DataSource inlineChannelData(final DataSource originalDataSource) { if (originalDataSource instanceof InputNumberDataSource) { final int inputNumber = ((InputNumberDataSource) originalDataSource).getInputNumber(); @@ -158,25 +222,44 @@ public DataSource inlineChannelData(final DataSource originalDataSource) } } - private void addFrame(final int channelNumber, final Frame frame) + /** + * Reads up to one frame from each readable side channel, and uses them to incrementally build up joinable + * broadcast tables. + * + * @param readableInputs all readable input channel numbers, including non-side-channels + * + * @return whether side channels have been fully read + */ + boolean buildBroadcastTablesIncrementally(final IntSet readableInputs) { - final List data = channelData.get(channelNumber); - final FrameReader frameReader = channelReaders.get(channelNumber); - final Cursor cursor = FrameProcessors.makeCursor(frame, frameReader); + final IntIterator inputChannelIterator = readableInputs.iterator(); - final List selectors = - frameReader.signature().getColumnNames().stream().map( - columnName -> - cursor.getColumnSelectorFactory().makeColumnValueSelector(columnName) - ).collect(Collectors.toList()); + while (inputChannelIterator.hasNext()) { + final int channelNumber = inputChannelIterator.nextInt(); + if (sideChannelNumbers.contains(channelNumber) && channels.get(channelNumber).canRead()) { + final Frame frame = channels.get(channelNumber).read(); - while (!cursor.isDone()) { - final Object[] row = new Object[selectors.size()]; - for (int i = 0; i < row.length; i++) { - row[i] = selectors.get(i).getObject(); + memoryUsed += frame.numBytes(); + + if (memoryUsed > memoryReservedForBroadcastJoin) { + throw new MSQException(new BroadcastTablesTooLargeFault(memoryReservedForBroadcastJoin)); + } + + addFrame(channelNumber, frame); + } + } + + for (int channelNumber : sideChannelNumbers) { + if (!channels.get(channelNumber).isFinished()) { + return false; } - data.add(row); - cursor.advance(); } + + return true; + } + + IntSet getSideChannelNumbers() + { + return sideChannelNumbers; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ChainedProcessorManager.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ChainedProcessorManager.java new file mode 100644 index 000000000000..9d4eb8e8f0e8 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/ChainedProcessorManager.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.msq.querykit; + +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.manager.ProcessorAndCallback; +import org.apache.druid.frame.processor.manager.ProcessorManager; +import org.apache.druid.utils.CloseableUtils; + +import javax.annotation.Nullable; +import java.util.Optional; +import java.util.function.Function; + +/** + * Manager that chains processors: runs {@link #first} first, then based on its result, creates {@link #restFuture} + * using {@link #restFactory} and runs that next. + */ +public class ChainedProcessorManager implements ProcessorManager +{ + /** + * First processor. This one blocks all the others. The reference is set to null once the processor has been + * returned by the channel. + */ + @Nullable + private FrameProcessor first; + + /** + * Produces {@link #restFuture}. + */ + private final Function> restFactory; + + /** + * The rest of the processors. Produced by {@link #restFactory} once {@link #first} has completed. + */ + private final SettableFuture> restFuture = SettableFuture.create(); + + /** + * Whether {@link #close()} has been called. + */ + private boolean closed; + + public ChainedProcessorManager( + final FrameProcessor first, + final Function> restFactory + ) + { + this.first = Preconditions.checkNotNull(first, "first"); + this.restFactory = Preconditions.checkNotNull(restFactory, "restFactory"); + } + + @Override + public ListenableFuture>> next() + { + if (closed) { + throw new IllegalStateException(); + } else if (first != null) { + //noinspection unchecked + final FrameProcessor tmp = (FrameProcessor) first; + first = null; + return Futures.immediateFuture(Optional.of(new ProcessorAndCallback<>(tmp, this::onFirstProcessorComplete))); + } else { + return FutureUtils.transformAsync( + restFuture, + rest -> (ListenableFuture) rest.next() + ); + } + } + + private void onFirstProcessorComplete(final Object firstResult) + { + //noinspection unchecked + restFuture.set(restFactory.apply((A) firstResult)); + } + + @Override + public R result() + { + return FutureUtils.getUncheckedImmediately(restFuture).result(); + } + + @Override + public void close() + { + if (!closed) { + closed = true; + CloseableUtils.closeAndWrapExceptions(() -> CloseableUtils.closeAll( + first != null ? first::cleanup : null, + restFuture.isDone() ? FutureUtils.getUnchecked(restFuture, false) : null + )); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index b28e8527c39b..ccd3aef7573a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -35,6 +35,15 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; +/** + * Represents an input number, i.e., a positional index into + * {@link org.apache.druid.msq.kernel.StageDefinition#getInputSpecs()}. + * + * Used by {@link DataSourcePlan} to note which inputs correspond to which datasources in the query being planned. + * + * Used by {@link BroadcastJoinSegmentMapFnProcessor} to associate broadcast inputs with the correct datasources in a + * join tree. + */ @JsonTypeName("inputNumber") public class InputNumberDataSource implements DataSource { @@ -83,14 +92,12 @@ public boolean isGlobal() @Override public boolean isConcrete() { + // InputNumberDataSource represents InputSpecs, which are scannable via Segment adapters. return true; } @Override - public Function createSegmentMapFunction( - Query query, - AtomicLong cpuTimeAcc - ) + public Function createSegmentMapFunction(Query query, AtomicLong cpuTimeAcc) { return Function.identity(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/SimpleSegmentMapFnProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/SimpleSegmentMapFnProcessor.java new file mode 100644 index 000000000000..22ee0dd7fb4a --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/SimpleSegmentMapFnProcessor.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.msq.querykit; + +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.ReturnOrAwait; +import org.apache.druid.query.Query; +import org.apache.druid.segment.SegmentReference; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; + +/** + * Processor that creates a segment mapping function that does *not* require broadcast join data. The resulting segment + * mapping function embeds the joinable data within itself, and can be applied anywhere that would otherwise have used + * {@link org.apache.druid.query.JoinDataSource#createSegmentMapFunction(Query, AtomicLong)}. + * + * @see BroadcastJoinSegmentMapFnProcessor processor that creates a segment mapping function when there is + * broadcast input + */ +public class SimpleSegmentMapFnProcessor implements FrameProcessor> +{ + private final Query query; + + public SimpleSegmentMapFnProcessor(final Query query) + { + this.query = query; + } + + @Override + public List inputChannels() + { + return Collections.emptyList(); + } + + @Override + public List outputChannels() + { + return Collections.emptyList(); + } + + @Override + public ReturnOrAwait> runIncrementally(final IntSet readableInputs) + { + return ReturnOrAwait.returnObject(query.getDataSource().createSegmentMapFunction(query, new AtomicLong())); + } + + @Override + public void cleanup() + { + // Nothing to do. + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessor.java index aa2e054ec9fe..435f6a0fc849 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessor.java @@ -32,6 +32,7 @@ import org.apache.druid.frame.write.FrameWriter; import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Unit; import org.apache.druid.segment.Cursor; import javax.annotation.Nullable; @@ -39,7 +40,7 @@ import java.util.Collections; import java.util.List; -public class OffsetLimitFrameProcessor implements FrameProcessor +public class OffsetLimitFrameProcessor implements FrameProcessor { private final ReadableFrameChannel inputChannel; private final WritableFrameChannel outputChannel; @@ -86,12 +87,12 @@ public List outputChannels() } @Override - public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException + public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException { if (readableInputs.isEmpty()) { return ReturnOrAwait.awaitAll(1); } else if (inputChannel.isFinished() || rowsProcessedSoFar == offset + limit) { - return ReturnOrAwait.returnObject(rowsProcessedSoFar); + return ReturnOrAwait.returnObject(Unit.instance()); } final Frame frame = inputChannel.read(); @@ -103,7 +104,7 @@ public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws if (rowsProcessedSoFar == offset + limit) { // This check is not strictly necessary, given the check above, but prevents one extra scheduling round. - return ReturnOrAwait.returnObject(rowsProcessedSoFar); + return ReturnOrAwait.returnObject(Unit.instance()); } else { assert rowsProcessedSoFar < offset + limit; return ReturnOrAwait.awaitAll(1); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessorFactory.java index 134c9ddd4e50..a1241242fb9d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/OffsetLimitFrameProcessorFactory.java @@ -31,9 +31,8 @@ import org.apache.druid.frame.processor.OutputChannel; import org.apache.druid.frame.processor.OutputChannelFactory; import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.msq.counters.CounterTracker; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; @@ -43,7 +42,6 @@ import org.apache.druid.msq.kernel.ProcessorsAndChannels; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.querykit.BaseFrameProcessorFactory; -import org.apache.druid.msq.util.SupplierIterator; import javax.annotation.Nullable; import java.io.IOException; @@ -87,7 +85,7 @@ public Long getLimit() } @Override - public ProcessorsAndChannels, Long> makeProcessors( + public ProcessorsAndChannels makeProcessors( StageDefinition stageDefinition, int workerNumber, List inputSlices, @@ -110,12 +108,12 @@ public ProcessorsAndChannels, Long> makeProcessors( final InputSlice slice = Iterables.getOnlyElement(inputSlices); if (inputSliceReader.numReadableInputs(slice) == 0) { - return new ProcessorsAndChannels<>(Sequences.empty(), OutputChannels.none()); + return new ProcessorsAndChannels<>(ProcessorManagers.none(), OutputChannels.none()); } final OutputChannel outputChannel = outputChannelFactory.openChannel(0); - final Supplier> workerSupplier = () -> { + final Supplier> workerSupplier = () -> { final ReadableInputs readableInputs = inputSliceReader.attach(0, slice, counters, warningPublisher); if (!readableInputs.isChannelBased()) { @@ -135,11 +133,8 @@ public ProcessorsAndChannels, Long> makeProcessors( ); }; - final Sequence> processors = - Sequences.simple(() -> new SupplierIterator<>(workerSupplier)); - return new ProcessorsAndChannels<>( - processors, + ProcessorManagers.of(workerSupplier), OutputChannels.wrapReadOnly(Collections.singletonList(outputChannel)) ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java index fdc80560f29d..239dc9803095 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java @@ -41,6 +41,7 @@ import org.apache.druid.frame.write.FrameWriter; import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Unit; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.TooManyRowsWithSameKeyFault; import org.apache.druid.msq.input.ReadableInput; @@ -101,7 +102,7 @@ * 5) Once we process the final row on the *other* side, reset both marks with {@link Tracker#markCurrent()} and * continue the algorithm. */ -public class SortMergeJoinFrameProcessor implements FrameProcessor +public class SortMergeJoinFrameProcessor implements FrameProcessor { private static final int LEFT = 0; private static final int RIGHT = 1; @@ -166,7 +167,7 @@ public List outputChannels() } @Override - public ReturnOrAwait runIncrementally(IntSet readableInputs) throws IOException + public ReturnOrAwait runIncrementally(IntSet readableInputs) throws IOException { // Fetch enough frames such that each tracker has one readable row (or is done). for (int i = 0; i < inputChannels.size(); i++) { @@ -218,7 +219,7 @@ public ReturnOrAwait runIncrementally(IntSet readableInputs) throws IOExce if (allTrackersAreAtEnd()) { flushCurrentFrame(); - return ReturnOrAwait.returnObject(0L); + return ReturnOrAwait.returnObject(Unit.instance()); } else { // Keep reading. return nextAwait(); @@ -381,7 +382,7 @@ private void advanceTrackersAfterEmittingRow(final int markCmp, final boolean ma * * If all channels have hit their limit, throws {@link MSQException} with {@link TooManyRowsWithSameKeyFault}. */ - private ReturnOrAwait nextAwait() + private ReturnOrAwait nextAwait() { final IntSet awaitSet = new IntOpenHashSet(); int trackerAtLimit = -1; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorFactory.java index 76e05d3ce0cf..ef4d9f280a9c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorFactory.java @@ -34,10 +34,10 @@ import org.apache.druid.frame.processor.OutputChannel; import org.apache.druid.frame.processor.OutputChannelFactory; import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.msq.counters.CounterTracker; import org.apache.druid.msq.input.InputSlice; @@ -122,7 +122,7 @@ public JoinType getJoinType() } @Override - public ProcessorsAndChannels, Long> makeProcessors( + public ProcessorsAndChannels makeProcessors( StageDefinition stageDefinition, int workerNumber, List inputSlices, @@ -156,7 +156,7 @@ public ProcessorsAndChannels, Long> makeProcessors( ); if (inputsByPartition.isEmpty()) { - return new ProcessorsAndChannels<>(Sequences.empty(), OutputChannels.none()); + return new ProcessorsAndChannels<>(ProcessorManagers.none(), OutputChannels.none()); } // Create output channels. @@ -166,7 +166,7 @@ public ProcessorsAndChannels, Long> makeProcessors( } // Create processors. - final Iterable> processors = Iterables.transform( + final Iterable> processors = Iterables.transform( inputsByPartition.int2ObjectEntrySet(), entry -> { final int partitionNumber = entry.getIntKey(); @@ -187,7 +187,7 @@ public ProcessorsAndChannels, Long> makeProcessors( ); return new ProcessorsAndChannels<>( - Sequences.simple(processors), + ProcessorManagers.of(processors), OutputChannels.wrap(ImmutableList.copyOf(outputChannels.values())) ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java index 6356d00b0563..ed14dd52739c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessor.java @@ -32,6 +32,7 @@ import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.write.FrameWriter; import org.apache.druid.frame.write.FrameWriterFactory; +import org.apache.druid.java.util.common.Unit; import org.apache.druid.msq.querykit.QueryKitUtils; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; @@ -59,7 +60,7 @@ import java.util.function.Consumer; import java.util.function.Supplier; -public class GroupByPostShuffleFrameProcessor implements FrameProcessor +public class GroupByPostShuffleFrameProcessor implements FrameProcessor { private final GroupByQuery query; private final ReadableFrameChannel inputChannel; @@ -121,7 +122,7 @@ public List outputChannels() } @Override - public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException + public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException { if (frameCursor == null || frameCursor.isDone()) { // Keep reading through the input channel. @@ -133,7 +134,7 @@ public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws } writeCurrentFrameIfNeeded(); - return ReturnOrAwait.returnObject(0L); + return ReturnOrAwait.returnObject(Unit.instance()); } else { final Frame frame = inputChannel.read(); frameCursor = FrameProcessors.makeCursor(frame, frameReader); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java index ab7f4fc26ba8..c74b7ea246fd 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPostShuffleFrameProcessorFactory.java @@ -30,6 +30,7 @@ import org.apache.druid.frame.processor.OutputChannel; import org.apache.druid.frame.processor.OutputChannelFactory; import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.msq.counters.CounterTracker; @@ -70,7 +71,7 @@ public GroupByQuery getQuery() } @Override - public ProcessorsAndChannels, Long> makeProcessors( + public ProcessorsAndChannels makeProcessors( StageDefinition stageDefinition, int workerNumber, List inputSlices, @@ -105,7 +106,7 @@ public ProcessorsAndChannels, Long> makeProcessors( final Sequence readableInputs = Sequences.simple(inputSliceReader.attach(0, slice, counters, warningPublisher)); - final Sequence> processors = readableInputs.map( + final Sequence> processors = readableInputs.map( readableInput -> { final OutputChannel outputChannel = outputChannels.get(readableInput.getStagePartition().getPartitionNumber()); @@ -123,7 +124,7 @@ public ProcessorsAndChannels, Long> makeProcessors( ); return new ProcessorsAndChannels<>( - processors, + ProcessorManagers.of(processors), OutputChannels.wrapReadOnly(ImmutableList.copyOf(outputChannels.values())) ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java index 67e53cd41e7d..63f5ad6650ae 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java @@ -20,7 +20,6 @@ package org.apache.druid.msq.querykit.groupby; import com.google.common.collect.Iterables; -import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.frame.Frame; import org.apache.druid.frame.channel.FrameWithPartition; @@ -34,6 +33,7 @@ import org.apache.druid.frame.write.FrameWriter; import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Unit; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; @@ -49,10 +49,12 @@ import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.timeline.SegmentId; import java.io.IOException; +import java.util.function.Function; /** * A {@link FrameProcessor} that reads one {@link Frame} at a time from a particular segment, writes them @@ -67,26 +69,22 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor private Yielder resultYielder; private FrameWriter frameWriter; - private long rowsOutput; private long currentAllocatorCapacity; // Used for generating FrameRowTooLargeException if needed public GroupByPreShuffleFrameProcessor( final GroupByQuery query, - final ReadableInput baseInput, - final Int2ObjectMap sideChannels, final GroupingEngine groupingEngine, + final ReadableInput baseInput, + final Function segmentMapFn, final ResourceHolder outputChannelHolder, - final ResourceHolder frameWriterFactoryHolder, - final long memoryReservedForBroadcastJoin + final ResourceHolder frameWriterFactoryHolder ) { super( - query, baseInput, - sideChannels, + segmentMapFn, outputChannelHolder, - frameWriterFactoryHolder, - memoryReservedForBroadcastJoin + frameWriterFactoryHolder ); this.query = query; this.groupingEngine = groupingEngine; @@ -98,7 +96,7 @@ public GroupByPreShuffleFrameProcessor( } @Override - protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException + protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException { if (resultYielder == null) { final ResourceHolder segmentHolder = closer.register(segment.getOrLoad()); @@ -116,14 +114,14 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment populateFrameWriterAndFlushIfNeeded(); if (resultYielder == null || resultYielder.isDone()) { - return ReturnOrAwait.returnObject(rowsOutput); + return ReturnOrAwait.returnObject(Unit.instance()); } else { return ReturnOrAwait.runAgain(); } } @Override - protected ReturnOrAwait runWithInputChannel( + protected ReturnOrAwait runWithInputChannel( final ReadableFrameChannel inputChannel, final FrameReader inputFrameReader ) throws IOException @@ -145,7 +143,7 @@ protected ReturnOrAwait runWithInputChannel( resultYielder = Yielders.each(rowSequence); } else if (inputChannel.isFinished()) { flushFrameWriterIfNeeded(); - return ReturnOrAwait.returnObject(rowsOutput); + return ReturnOrAwait.returnObject(Unit.instance()); } else { return ReturnOrAwait.awaitAll(inputChannels().size()); } @@ -208,7 +206,6 @@ private void flushFrameWriterIfNeeded() throws IOException Iterables.getOnlyElement(outputChannels()).write(new FrameWithPartition(frame, FrameWithPartition.NO_PARTITION)); frameWriter.close(); frameWriter = null; - rowsOutput += frame.numRows(); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessorFactory.java index 4c0601d6c90c..71f1e531b40e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessorFactory.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; -import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.frame.channel.WritableFrameChannel; import org.apache.druid.frame.processor.FrameProcessor; @@ -32,6 +31,9 @@ import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.querykit.BaseLeafFrameProcessorFactory; import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.segment.SegmentReference; + +import java.util.function.Function; @JsonTypeName("groupByPreShuffle") public class GroupByPreShuffleFrameProcessorFactory extends BaseLeafFrameProcessorFactory @@ -41,6 +43,7 @@ public class GroupByPreShuffleFrameProcessorFactory extends BaseLeafFrameProcess @JsonCreator public GroupByPreShuffleFrameProcessorFactory(@JsonProperty("query") GroupByQuery query) { + super(query); this.query = Preconditions.checkNotNull(query, "query"); } @@ -51,9 +54,9 @@ public GroupByQuery getQuery() } @Override - protected FrameProcessor makeProcessor( + protected FrameProcessor makeProcessor( final ReadableInput baseInput, - final Int2ObjectMap sideChannels, + final Function segmentMapFn, final ResourceHolder outputChannelHolder, final ResourceHolder frameWriterFactoryHolder, final FrameContext frameContext @@ -61,12 +64,11 @@ protected FrameProcessor makeProcessor( { return new GroupByPreShuffleFrameProcessor( query, - baseInput, - sideChannels, frameContext.groupingEngine(), + baseInput, + segmentMapFn, outputChannelHolder, - frameWriterFactoryHolder, - frameContext.memoryParameters().getBroadcastJoinMemory() + frameWriterFactoryHolder ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java index 30c20fe22dad..535819aff527 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultFrameProcessorFactory.java @@ -29,6 +29,7 @@ import org.apache.druid.frame.processor.OutputChannel; import org.apache.druid.frame.processor.OutputChannelFactory; import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.msq.counters.CounterTracker; @@ -57,7 +58,7 @@ public QueryResultFrameProcessorFactory() } @Override - public ProcessorsAndChannels, Long> makeProcessors( + public ProcessorsAndChannels makeProcessors( StageDefinition stageDefinition, int workerNumber, List inputSlices, @@ -74,7 +75,7 @@ public ProcessorsAndChannels, Long> makeProcessors( final StageInputSlice slice = (StageInputSlice) Iterables.getOnlyElement(inputSlices); if (inputSliceReader.numReadableInputs(slice) == 0) { - return new ProcessorsAndChannels<>(Sequences.empty(), OutputChannels.none()); + return new ProcessorsAndChannels<>(ProcessorManagers.none(), OutputChannels.none()); } final Int2ObjectSortedMap outputChannels = new Int2ObjectAVLTreeMap<>(); @@ -96,7 +97,7 @@ public ProcessorsAndChannels, Long> makeProcessors( final Sequence readableInputs = Sequences.simple(inputSliceReader.attach(0, slice, counters, warningPublisher)); - final Sequence> processors = readableInputs.map( + final Sequence> processors = readableInputs.map( readableInput -> { final OutputChannel outputChannel = outputChannels.get(readableInput.getStagePartition().getPartitionNumber()); @@ -109,9 +110,8 @@ public ProcessorsAndChannels, Long> makeProcessors( ); return new ProcessorsAndChannels<>( - processors, + ProcessorManagers.of(processors), OutputChannels.wrapReadOnly(ImmutableList.copyOf(outputChannels.values())) ); - } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessor.java index c945fd33e7c6..7871af4bcfbd 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessor.java @@ -26,14 +26,14 @@ import org.apache.druid.frame.processor.FrameProcessor; import org.apache.druid.frame.processor.FrameProcessors; import org.apache.druid.frame.processor.ReturnOrAwait; +import org.apache.druid.java.util.common.Unit; import java.io.IOException; import java.util.Collections; import java.util.List; -public class QueryResultsFrameProcessor implements FrameProcessor +public class QueryResultsFrameProcessor implements FrameProcessor { - long numRows = 0L; private final ReadableFrameChannel inChannel; private final WritableFrameChannel outChannel; @@ -59,13 +59,13 @@ public List outputChannels() } @Override - public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException + public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException { if (readableInputs.isEmpty()) { return ReturnOrAwait.awaitAll(1); } if (inChannel.isFinished()) { - return ReturnOrAwait.returnObject(numRows); + return ReturnOrAwait.returnObject(Unit.instance()); } writeFrame(inChannel.read()); return ReturnOrAwait.awaitAll(1); @@ -80,6 +80,5 @@ public void cleanup() throws IOException private void writeFrame(final Frame frame) throws IOException { outChannel.write(frame); - numRows += frame.numRows(); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java index 543dd01f5b6b..99ea8037b7bb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessor.java @@ -23,7 +23,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; -import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.frame.Frame; @@ -41,6 +40,7 @@ import org.apache.druid.frame.write.InvalidNullByteException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Unit; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; @@ -59,6 +59,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleSettableOffset; import org.apache.druid.segment.StorageAdapter; @@ -73,6 +74,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; /** * A {@link FrameProcessor} that reads one {@link Frame} at a time from a particular segment, writes them @@ -86,7 +88,6 @@ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor private final VirtualColumns frameWriterVirtualColumns; private final Closer closer = Closer.create(); - private long rowsOutput = 0; private Cursor cursor; private Segment segment; private final SimpleSettableOffset cursorOffset = new SimpleAscendingOffset(Integer.MAX_VALUE); @@ -95,22 +96,19 @@ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor public ScanQueryFrameProcessor( final ScanQuery query, + @Nullable final AtomicLong runningCountForLimit, + final ObjectMapper jsonMapper, final ReadableInput baseInput, - final Int2ObjectMap sideChannels, + final Function segmentMapFn, final ResourceHolder outputChannelHolder, - final ResourceHolder frameWriterFactoryHolder, - @Nullable final AtomicLong runningCountForLimit, - final long memoryReservedForBroadcastJoin, - final ObjectMapper jsonMapper + final ResourceHolder frameWriterFactoryHolder ) { super( - query, baseInput, - sideChannels, + segmentMapFn, outputChannelHolder, - frameWriterFactoryHolder, - memoryReservedForBroadcastJoin + frameWriterFactoryHolder ); this.query = query; this.runningCountForLimit = runningCountForLimit; @@ -130,7 +128,7 @@ public ScanQueryFrameProcessor( } @Override - public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException + public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws IOException { final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "Expected non-null 'legacy' parameter"); @@ -140,7 +138,7 @@ public ReturnOrAwait runIncrementally(final IntSet readableInputs) throws if (runningCountForLimit != null && runningCountForLimit.get() > query.getScanRowsOffset() + query.getScanRowsLimit()) { - return ReturnOrAwait.returnObject(rowsOutput); + return ReturnOrAwait.returnObject(Unit.instance()); } return super.runIncrementally(readableInputs); @@ -155,7 +153,7 @@ public void cleanup() throws IOException } @Override - protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException + protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException { if (cursor == null) { final ResourceHolder segmentHolder = closer.register(segment.getOrLoad()); @@ -170,7 +168,7 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment if (cursorYielder.isDone()) { // No cursors! cursorYielder.close(); - return ReturnOrAwait.returnObject(rowsOutput); + return ReturnOrAwait.returnObject(Unit.instance()); } else { final long rowsFlushed = setNextCursor(cursorYielder.get(), segmentHolder.get()); assert rowsFlushed == 0; // There's only ever one cursor when running with a segment @@ -185,14 +183,14 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment } if (cursor.isDone() && (frameWriter == null || frameWriter.getNumRows() == 0)) { - return ReturnOrAwait.returnObject(rowsOutput); + return ReturnOrAwait.returnObject(Unit.instance()); } else { return ReturnOrAwait.runAgain(); } } @Override - protected ReturnOrAwait runWithInputChannel( + protected ReturnOrAwait runWithInputChannel( final ReadableFrameChannel inputChannel, final FrameReader inputFrameReader ) throws IOException @@ -217,7 +215,7 @@ protected ReturnOrAwait runWithInputChannel( } } else if (inputChannel.isFinished()) { flushFrameWriter(); - return ReturnOrAwait.returnObject(rowsOutput); + return ReturnOrAwait.returnObject(Unit.instance()); } else { return ReturnOrAwait.awaitAll(inputChannels().size()); } @@ -296,7 +294,6 @@ private long flushFrameWriter() throws IOException Iterables.getOnlyElement(outputChannels()).write(new FrameWithPartition(frame, FrameWithPartition.NO_PARTITION)); frameWriter.close(); frameWriter = null; - rowsOutput += frame.numRows(); return frame.numRows(); } else { if (frameWriter != null) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorFactory.java index 391ad5ba90f3..e3d3619dd955 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorFactory.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; -import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.frame.channel.WritableFrameChannel; import org.apache.druid.frame.processor.FrameProcessor; @@ -32,9 +31,11 @@ import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.querykit.BaseLeafFrameProcessorFactory; import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.segment.SegmentReference; import javax.annotation.Nullable; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; @JsonTypeName("scan") public class ScanQueryFrameProcessorFactory extends BaseLeafFrameProcessorFactory @@ -51,10 +52,9 @@ public class ScanQueryFrameProcessorFactory extends BaseLeafFrameProcessorFactor private final AtomicLong runningCountForLimit; @JsonCreator - public ScanQueryFrameProcessorFactory( - @JsonProperty("query") ScanQuery query - ) + public ScanQueryFrameProcessorFactory(@JsonProperty("query") ScanQuery query) { + super(query); this.query = Preconditions.checkNotNull(query, "query"); this.runningCountForLimit = query.isLimited() && query.getOrderBys().isEmpty() ? new AtomicLong() : null; @@ -67,9 +67,9 @@ public ScanQuery getQuery() } @Override - protected FrameProcessor makeProcessor( + protected FrameProcessor makeProcessor( ReadableInput baseInput, - Int2ObjectMap sideChannels, + Function segmentMapFn, ResourceHolder outputChannelHolder, ResourceHolder frameWriterFactoryHolder, FrameContext frameContext @@ -77,13 +77,12 @@ protected FrameProcessor makeProcessor( { return new ScanQueryFrameProcessor( query, + runningCountForLimit, + frameContext.jsonMapper(), baseInput, - sideChannels, + segmentMapFn, outputChannelHolder, - frameWriterFactoryHolder, - runningCountForLimit, - frameContext.memoryParameters().getBroadcastJoinMemory(), - frameContext.jsonMapper() + frameWriterFactoryHolder ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java index c7b20ac46e08..2efc94740ac7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java @@ -75,6 +75,10 @@ public static DimensionSchema createDimensionSchema( switch (type.getElementType().getType()) { case STRING: return new StringDimensionSchema(column, DimensionSchema.MultiValueHandling.ARRAY, null); + case LONG: + case FLOAT: + case DOUBLE: + return new AutoTypeColumnSchema(column); default: throw new ISE("Cannot create dimension for type [%s]", type.toString()); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java index b3bf3c408a29..42bb1506a307 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java @@ -31,7 +31,6 @@ import org.apache.druid.msq.indexing.error.TooManyColumnsFault; import org.apache.druid.msq.indexing.error.TooManyInputFilesFault; import org.apache.druid.msq.indexing.error.TooManyPartitionsFault; -import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.test.MSQTestFileUtils; import org.apache.druid.segment.column.ColumnType; @@ -220,31 +219,6 @@ public void testInsertWithTooManySegments() throws IOException } - @Test - public void testInsertWithUnsupportedColumnType() - { - RowSignature dummyRowSignature = RowSignature.builder().add("__time", ColumnType.LONG).build(); - - testIngestQuery() - .setSql(StringUtils.format( - " insert into foo1 SELECT\n" - + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" - + " col1\n" - + "FROM TABLE(\n" - + " EXTERN(\n" - + " '{ \"files\": [\"ignored\"],\"type\":\"local\"}',\n" - + " '{\"type\": \"json\"}',\n" - + " '[{\"name\": \"timestamp\", \"type\": \"string\"},{\"name\": \"col1\", \"type\": \"long_array\"} ]'\n" - + " )\n" - + ") PARTITIONED by day" - )) - .setExpectedDataSource("foo1") - .setExpectedRowSignature(dummyRowSignature) - .setExpectedMSQFault(UnknownFault.forMessage( - "org.apache.druid.java.util.common.ISE: Cannot create dimension for type [ARRAY]")) - .verifyResults(); - } - @Test public void testInsertWithManyColumns() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java index 81ba53f755ef..e54027c2449b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java @@ -63,6 +63,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -1463,6 +1464,193 @@ public void testInsertArraysAutoType() throws IOException .verifyResults(); } + @Test + public void testInsertArrays() throws IOException + { + List expectedRows = Arrays.asList( + new Object[]{ + 1672531200000L, + null, + null, + new Object[]{1L, 2L, 3L}, + new Object[]{}, + new Object[]{1.1d, 2.2d, 3.3d}, + null + }, + new Object[]{ + 1672531200000L, + null, + Arrays.asList("a", "b"), + null, + new Object[]{2L, 3L}, + null, + new Object[]{null} + }, + new Object[]{ + 1672531200000L, + Arrays.asList("a", "b"), + null, + null, + new Object[]{null, 2L, 9L}, + null, + new Object[]{999.0d, 5.5d, null} + }, + new Object[]{ + 1672531200000L, + Arrays.asList("a", "b"), + Arrays.asList("a", "b"), + new Object[]{1L, 2L, 3L}, + new Object[]{1L, null, 3L}, + new Object[]{1.1d, 2.2d, 3.3d}, + new Object[]{1.1d, 2.2d, null} + }, + new Object[]{ + 1672531200000L, + Arrays.asList("a", "b", "c"), + Arrays.asList(null, "b"), + new Object[]{2L, 3L}, + null, + new Object[]{3.3d, 4.4d, 5.5d}, + new Object[]{999.0d, null, 5.5d} + }, + new Object[]{ + 1672531200000L, + Arrays.asList("b", "c"), + Arrays.asList("d", null, "b"), + new Object[]{1L, 2L, 3L, 4L}, + new Object[]{1L, 2L, 3L}, + new Object[]{1.1d, 3.3d}, + new Object[]{null, 2.2d, null} + }, + new Object[]{ + 1672531200000L, + Arrays.asList("d", "e"), + Arrays.asList("b", "b"), + new Object[]{1L, 4L}, + new Object[]{1L}, + new Object[]{2.2d, 3.3d, 4.0d}, + null + }, + new Object[]{ + 1672617600000L, + null, + null, + new Object[]{1L, 2L, 3L}, + null, + new Object[]{1.1d, 2.2d, 3.3d}, + new Object[]{} + }, + new Object[]{ + 1672617600000L, + null, + Arrays.asList("a", "b"), + null, + new Object[]{2L, 3L}, + null, + new Object[]{null, 1.1d} + }, + new Object[]{ + 1672617600000L, + Arrays.asList("a", "b"), + null, + null, + new Object[]{null, 2L, 9L}, + null, + new Object[]{999.0d, 5.5d, null} + }, + new Object[]{ + 1672617600000L, + Arrays.asList("a", "b"), + Collections.emptyList(), + new Object[]{1L, 2L, 3L}, + new Object[]{1L, null, 3L}, + new Object[]{1.1d, 2.2d, 3.3d}, + new Object[]{1.1d, 2.2d, null} + }, + new Object[]{ + 1672617600000L, + Arrays.asList("a", "b", "c"), + Arrays.asList(null, "b"), + new Object[]{2L, 3L}, + null, + new Object[]{3.3d, 4.4d, 5.5d}, + new Object[]{999.0d, null, 5.5d} + }, + new Object[]{ + 1672617600000L, + Arrays.asList("b", "c"), + Arrays.asList("d", null, "b"), + new Object[]{1L, 2L, 3L, 4L}, + new Object[]{1L, 2L, 3L}, + new Object[]{1.1d, 3.3d}, + new Object[]{null, 2.2d, null} + }, + new Object[]{ + 1672617600000L, + Arrays.asList("d", "e"), + Arrays.asList("b", "b"), + new Object[]{1L, 4L}, + new Object[]{null}, + new Object[]{2.2d, 3.3d, 4.0}, + null + } + ); + + RowSignature rowSignatureWithoutTimeAndStringColumns = + RowSignature.builder() + .add("arrayLong", ColumnType.LONG_ARRAY) + .add("arrayLongNulls", ColumnType.LONG_ARRAY) + .add("arrayDouble", ColumnType.DOUBLE_ARRAY) + .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY) + .build(); + + + RowSignature fileSignature = RowSignature.builder() + .add("timestamp", ColumnType.STRING) + .add("arrayString", ColumnType.STRING_ARRAY) + .add("arrayStringNulls", ColumnType.STRING_ARRAY) + .addAll(rowSignatureWithoutTimeAndStringColumns) + .build(); + + // MSQ writes strings instead of string arrays + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("arrayString", ColumnType.STRING) + .add("arrayStringNulls", ColumnType.STRING) + .addAll(rowSignatureWithoutTimeAndStringColumns) + .build(); + + final Map adjustedContext = new HashMap<>(context); + final File tmpFile = temporaryFolder.newFile(); + final InputStream resourceStream = NestedDataTestUtils.class.getClassLoader().getResourceAsStream(NestedDataTestUtils.ARRAY_TYPES_DATA_FILE); + final InputStream decompressing = CompressionUtils.decompress(resourceStream, NestedDataTestUtils.ARRAY_TYPES_DATA_FILE); + Files.copy(decompressing, tmpFile.toPath(), StandardCopyOption.REPLACE_EXISTING); + decompressing.close(); + + final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(tmpFile); + + testIngestQuery().setSql(" INSERT INTO foo1 SELECT\n" + + " TIME_PARSE(\"timestamp\") as __time,\n" + + " arrayString,\n" + + " arrayStringNulls,\n" + + " arrayLong,\n" + + " arrayLongNulls,\n" + + " arrayDouble,\n" + + " arrayDoubleNulls\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '" + queryFramework().queryJsonMapper().writeValueAsString(fileSignature) + "'\n" + + " )\n" + + ") PARTITIONED BY day") + .setQueryContext(adjustedContext) + .setExpectedResultRows(expectedRows) + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .verifyResults(); + } + @Nonnull private List expectedFooRows() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessorTest.java similarity index 90% rename from extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java rename to extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessorTest.java index cdc3a369a4d1..8d9437262148 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessorTest.java @@ -47,8 +47,6 @@ import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.JoinType; -import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.server.QueryStackTests; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -65,12 +63,11 @@ import java.util.List; import java.util.stream.Collectors; -public class BroadcastJoinHelperTest extends InitializedNullHandlingTest +public class BroadcastJoinSegmentMapFnProcessorTest extends InitializedNullHandlingTest { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private JoinableFactory joinableFactory; private StorageAdapter adapter; private File testDataFile1; private File testDataFile2; @@ -81,9 +78,6 @@ public class BroadcastJoinHelperTest extends InitializedNullHandlingTest public void setUp() throws IOException { final ArenaMemoryAllocator allocator = ArenaMemoryAllocator.createOnHeap(10_000); - - joinableFactory = QueryStackTests.makeJoinableFactoryFromDefault(null, null, null); - adapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); // File 1: the entire test dataset. @@ -127,14 +121,15 @@ public void testBuildTableAndInlineData() throws IOException channelReaders.add(frameReader1); channelReaders.add(frameReader2); - final BroadcastJoinHelper broadcastJoinHelper = new BroadcastJoinHelper( + final BroadcastJoinSegmentMapFnProcessor broadcastJoinReader = new BroadcastJoinSegmentMapFnProcessor( + null /* Query; not used for the methods we're testing */, sideStageChannelNumberMap, channels, channelReaders, 25_000_000L // High enough memory limit that we won't hit it ); - Assert.assertEquals(ImmutableSet.of(1, 2), broadcastJoinHelper.getSideChannelNumbers()); + Assert.assertEquals(ImmutableSet.of(1, 2), broadcastJoinReader.getSideChannelNumbers()); boolean doneReading = false; while (!doneReading) { @@ -142,7 +137,7 @@ public void testBuildTableAndInlineData() throws IOException for (int i = 1; i < channels.size(); i++) { readableInputs.add(i); // Frame file channels are always ready, so this is OK. } - doneReading = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); + doneReading = broadcastJoinReader.buildBroadcastTablesIncrementally(readableInputs); } Assert.assertTrue(channels.get(1).isFinished()); @@ -150,21 +145,21 @@ public void testBuildTableAndInlineData() throws IOException Assert.assertEquals( new InputNumberDataSource(0), - broadcastJoinHelper.inlineChannelData(new InputNumberDataSource(0)) + broadcastJoinReader.inlineChannelData(new InputNumberDataSource(0)) ); Assert.assertEquals( new InputNumberDataSource(1), - broadcastJoinHelper.inlineChannelData(new InputNumberDataSource(1)) + broadcastJoinReader.inlineChannelData(new InputNumberDataSource(1)) ); Assert.assertEquals( new InputNumberDataSource(2), - broadcastJoinHelper.inlineChannelData(new InputNumberDataSource(2)) + broadcastJoinReader.inlineChannelData(new InputNumberDataSource(2)) ); final List rowsFromStage3 = - ((InlineDataSource) broadcastJoinHelper.inlineChannelData(new InputNumberDataSource(3))).getRowsAsList(); + ((InlineDataSource) broadcastJoinReader.inlineChannelData(new InputNumberDataSource(3))).getRowsAsList(); Assert.assertEquals(1209, rowsFromStage3.size()); FrameTestUtil.assertRowsEqual( @@ -173,7 +168,7 @@ public void testBuildTableAndInlineData() throws IOException ); final List rowsFromStage4 = - ((InlineDataSource) broadcastJoinHelper.inlineChannelData(new InputNumberDataSource(4))).getRowsAsList(); + ((InlineDataSource) broadcastJoinReader.inlineChannelData(new InputNumberDataSource(4))).getRowsAsList(); Assert.assertEquals(2, rowsFromStage4.size()); FrameTestUtil.assertRowsEqual( @@ -181,7 +176,7 @@ public void testBuildTableAndInlineData() throws IOException Sequences.simple(rowsFromStage4.stream().map(Arrays::asList).collect(Collectors.toList())) ); - final DataSource inlinedJoinDataSource = broadcastJoinHelper.inlineChannelData( + final DataSource inlinedJoinDataSource = broadcastJoinReader.inlineChannelData( JoinDataSource.create( new InputNumberDataSource(0), new InputNumberDataSource(4), @@ -216,7 +211,8 @@ public void testBuildTableMemoryLimit() throws IOException final List channelReaders = new ArrayList<>(); channelReaders.add(frameReader1); - final BroadcastJoinHelper broadcastJoinHelper = new BroadcastJoinHelper( + final BroadcastJoinSegmentMapFnProcessor broadcastJoinHelper = new BroadcastJoinSegmentMapFnProcessor( + null /* Query; not used for the methods we're testing */, sideStageChannelNumberMap, channels, channelReaders, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java index 060b14cec12a..4b750f167d8d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessorTest.java @@ -45,6 +45,7 @@ import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Unit; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -1215,14 +1216,14 @@ private List> run( final RowSignature joinSignature ) { - final ListenableFuture retValFromProcessor = exec.runFully(processor, null); + final ListenableFuture retValFromProcessor = exec.runFully(processor, null); final Sequence> rowsFromProcessor = FrameTestUtil.readRowsFromFrameChannel( readableOutputChannel, FrameReader.create(joinSignature) ); final List> rows = rowsFromProcessor.toList(); - Assert.assertEquals(0L, (long) FutureUtils.getUnchecked(retValFromProcessor, true)); + Assert.assertEquals(Unit.instance(), FutureUtils.getUnchecked(retValFromProcessor, true)); return rows; } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java index 70700c026d63..d89b62977a34 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/results/QueryResultsFrameProcessorTest.java @@ -30,6 +30,7 @@ import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.java.util.common.Unit; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.msq.input.ReadableInput; @@ -100,7 +101,7 @@ public void sanityTest() throws ExecutionException, InterruptedException, IOExce stagePartition ).getChannel(), outputChannel.writable()); - ListenableFuture retVal = exec.runFully(processor, null); + ListenableFuture retVal = exec.runFully(processor, null); final Sequence> rowsFromProcessor = FrameTestUtil.readRowsFromFrameChannel( outputChannel.readable(), FrameReader.create(signature) @@ -109,7 +110,7 @@ public void sanityTest() throws ExecutionException, InterruptedException, IOExce FrameTestUtil.readRowsFromAdapter(adapter, signature, false), rowsFromProcessor ); - Assert.assertEquals(adapter.getNumRows(), (long) retVal.get()); + Assert.assertEquals(Unit.instance(), retVal.get()); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java index 283c8c6d1e2c..de6d6a65fe6c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/scan/ScanQueryFrameProcessorTest.java @@ -21,7 +21,6 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; -import it.unimi.dsi.fastutil.ints.Int2ObjectMaps; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.frame.Frame; @@ -39,6 +38,7 @@ import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Unit; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.msq.input.ReadableInput; @@ -61,6 +61,7 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; +import java.util.function.Function; public class ScanQueryFrameProcessorTest extends InitializedNullHandlingTest { @@ -125,8 +126,10 @@ public void test_runWithInputChannel() throws Exception final ScanQueryFrameProcessor processor = new ScanQueryFrameProcessor( query, + null, + new DefaultObjectMapper(), ReadableInput.channel(inputChannel.readable(), FrameReader.create(signature), stagePartition), - Int2ObjectMaps.emptyMap(), + Function.identity(), new ResourceHolder() { @Override @@ -146,13 +149,10 @@ public void close() } } }, - new ReferenceCountingResourceHolder<>(frameWriterFactory, () -> {}), - null, - 0L, - new DefaultObjectMapper() + new ReferenceCountingResourceHolder<>(frameWriterFactory, () -> {}) ); - ListenableFuture retVal = exec.runFully(processor, null); + ListenableFuture retVal = exec.runFully(processor, null); final Sequence> rowsFromProcessor = FrameTestUtil.readRowsFromFrameChannel( outputChannel.readable(), @@ -164,6 +164,6 @@ public void close() rowsFromProcessor ); - Assert.assertEquals(adapter.getNumRows(), (long) retVal.get()); + Assert.assertEquals(Unit.instance(), retVal.get()); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysSelectQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java similarity index 97% rename from extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysSelectQueryMSQTest.java rename to extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java index 16aef6bdcaaa..82301f4ddfed 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysSelectQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java @@ -36,7 +36,7 @@ /** * Runs {@link CalciteArraysQueryTest} but with MSQ engine */ -public class CalciteArraysSelectQueryMSQTest extends CalciteArraysQueryTest +public class CalciteArraysQueryMSQTest extends CalciteArraysQueryTest { private TestGroupByBuffers groupByBuffers; diff --git a/extensions-core/protobuf-extensions/pom.xml b/extensions-core/protobuf-extensions/pom.xml index 1f267b51754c..cc39430b96ff 100644 --- a/extensions-core/protobuf-extensions/pom.xml +++ b/extensions-core/protobuf-extensions/pom.xml @@ -187,7 +187,6 @@ org.apache.maven.plugins maven-shade-plugin - 3.4.1 true diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 95a7f0c05288..9481b78cc51d 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -140,17 +140,20 @@ hamcrest-core test + org.mockito - mockito-inline + mockito-core + ${mockito.version} test org.mockito - mockito-core + mockito-${mockito.inline.artifact} test + hadoop3 @@ -197,6 +200,12 @@ shade + + false ${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 067abfdb0fb2..7aacd1b5f43b 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -265,7 +265,6 @@ org.mockito mockito-core - ${mockito.version} test diff --git a/licenses.yaml b/licenses.yaml index f213152a4eeb..3b42871419ae 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -857,27 +857,6 @@ libraries: --- -name: net.bytebuddy byte-buddy -license_category: binary -module: extensions/druid-pac4j -license_name: Apache License version 2.0 -version: 1.12.7 -libraries: - - net.bytebuddy: byte-buddy - - net.bytebuddy: byte-buddy-agent - ---- - -name: org.mockito mockito-core -license_category: binary -module: extensions/druid-pac4j -license_name: MIT License -version: 4.3.1 -libraries: - - org.mockito: mockito-core - ---- - name: javax.activation activation license_category: binary module: extensions/druid-pac4j diff --git a/owasp-dependency-check-suppressions.xml b/owasp-dependency-check-suppressions.xml index 2813623f7a70..e33231ea9ee3 100644 --- a/owasp-dependency-check-suppressions.xml +++ b/owasp-dependency-check-suppressions.xml @@ -758,6 +758,7 @@ https://github.com/apache/hadoop/commit/ad49ddda0e1d9632c8c9fcdc78fca8244e1248c9 --> CVE-2023-1370 CVE-2023-37475 + CVE-2023-39410 @@ -801,4 +802,13 @@ CVE-2023-4785 CVE-2023-33953 + + + + + ^pkg:maven/org\.codehaus\.plexus/plexus-interpolation@.*$ + CVE-2022-4244 + diff --git a/pom.xml b/pom.xml index 8fb99191ad0b..6be27033ff92 100644 --- a/pom.xml +++ b/pom.xml @@ -111,7 +111,11 @@ 5.13.0 5.13.0 3.3.6 - 4.3.1 + 5.5.0 + + core 1.12.497 2.8.0 0.8.7 @@ -125,6 +129,7 @@ v1-rev20230606-2.0.0 v1-rev20230301-2.0.0 + maven.org Maven Central Repository https://repo1.maven.org/maven2/ @@ -805,7 +810,7 @@ org.xerial.snappy snappy-java - 1.1.10.3 + 1.1.10.4 com.google.protobuf @@ -1039,7 +1044,7 @@ org.easymock easymock - 4.3 + 5.2.0 test @@ -1049,11 +1054,11 @@ test - org.mockito - mockito-bom - ${mockito.version} - import - pom + org.mockito + mockito-bom + ${mockito.version} + import + pom com.github.docker-java @@ -1103,6 +1108,13 @@ assertj-core 3.24.2 test + + + + net.bytebuddy + byte-buddy + + com.ircclouds.irc @@ -1240,7 +1252,7 @@ nl.jqno.equalsverifier equalsverifier - 3.10.1 + 3.15.1 test @@ -1480,7 +1492,7 @@ org.codehaus.mojo animal-sniffer-maven-plugin - 1.17 + 1.23 check-java-api @@ -1663,6 +1675,7 @@ @{jacocoArgLine} ${jdk.strong.encapsulation.argLine} + ${jdk.security.manager.allow.argLine} -Xmx1500m -XX:MaxDirectMemorySize=2500m -XX:+ExitOnOutOfMemoryError @@ -1730,10 +1743,7 @@ org.apache.maven.plugins maven-shade-plugin - 3.4.1 - - false - + 3.5.0 org.apache.maven.plugins @@ -1763,7 +1773,7 @@ org.codehaus.mojo exec-maven-plugin - 1.6.0 + 3.1.0 org.codehaus.mojo @@ -1806,6 +1816,31 @@ + + + pre-java-11 + + (,11) + + + + 4.11.0 + inline + + + + java-12+ + + [12,) + + + + + + -Djava.security.manager=allow + + + java-9+ diff --git a/processing/pom.xml b/processing/pom.xml index 492980cc5925..3401a248f509 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -364,9 +364,16 @@ caliper test + org.mockito mockito-core + ${mockito.version} + test + + + org.mockito + mockito-${mockito.inline.artifact} test @@ -426,11 +433,6 @@ jetty-util test - - org.mockito - mockito-inline - test - mysql mysql-connector-java diff --git a/processing/src/main/java/org/apache/druid/frame/field/DoubleArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/DoubleArrayFieldReader.java new file mode 100644 index 000000000000..48f2a5117ee8 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/DoubleArrayFieldReader.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; + +/** + * Reader for fields written by {@link NumericArrayFieldWriter#getDoubleArrayFieldWriter} + */ +public class DoubleArrayFieldReader extends NumericArrayFieldReader +{ + @Override + public ColumnValueSelector makeColumnValueSelector( + Memory memory, + ReadableFieldPointer fieldPointer + ) + { + return new NumericArrayFieldSelector(memory, fieldPointer) + { + final SettableFieldPointer fieldPointer = new SettableFieldPointer(); + final ColumnValueSelector columnValueSelector = + DoubleFieldReader.forArray().makeColumnValueSelector(memory, fieldPointer); + + @Nullable + @Override + public Double getIndividualValueAtMemory(long position) + { + fieldPointer.setPosition(position); + if (columnValueSelector.isNull()) { + return null; + } + return columnValueSelector.getDouble(); + } + + @Override + public int getIndividualFieldSize() + { + return Byte.BYTES + Double.BYTES; + } + }; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldReader.java index 531e14bbda29..7f7a3f8639eb 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldReader.java @@ -21,74 +21,60 @@ import org.apache.datasketches.memory.Memory; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.DoubleColumnSelector; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.column.ValueTypes; - -import javax.annotation.Nullable; /** - * Reads values written by {@link DoubleFieldWriter}. - * - * Values are sortable as bytes without decoding. + * Reads the values produced by {@link DoubleFieldWriter} * - * Format: - * - * - 1 byte: {@link DoubleFieldWriter#NULL_BYTE} or {@link DoubleFieldWriter#NOT_NULL_BYTE} - * - 8 bytes: encoded double, using {@link DoubleFieldWriter#transform} + * @see DoubleFieldWriter + * @see NumericFieldWriter for the details of the byte-format that it expects for reading */ -public class DoubleFieldReader implements FieldReader +public class DoubleFieldReader extends NumericFieldReader { - DoubleFieldReader() + + public static DoubleFieldReader forPrimitive() { + return new DoubleFieldReader(false); } - @Override - public ColumnValueSelector makeColumnValueSelector(Memory memory, ReadableFieldPointer fieldPointer) + public static DoubleFieldReader forArray() { - return new Selector(memory, fieldPointer); + return new DoubleFieldReader(true); } - @Override - public DimensionSelector makeDimensionSelector( - Memory memory, - ReadableFieldPointer fieldPointer, - @Nullable ExtractionFn extractionFn - ) + private DoubleFieldReader(final boolean forArray) { - return ValueTypes.makeNumericWrappingDimensionSelector( - ValueType.DOUBLE, - makeColumnValueSelector(memory, fieldPointer), - extractionFn - ); + super(forArray); } @Override - public boolean isNull(Memory memory, long position) + public ValueType getValueType() { - return memory.getByte(position) == DoubleFieldWriter.NULL_BYTE; + return ValueType.DOUBLE; } @Override - public boolean isComparable() + public ColumnValueSelector getColumnValueSelector( + final Memory memory, + final ReadableFieldPointer fieldPointer, + final byte nullIndicatorByte + ) { - return true; + return new DoubleFieldReader.DoubleFieldSelector(memory, fieldPointer, nullIndicatorByte); } - /** - * Selector that reads a value from a location pointed to by {@link ReadableFieldPointer}. - */ - private static class Selector implements DoubleColumnSelector + private static class DoubleFieldSelector extends NumericFieldReader.Selector implements DoubleColumnSelector { - private final Memory dataRegion; - private final ReadableFieldPointer fieldPointer; - private Selector(final Memory dataRegion, final ReadableFieldPointer fieldPointer) + final Memory dataRegion; + final ReadableFieldPointer fieldPointer; + + public DoubleFieldSelector(Memory dataRegion, ReadableFieldPointer fieldPointer, byte nullIndicatorByte) { + super(dataRegion, fieldPointer, nullIndicatorByte); this.dataRegion = dataRegion; this.fieldPointer = fieldPointer; } @@ -98,19 +84,19 @@ public double getDouble() { assert NullHandling.replaceWithDefault() || !isNull(); final long bits = dataRegion.getLong(fieldPointer.position() + Byte.BYTES); - return DoubleFieldWriter.detransform(bits); + return TransformUtils.detransformToDouble(bits); } @Override - public boolean isNull() + public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - return dataRegion.getByte(fieldPointer.position()) == DoubleFieldWriter.NULL_BYTE; + } @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) + public boolean isNull() { - // Do nothing. + return super._isNull(); } } } diff --git a/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldWriter.java b/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldWriter.java index 87ab109af1c3..08cd2a2bf4c8 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldWriter.java +++ b/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldWriter.java @@ -25,65 +25,49 @@ /** * Wraps a {@link BaseDoubleColumnValueSelector} and writes field values. * - * See {@link DoubleFieldReader} for format details. + * @see NumericFieldWriter for the details of the byte-format that it writes as */ -public class DoubleFieldWriter implements FieldWriter +public class DoubleFieldWriter extends NumericFieldWriter { - public static final int SIZE = Double.BYTES + Byte.BYTES; + private final BaseDoubleColumnValueSelector selector; - // Different from the values in NullHandling, since we want to be able to sort as bytes, and we want - // nulls to come before non-nulls. - public static final byte NULL_BYTE = 0x00; - public static final byte NOT_NULL_BYTE = 0x01; + public static DoubleFieldWriter forPrimitive(final BaseDoubleColumnValueSelector selector) + { + return new DoubleFieldWriter(selector, false); + } - private final BaseDoubleColumnValueSelector selector; + public static DoubleFieldWriter forArray(final BaseDoubleColumnValueSelector selector) + { + return new DoubleFieldWriter(selector, true); + } - public DoubleFieldWriter(final BaseDoubleColumnValueSelector selector) + private DoubleFieldWriter(final BaseDoubleColumnValueSelector selector, final boolean forArray) { + super(selector, forArray); this.selector = selector; } @Override - public long writeTo(final WritableMemory memory, final long position, final long maxSize) + public int getNumericSizeBytes() { - if (maxSize < SIZE) { - return -1; - } - - if (selector.isNull()) { - memory.putByte(position, NULL_BYTE); - memory.putLong(position + Byte.BYTES, transform(0)); - } else { - memory.putByte(position, NOT_NULL_BYTE); - memory.putLong(position + Byte.BYTES, transform(selector.getDouble())); - } - - return SIZE; + return Double.BYTES; } @Override - public void close() + public void writeSelectorToMemory(WritableMemory memory, long position) { - // Nothing to close. + writeToMemory(memory, position, selector.getDouble()); } - /** - * Transforms a double into a form where it can be compared as unsigned bytes without decoding. - */ - public static long transform(final double n) + @Override + public void writeNullToMemory(WritableMemory memory, long position) { - final long bits = Double.doubleToLongBits(n); - final long mask = ((bits & Long.MIN_VALUE) >> 11) | Long.MIN_VALUE; - return Long.reverseBytes(bits ^ mask); + writeToMemory(memory, position, 0); } - /** - * Inverse of {@link #transform}. - */ - public static double detransform(final long bits) + private void writeToMemory(WritableMemory memory, long position, double value) { - final long reversedBits = Long.reverseBytes(bits); - final long mask = (((reversedBits ^ Long.MIN_VALUE) & Long.MIN_VALUE) >> 11) | Long.MIN_VALUE; - return Double.longBitsToDouble(reversedBits ^ mask); + memory.putLong(position, TransformUtils.transformFromDouble(value)); } + } diff --git a/processing/src/main/java/org/apache/druid/frame/field/FieldReaders.java b/processing/src/main/java/org/apache/druid/frame/field/FieldReaders.java index 23b3907ff18c..7221bfe9da26 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/FieldReaders.java +++ b/processing/src/main/java/org/apache/druid/frame/field/FieldReaders.java @@ -24,7 +24,6 @@ import org.apache.druid.frame.key.RowKeyReader; import org.apache.druid.frame.write.UnsupportedColumnTypeException; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.ValueType; /** * Helper used to read field values from row-based frames or {@link RowKey}. @@ -46,23 +45,37 @@ public static FieldReader create(final String columnName, final ColumnType colum { switch (Preconditions.checkNotNull(columnType, "columnType").getType()) { case LONG: - return new LongFieldReader(); + return LongFieldReader.forPrimitive(); case FLOAT: - return new FloatFieldReader(); + return FloatFieldReader.forPrimitive(); case DOUBLE: - return new DoubleFieldReader(); + return DoubleFieldReader.forPrimitive(); case STRING: - return new StringFieldReader(false); + return new StringFieldReader(); case COMPLEX: return ComplexFieldReader.createFromType(columnType); case ARRAY: - if (columnType.getElementType().getType() == ValueType.STRING) { - return new StringFieldReader(true); + switch (Preconditions.checkNotNull(columnType.getElementType().getType(), "array elementType")) { + case STRING: + return new StringArrayFieldReader(); + + case LONG: + return new LongArrayFieldReader(); + + case FLOAT: + return new FloatArrayFieldReader(); + + case DOUBLE: + return new DoubleArrayFieldReader(); + + default: + throw new UnsupportedColumnTypeException(columnName, columnType); + } // Fall through to error for other array types diff --git a/processing/src/main/java/org/apache/druid/frame/field/FieldWriters.java b/processing/src/main/java/org/apache/druid/frame/field/FieldWriters.java index 622f9a6751ea..cc7f68d099d2 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/FieldWriters.java +++ b/processing/src/main/java/org/apache/druid/frame/field/FieldWriters.java @@ -66,21 +66,30 @@ public static FieldWriter create( switch (columnType.getType()) { case LONG: return makeLongWriter(columnSelectorFactory, columnName); + case FLOAT: return makeFloatWriter(columnSelectorFactory, columnName); + case DOUBLE: return makeDoubleWriter(columnSelectorFactory, columnName); + case STRING: return makeStringWriter(columnSelectorFactory, columnName); + + case COMPLEX: + return makeComplexWriter(columnSelectorFactory, columnName, columnType.getComplexTypeName()); + case ARRAY: switch (columnType.getElementType().getType()) { case STRING: return makeStringArrayWriter(columnSelectorFactory, columnName); - default: - throw new UnsupportedColumnTypeException(columnName, columnType); + case LONG: + return makeLongArrayWriter(columnSelectorFactory, columnName); + case FLOAT: + return makeFloatArrayWriter(columnSelectorFactory, columnName); + case DOUBLE: + return makeDoubleArrayWriter(columnSelectorFactory, columnName); } - case COMPLEX: - return makeComplexWriter(columnSelectorFactory, columnName, columnType.getComplexTypeName()); default: throw new UnsupportedColumnTypeException(columnName, columnType); } @@ -92,7 +101,7 @@ private static FieldWriter makeLongWriter( ) { final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); - return new LongFieldWriter(selector); + return LongFieldWriter.forPrimitive(selector); } private static FieldWriter makeFloatWriter( @@ -101,7 +110,7 @@ private static FieldWriter makeFloatWriter( ) { final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); - return new FloatFieldWriter(selector); + return FloatFieldWriter.forPrimitive(selector); } private static FieldWriter makeDoubleWriter( @@ -110,7 +119,7 @@ private static FieldWriter makeDoubleWriter( ) { final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); - return new DoubleFieldWriter(selector); + return DoubleFieldWriter.forPrimitive(selector); } private static FieldWriter makeStringWriter( @@ -127,11 +136,37 @@ private static FieldWriter makeStringArrayWriter( final String columnName ) { - //noinspection rawtypes - final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); + final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); return new StringArrayFieldWriter(selector); } + private static FieldWriter makeLongArrayWriter( + final ColumnSelectorFactory selectorFactory, + final String columnName + ) + { + final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); + return NumericArrayFieldWriter.getLongArrayFieldWriter(selector); + } + + private static FieldWriter makeFloatArrayWriter( + final ColumnSelectorFactory selectorFactory, + final String columnName + ) + { + final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); + return NumericArrayFieldWriter.getFloatArrayFieldWriter(selector); + } + + private static FieldWriter makeDoubleArrayWriter( + final ColumnSelectorFactory selectorFactory, + final String columnName + ) + { + final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); + return NumericArrayFieldWriter.getDoubleArrayFieldWriter(selector); + } + private static FieldWriter makeComplexWriter( final ColumnSelectorFactory selectorFactory, final String columnName, diff --git a/processing/src/main/java/org/apache/druid/frame/field/FloatArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/FloatArrayFieldReader.java new file mode 100644 index 000000000000..fcbe407bdb26 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/FloatArrayFieldReader.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; + +/** + * Reader for fields written by {@link NumericArrayFieldWriter#getFloatArrayFieldWriter} + */ +public class FloatArrayFieldReader extends NumericArrayFieldReader +{ + @Override + public ColumnValueSelector makeColumnValueSelector( + Memory memory, + ReadableFieldPointer fieldPointer + ) + { + return new NumericArrayFieldSelector(memory, fieldPointer) + { + final SettableFieldPointer fieldPointer = new SettableFieldPointer(); + final ColumnValueSelector columnValueSelector = + FloatFieldReader.forArray().makeColumnValueSelector(memory, fieldPointer); + + @Nullable + @Override + public Float getIndividualValueAtMemory(long position) + { + fieldPointer.setPosition(position); + if (columnValueSelector.isNull()) { + return null; + } + return columnValueSelector.getFloat(); + } + + @Override + public int getIndividualFieldSize() + { + return Byte.BYTES + Float.BYTES; + } + }; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/FloatFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/FloatFieldReader.java index 17baaa95df47..6617d563d679 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/FloatFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/FloatFieldReader.java @@ -21,71 +21,60 @@ import org.apache.datasketches.memory.Memory; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.FloatColumnSelector; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.column.ValueTypes; - -import javax.annotation.Nullable; /** * Reads values written by {@link FloatFieldWriter}. * - * Values are sortable as bytes without decoding. - * - * Format: - * - * - 1 byte: {@link FloatFieldWriter#NULL_BYTE} or {@link FloatFieldWriter#NOT_NULL_BYTE} - * - 4 bytes: encoded float, using {@link FloatFieldWriter#transform} + * @see FloatFieldWriter + * @see NumericFieldWriter for the details of the byte-format that it expects for reading */ -public class FloatFieldReader implements FieldReader +public class FloatFieldReader extends NumericFieldReader { - FloatFieldReader() + + public static FloatFieldReader forPrimitive() { + return new FloatFieldReader(false); } - @Override - public ColumnValueSelector makeColumnValueSelector(Memory memory, ReadableFieldPointer fieldPointer) + public static FloatFieldReader forArray() { - return new Selector(memory, fieldPointer); + return new FloatFieldReader(true); } - @Override - public DimensionSelector makeDimensionSelector( - Memory memory, - ReadableFieldPointer fieldPointer, - @Nullable ExtractionFn extractionFn - ) + private FloatFieldReader(final boolean forArray) { - return ValueTypes.makeNumericWrappingDimensionSelector( - ValueType.FLOAT, - makeColumnValueSelector(memory, fieldPointer), - extractionFn - ); + super(forArray); } @Override - public boolean isNull(Memory memory, long position) + public ValueType getValueType() { - return memory.getByte(position) == FloatFieldWriter.NULL_BYTE; + return ValueType.FLOAT; } @Override - public boolean isComparable() + public ColumnValueSelector getColumnValueSelector( + final Memory memory, + final ReadableFieldPointer fieldPointer, + final byte nullIndicatorByte + ) { - return true; + return new FloatFieldSelector(memory, fieldPointer, nullIndicatorByte); } - private static class Selector implements FloatColumnSelector + private static class FloatFieldSelector extends NumericFieldReader.Selector implements FloatColumnSelector { - private final Memory dataRegion; - private final ReadableFieldPointer fieldPointer; - private Selector(final Memory dataRegion, final ReadableFieldPointer fieldPointer) + final Memory dataRegion; + final ReadableFieldPointer fieldPointer; + + public FloatFieldSelector(Memory dataRegion, ReadableFieldPointer fieldPointer, byte nullIndicatorByte) { + super(dataRegion, fieldPointer, nullIndicatorByte); this.dataRegion = dataRegion; this.fieldPointer = fieldPointer; } @@ -95,19 +84,19 @@ public float getFloat() { assert NullHandling.replaceWithDefault() || !isNull(); final int bits = dataRegion.getInt(fieldPointer.position() + Byte.BYTES); - return FloatFieldWriter.detransform(bits); + return TransformUtils.detransformToFloat(bits); } @Override - public boolean isNull() + public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - return dataRegion.getByte(fieldPointer.position()) == FloatFieldWriter.NULL_BYTE; + } @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) + public boolean isNull() { - // Do nothing. + return super._isNull(); } } } diff --git a/processing/src/main/java/org/apache/druid/frame/field/FloatFieldWriter.java b/processing/src/main/java/org/apache/druid/frame/field/FloatFieldWriter.java index e81f0b8e099c..cf826932f8b0 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/FloatFieldWriter.java +++ b/processing/src/main/java/org/apache/druid/frame/field/FloatFieldWriter.java @@ -25,65 +25,54 @@ /** * Wraps a {@link BaseFloatColumnValueSelector} and writes field values. * - * See {@link FloatFieldReader} for format details. + * @see NumericFieldWriter for the details of the byte-format that it writes as */ -public class FloatFieldWriter implements FieldWriter +public class FloatFieldWriter extends NumericFieldWriter { - public static final int SIZE = Float.BYTES + Byte.BYTES; + private final BaseFloatColumnValueSelector selector; - // Different from the values in NullHandling, since we want to be able to sort as bytes, and we want - // nulls to come before non-nulls. - public static final byte NULL_BYTE = 0x00; - public static final byte NOT_NULL_BYTE = 0x01; + public static FloatFieldWriter forPrimitive(final BaseFloatColumnValueSelector selector) + { + return new FloatFieldWriter(selector, false); + } - private final BaseFloatColumnValueSelector selector; + public static FloatFieldWriter forArray(final BaseFloatColumnValueSelector selector) + { + return new FloatFieldWriter(selector, true); + } - public FloatFieldWriter(final BaseFloatColumnValueSelector selector) + private FloatFieldWriter(final BaseFloatColumnValueSelector selector, final boolean forArray) { + super(selector, forArray); this.selector = selector; } @Override - public long writeTo(final WritableMemory memory, final long position, final long maxSize) + public void close() { - if (maxSize < SIZE) { - return -1; - } - - if (selector.isNull()) { - memory.putByte(position, NULL_BYTE); - memory.putInt(position + Byte.BYTES, transform(0)); - } else { - memory.putByte(position, NOT_NULL_BYTE); - memory.putInt(position + Byte.BYTES, transform(selector.getFloat())); - } + // Nothing to close. + } - return SIZE; + @Override + public int getNumericSizeBytes() + { + return Float.BYTES; } @Override - public void close() + public void writeSelectorToMemory(WritableMemory memory, long position) { - // Nothing to close. + writeToMemory(memory, position, selector.getFloat()); } - /** - * Transforms a float into a form where it can be compared as unsigned bytes without decoding. - */ - public static int transform(final float n) + @Override + public void writeNullToMemory(WritableMemory memory, long position) { - final int bits = Float.floatToIntBits(n); - final int mask = ((bits & Integer.MIN_VALUE) >> 8) | Integer.MIN_VALUE; - return Integer.reverseBytes(bits ^ mask); + writeToMemory(memory, position, 0); } - /** - * Inverse of {@link #transform}. - */ - public static float detransform(final int bits) + private void writeToMemory(WritableMemory memory, long position, float value) { - final int reversedBits = Integer.reverseBytes(bits); - final int mask = (((reversedBits ^ Integer.MIN_VALUE) & Integer.MIN_VALUE) >> 8) | Integer.MIN_VALUE; - return Float.intBitsToFloat(reversedBits ^ mask); + memory.putInt(position, TransformUtils.transformFromFloat(value)); } } diff --git a/processing/src/main/java/org/apache/druid/frame/field/LongArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/LongArrayFieldReader.java new file mode 100644 index 000000000000..b52b39d13c47 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/LongArrayFieldReader.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; + +/** + * Reader for fields written by {@link NumericArrayFieldWriter#getLongArrayFieldWriter} + */ +public class LongArrayFieldReader extends NumericArrayFieldReader +{ + @Override + public ColumnValueSelector makeColumnValueSelector( + Memory memory, + ReadableFieldPointer fieldPointer + ) + { + return new NumericArrayFieldSelector(memory, fieldPointer) + { + final SettableFieldPointer fieldPointer = new SettableFieldPointer(); + final ColumnValueSelector columnValueSelector = + LongFieldReader.forArray().makeColumnValueSelector(memory, fieldPointer); + + @Nullable + @Override + public Long getIndividualValueAtMemory(long position) + { + fieldPointer.setPosition(position); + if (columnValueSelector.isNull()) { + return null; + } + return columnValueSelector.getLong(); + } + + @Override + public int getIndividualFieldSize() + { + return Byte.BYTES + Long.BYTES; + } + }; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/LongFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/LongFieldReader.java index dd8b6b40ad27..8f3bbbf04517 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/LongFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/LongFieldReader.java @@ -21,75 +21,61 @@ import org.apache.datasketches.memory.Memory; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.LongColumnSelector; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.column.ValueTypes; - -import javax.annotation.Nullable; /** * Reads values written by {@link LongFieldWriter}. * - * Values are sortable as bytes without decoding. - * - * Format: - * - * - 1 byte: {@link LongFieldWriter#NULL_BYTE} or {@link LongFieldWriter#NOT_NULL_BYTE} - * - 8 bytes: encoded long: big-endian order, with sign flipped + * @see LongFieldWriter + * @see NumericFieldWriter for the details of the byte-format that it expects for reading */ -public class LongFieldReader implements FieldReader +public class LongFieldReader extends NumericFieldReader { - LongFieldReader() + + public static LongFieldReader forPrimitive() { + return new LongFieldReader(false); } - @Override - public ColumnValueSelector makeColumnValueSelector(Memory memory, ReadableFieldPointer fieldPointer) + public static LongFieldReader forArray() { - return new Selector(memory, fieldPointer); + return new LongFieldReader(true); } - @Override - public DimensionSelector makeDimensionSelector( - Memory memory, - ReadableFieldPointer fieldPointer, - @Nullable ExtractionFn extractionFn - ) + private LongFieldReader(final boolean forArray) { - return ValueTypes.makeNumericWrappingDimensionSelector( - ValueType.LONG, - makeColumnValueSelector(memory, fieldPointer), - extractionFn - ); + super(forArray); } @Override - public boolean isNull(Memory memory, long position) + public ValueType getValueType() { - return memory.getByte(position) == LongFieldWriter.NULL_BYTE; + return ValueType.LONG; } @Override - public boolean isComparable() + public ColumnValueSelector getColumnValueSelector( + final Memory memory, + final ReadableFieldPointer fieldPointer, + final byte nullIndicatorByte + ) { - return true; + return new LongFieldSelector(memory, fieldPointer, nullIndicatorByte); } - /** - * Selector that reads a value from a location pointed to by {@link ReadableFieldPointer}. - */ - private static class Selector implements LongColumnSelector + private static class LongFieldSelector extends NumericFieldReader.Selector implements LongColumnSelector { - private final Memory memory; - private final ReadableFieldPointer fieldPointer; - private Selector(final Memory memory, final ReadableFieldPointer fieldPointer) + final Memory dataRegion; + final ReadableFieldPointer fieldPointer; + + public LongFieldSelector(Memory dataRegion, ReadableFieldPointer fieldPointer, byte nullIndicatorByte) { - this.memory = memory; + super(dataRegion, fieldPointer, nullIndicatorByte); + this.dataRegion = dataRegion; this.fieldPointer = fieldPointer; } @@ -97,20 +83,20 @@ private Selector(final Memory memory, final ReadableFieldPointer fieldPointer) public long getLong() { assert NullHandling.replaceWithDefault() || !isNull(); - final long bits = memory.getLong(fieldPointer.position() + Byte.BYTES); - return LongFieldWriter.detransform(bits); + final long bits = dataRegion.getLong(fieldPointer.position() + Byte.BYTES); + return TransformUtils.detransformToLong(bits); } @Override - public boolean isNull() + public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - return memory.getByte(fieldPointer.position()) == LongFieldWriter.NULL_BYTE; + } @Override - public void inspectRuntimeShape(final RuntimeShapeInspector inspector) + public boolean isNull() { - // Do nothing. + return super._isNull(); } } } diff --git a/processing/src/main/java/org/apache/druid/frame/field/LongFieldWriter.java b/processing/src/main/java/org/apache/druid/frame/field/LongFieldWriter.java index ddd1884a6d21..669d384ff157 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/LongFieldWriter.java +++ b/processing/src/main/java/org/apache/druid/frame/field/LongFieldWriter.java @@ -23,63 +23,55 @@ import org.apache.druid.segment.BaseLongColumnValueSelector; /** - * Wraps a {@link BaseLongColumnValueSelector} and writes individual values into rframe rows. + * Wraps a {@link BaseLongColumnValueSelector} and writes individual values into frame rows. * - * See {@link LongFieldReader} for format details. + * @see NumericFieldWriter for the details of the byte-format that it writes as */ -public class LongFieldWriter implements FieldWriter +public class LongFieldWriter extends NumericFieldWriter { - public static final int SIZE = Long.BYTES + Byte.BYTES; - public static final byte NULL_BYTE = 0x00; - public static final byte NOT_NULL_BYTE = 0x01; - private final BaseLongColumnValueSelector selector; + public static LongFieldWriter forPrimitive(final BaseLongColumnValueSelector selector) + { + return new LongFieldWriter(selector, false); + } + + static LongFieldWriter forArray(final BaseLongColumnValueSelector selector) + { + return new LongFieldWriter(selector, true); + } + + // Different from the values in NullHandling, since we want to be able to sort as bytes, and we want // nulls to come before non-nulls. - public LongFieldWriter(final BaseLongColumnValueSelector selector) + private LongFieldWriter(final BaseLongColumnValueSelector selector, final boolean forArray) { + super(selector, forArray); this.selector = selector; } @Override - public long writeTo(final WritableMemory memory, final long position, final long maxSize) + public int getNumericSizeBytes() { - if (maxSize < SIZE) { - return -1; - } - - if (selector.isNull()) { - memory.putByte(position, NULL_BYTE); - memory.putLong(position + Byte.BYTES, transform(0)); - } else { - memory.putByte(position, NOT_NULL_BYTE); - memory.putLong(position + Byte.BYTES, transform(selector.getLong())); - } - - return SIZE; + return Long.BYTES; } @Override - public void close() + public void writeSelectorToMemory(WritableMemory memory, long position) { - // Nothing to do. + writeToMemory(memory, position, selector.getLong()); } - /** - * Transforms a long into a form where it can be compared as unsigned bytes without decoding. - */ - public static long transform(final long n) + @Override + public void writeNullToMemory(WritableMemory memory, long position) { - // Must flip the first (sign) bit so comparison-as-bytes works. - return Long.reverseBytes(n ^ Long.MIN_VALUE); + writeToMemory(memory, position, 0); + } - /** - * Reverse the {@link #transform(long)} function. - */ - public static long detransform(final long bits) + private void writeToMemory(WritableMemory memory, long position, long value) { - return Long.reverseBytes(bits) ^ Long.MIN_VALUE; + memory.putLong(position, TransformUtils.transformFromLong(value)); } + } diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldReader.java new file mode 100644 index 000000000000..7eab229e6be6 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldReader.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.error.DruidException; +import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.segment.DimensionSelector; + +import javax.annotation.Nullable; + +/** + * Reader class for the fields written by {@link NumericArrayFieldWriter}. See the Javadoc for the writer for more + * information on the format + * + * The numeric array fields are byte comparable + */ +public abstract class NumericArrayFieldReader implements FieldReader +{ + @Override + public DimensionSelector makeDimensionSelector( + Memory memory, + ReadableFieldPointer fieldPointer, + @Nullable ExtractionFn extractionFn + ) + { + throw DruidException.defensive("Cannot call makeDimensionSelector on field of type ARRAY"); + } + + @Override + public boolean isNull(Memory memory, long position) + { + final byte firstByte = memory.getByte(position); + return firstByte == NumericArrayFieldWriter.NULL_ROW; + } + + @Override + public boolean isComparable() + { + return true; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldSelector.java b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldSelector.java new file mode 100644 index 000000000000..1871aef06e08 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldSelector.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.error.DruidException; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.List; + +/** + * Base implementation of the column value selector that the concrete numeric field reader implementations inherit from. + * The selector contains the logic to construct an array written by {@link NumericArrayFieldWriter}, and present it as + * a column value selector. + * + * The inheritors of this class are expected to implement + * 1. {@link #getIndividualValueAtMemory} Which extracts the element from the field where it was written to. Returns + * null if the value at that location represents a null element + * 2. {@link #getIndividualFieldSize} Which informs the method about the field size corresponding to each element in + * the numeric array's serialized representation + * + * @param Type of the individual array elements + */ +public abstract class NumericArrayFieldSelector implements ColumnValueSelector +{ + /** + * Memory containing the serialized values of the array + */ + protected final Memory memory; + + /** + * Pointer to location in the memory. The callers are expected to update the pointer's position to the start of the + * array that they wish to get prior to {@link #getObject()} call. + * + * Frames read and written using {@link org.apache.druid.frame.write.FrameWriter} and + * {@link org.apache.druid.frame.read.FrameReader} shouldn't worry about this detail, since they automatically update + * and handle the start location + */ + private final ReadableFieldPointer fieldPointer; + + /** + * Position last read, for caching the last fetched result + */ + private long currentFieldPosition = -1; + + /** + * Value of the row at the location beginning at {@link #currentFieldPosition} + */ + private final List currentRow = new ArrayList<>(); + + /** + * Nullity of the row at the location beginning at {@link #currentFieldPosition} + */ + private boolean currentRowIsNull; + + public NumericArrayFieldSelector(final Memory memory, final ReadableFieldPointer fieldPointer) + { + this.memory = memory; + this.fieldPointer = fieldPointer; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // Do nothing + } + + @Nullable + @Override + public Object getObject() + { + final List currentArray = computeCurrentArray(); + + if (currentArray == null) { + return null; + } + + return currentArray.toArray(); + } + + @Override + public Class classOfObject() + { + return Object.class; + } + + @Override + public double getDouble() + { + return 0; + } + + @Override + public float getFloat() + { + return 0; + } + + @Override + public long getLong() + { + return 0; + } + + @Override + public boolean isNull() + { + long position = fieldPointer.position(); + final byte firstByte = memory.getByte(position); + return firstByte == NumericArrayFieldWriter.NULL_ROW; + } + + /** + * Returns the value of the individual element written at the given position + */ + @Nullable + public abstract ElementType getIndividualValueAtMemory(long position); + + /** + * Returns the field size that each element in the reader array consumes. It is usually 1 + ElementType.SIZE, to hold + * the element's nullity, and it's representation. + */ + public abstract int getIndividualFieldSize(); + + @Nullable + private List computeCurrentArray() + { + final long fieldPosition = fieldPointer.position(); + + if (fieldPosition != currentFieldPosition) { + updateCurrentArray(fieldPosition); + } + + this.currentFieldPosition = fieldPosition; + + if (currentRowIsNull) { + return null; + } + return currentRow; + + } + + private void updateCurrentArray(final long fieldPosition) + { + currentRow.clear(); + currentRowIsNull = false; + + long position = fieldPosition; + long limit = memory.getCapacity(); + + // Check the first byte, and if it is null, update the current value to null and return + if (isNull()) { + currentRowIsNull = true; + return; + } + + // Adding a check here to prevent the position from potentially overflowing + if (position < limit) { + position++; + } + + // Sanity check, to make sure that we see the rowTerminator at the end + boolean rowTerminatorSeen = false; + + while (position < limit) { + final byte kind = memory.getByte(position); + + // Break as soon as we see the ARRAY_TERMINATOR (0x00) + if (kind == NumericArrayFieldWriter.ARRAY_TERMINATOR) { + rowTerminatorSeen = true; + break; + } + + // If terminator not seen, then read the field at that location, and increment the position by the element's field + // size to read the next element. + currentRow.add(getIndividualValueAtMemory(position)); + position += getIndividualFieldSize(); + } + + if (!rowTerminatorSeen) { + throw DruidException.defensive("Unexpected end of field"); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldWriter.java b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldWriter.java new file mode 100644 index 000000000000..e220d6bdc519 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldWriter.java @@ -0,0 +1,249 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.frame.write.FrameWriterUtils; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Writes the values of the type ARRAY where X is a numeric type to row based frames. + * The format of the array written is as follows: + *

+ * Format: + * - 1 Byte - {@link #NULL_ROW} or {@link #NON_NULL_ROW} denoting whether the array itself is null + * - If the array is null, then the writer stops here + * - If the array is not null, then it proceeds to the following steps + *

+ * For each value in the non-null array: + * - 1 Byte - {@link NumericFieldWriter#ARRAY_ELEMENT_NULL_BYTE} or {@link NumericFieldWriter#ARRAY_ELEMENT_NOT_NULL_BYTE} + * denothing whether the proceeding value is null or not. + * - ElementSize Bytes - The encoded value of the element + *

+ * Once all the values in the non-null arrays are over, writes {@link #ARRAY_TERMINATOR}. This is to aid the byte + * comparison, and also let the reader know that the number of elements in the array are over. + *

+ * The format doesn't add the number of elements in the array at the beginning, so that the serialization of the arrays + * are byte-by-byte comparable. + *

+ * Examples: + * 1. null + * | Bytes | Value | Interpretation | + * |--------|-------|-----------------------------| + * | 1 | 0x00 | Denotes that the array null | + *

+ * 2. [] (empty array) + * | Bytes | Value | Interpretation | + * |--------|----- -|------------------------------------| + * | 1 | 0x01 | Denotes that the array is not null | + * | 2 | 0x00 | End of the array | + *

+ * 3. [5L, null, 6L] + * | Bytes | Value | Interpretation | + * |---------|--------------|-----------------------------------------------------------------------------------| + * | 1 | 0x01 | Denotes that the array is not null | + * | 2 | 0x02 | Denotes that the next element is not null | + * | 3-10 | transform(5) | Representation of 5 | + * | 11 | 0x01 | Denotes that the next element is null | + * | 12-19 | transform(0) | Representation of 0 (default value, the reader will ignore it if SqlCompatible mode is on | + * | 20 | 0x02 | Denotes that the next element is not null | + * | 21-28 | transform(6) | Representation of 6 | + * | 29 | 0x00 | End of array | + */ +public class NumericArrayFieldWriter implements FieldWriter +{ + + /** + * Denotes that the array itself is null + */ + public static final byte NULL_ROW = 0x00; + + /** + * Denotes that the array is non null + */ + public static final byte NON_NULL_ROW = 0x01; + + /** + * Marks the end of the array. Since {@link #NULL_ROW} and {@link #ARRAY_TERMINATOR} will only occur at different + * locations, therefore there is no clash in keeping both's values at 0x00 + */ + public static final byte ARRAY_TERMINATOR = 0x00; + + private final ColumnValueSelector selector; + private final NumericFieldWriterFactory writerFactory; + + /** + * Returns the writer for ARRAY + */ + public static NumericArrayFieldWriter getLongArrayFieldWriter(final ColumnValueSelector selector) + { + return new NumericArrayFieldWriter(selector, LongFieldWriter::forArray); + } + + /** + * Returns the writer for ARRAY + */ + public static NumericArrayFieldWriter getFloatArrayFieldWriter(final ColumnValueSelector selector) + { + return new NumericArrayFieldWriter(selector, FloatFieldWriter::forArray); + } + + /** + * Returns the writer for ARRAY + */ + public static NumericArrayFieldWriter getDoubleArrayFieldWriter(final ColumnValueSelector selector) + { + return new NumericArrayFieldWriter(selector, DoubleFieldWriter::forArray); + } + + public NumericArrayFieldWriter(final ColumnValueSelector selector, NumericFieldWriterFactory writerFactory) + { + this.selector = selector; + this.writerFactory = writerFactory; + } + + @Override + public long writeTo(WritableMemory memory, long position, long maxSize) + { + Object row = selector.getObject(); + if (row == null) { + int requiredSize = Byte.BYTES; + if (requiredSize > maxSize) { + return -1; + } + memory.putByte(position, NULL_ROW); + return requiredSize; + } else { + + List list = FrameWriterUtils.getNumericArrayFromObject(row); + + if (list == null) { + int requiredSize = Byte.BYTES; + if (requiredSize > maxSize) { + return -1; + } + memory.putByte(position, NULL_ROW); + return requiredSize; + } + + // Create a columnValueSelector to write the individual elements re-using the NumericFieldWriter + AtomicInteger index = new AtomicInteger(0); + ColumnValueSelector columnValueSelector = new ColumnValueSelector() + { + @Override + public double getDouble() + { + final Number n = getObject(); + assert NullHandling.replaceWithDefault() || n != null; + return n != null ? n.doubleValue() : 0d; + } + + @Override + public float getFloat() + { + final Number n = getObject(); + assert NullHandling.replaceWithDefault() || n != null; + return n != null ? n.floatValue() : 0f; + } + + @Override + public long getLong() + { + final Number n = getObject(); + assert NullHandling.replaceWithDefault() || n != null; + return n != null ? n.longValue() : 0L; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + + @Override + public boolean isNull() + { + // Arrays preserve the individual element's nullity when they are written and read. + // Therefore, when working with SQL incompatible mode, [7, null] won't change to [7, 0] when written to and + // read from the underlying serialization (as compared with the primitives). Therefore, + // even when NullHandling.replaceWithDefault() is true we need to write null as is, and not convert it to their + // default value when writing the array. Therefore, the check is `getObject() == null` ignoring the value of + // `NullHandling.replaceWithDefaul()`. + return getObject() == null; + } + + @Nullable + @Override + public Number getObject() + { + return list.get(index.get()); + } + + @Override + public Class classOfObject() + { + return Number.class; + } + }; + + NumericFieldWriter writer = writerFactory.get(columnValueSelector); + + // First byte is reserved for null marker of the array + // Next [(1 + Numeric Size) x Number of elements of array] bytes are reserved for the elements of the array and + // their null markers + // Last byte is reserved for array termination + int requiredSize = Byte.BYTES + (writer.getNumericSizeBytes() + Byte.BYTES) * list.size() + Byte.BYTES; + + if (requiredSize > maxSize) { + return -1; + } + + long offset = 0; + memory.putByte(position + offset, NON_NULL_ROW); + offset += Byte.BYTES; + + for (; index.get() < list.size(); index.incrementAndGet()) { + writer.writeTo( + memory, + position + offset, + maxSize - offset + ); + offset += Byte.BYTES + writer.getNumericSizeBytes(); + } + + memory.putByte(position + offset, ARRAY_TERMINATOR); + + return requiredSize; + + } + } + + @Override + public void close() + { + // Do nothing + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldReader.java new file mode 100644 index 000000000000..cf83c405d092 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldReader.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ValueTypes; + +import javax.annotation.Nullable; + +/** + * Reads the fields created by the {@link NumericFieldWriter}. See the Javadoc for the writer for format details + * + * @see NumericFieldWriter + */ +public abstract class NumericFieldReader implements FieldReader +{ + + /** + * The indicator byte which denotes that the following value is null. + */ + private final byte nullIndicatorByte; + + public NumericFieldReader(boolean forArray) + { + if (!forArray) { + this.nullIndicatorByte = NumericFieldWriter.NULL_BYTE; + } else { + this.nullIndicatorByte = NumericFieldWriter.ARRAY_ELEMENT_NULL_BYTE; + } + } + + @Override + public ColumnValueSelector makeColumnValueSelector(Memory memory, ReadableFieldPointer fieldPointer) + { + return getColumnValueSelector(memory, fieldPointer, nullIndicatorByte); + } + + @Override + public DimensionSelector makeDimensionSelector( + Memory memory, + ReadableFieldPointer fieldPointer, + @Nullable ExtractionFn extractionFn + ) + { + return ValueTypes.makeNumericWrappingDimensionSelector( + getValueType(), + makeColumnValueSelector(memory, fieldPointer), + extractionFn + ); + } + + @Override + public boolean isNull(Memory memory, long position) + { + return memory.getByte(position) == nullIndicatorByte; + } + + + @Override + public boolean isComparable() + { + return true; + } + + /** + * Creates a column value selector for the element written at fieldPointer's position in the memory. + * The nullilty check is handled by the nullIndicatorByte + */ + public abstract ColumnValueSelector getColumnValueSelector( + Memory memory, + ReadableFieldPointer fieldPointer, + byte nullIndicatorByte + ); + + /** + * {@link ValueType} of the inheritor's element + */ + public abstract ValueType getValueType(); + + /** + * Helper class which allows the inheritors to fetch the nullity of the field located at fieldPointer's position in + * the dataRegion. + * + * The implementations of the column value selectors returned by the {@link #getColumnValueSelector} can inherit this + * class and call {@link #_isNull()} in their {@link ColumnValueSelector#isNull()} to offload the responsibility of + * detecting null elements to this Selector, instead of reworking the null handling + */ + public abstract static class Selector + { + private final Memory dataRegion; + private final ReadableFieldPointer fieldPointer; + private final byte nullIndicatorByte; + + public Selector( + final Memory dataRegion, + final ReadableFieldPointer fieldPointer, + final byte nullIndicatorByte + ) + { + this.dataRegion = dataRegion; + this.fieldPointer = fieldPointer; + this.nullIndicatorByte = nullIndicatorByte; + } + + protected boolean _isNull() + { + return dataRegion.getByte(fieldPointer.position()) == nullIndicatorByte; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriter.java b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriter.java new file mode 100644 index 000000000000..7aa604480c91 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriter.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.segment.BaseNullableColumnValueSelector; + +/** + * FieldWriter for numeric datatypes. The parent class does the null handling for the underlying data, while + * the individual subclasses write the individual element (long, float or double type). This also allows for a clean + * reuse of the readers and writers between the numeric types and also allowing the array writers ({@link NumericArrayFieldWriter}) + * to use these methods directly without duplication + * + * Format: + * - 1 byte: Whether the following value is null or not. Take a look at the note on the indicator bytes. + * - X bytes: Encoded value of the selector, or the default value if it is null. X denotes the size of the numeric value + * + * Indicator bytes for denoting whether the element is null or not null changes depending on whether the writer is used + * to write the data for individual value (like LONG) or for an element of an array (like ARRAY). This is because + * array support for the numeric types was added later and by then the field writers for individual fields were using + * 0x00 to denote the null byte, which is reserved for denoting the array end when we are writing the elements as part + * of the array instead. (0x00 is used for array end because it helps in preserving the byte comparison property of the + * numeric array field writers). + * + * Therefore, to preserve backward and forward compatibility, the individual element's writers were left unchanged, + * while the array's element's writers used 0x01 and 0x02 to denote null and non-null byte respectively + * + * Values produced by the writer are sortable without decoding + * + * @see NumericArrayFieldWriter for examples of how this class serializes the field for numeric arrays + */ +public abstract class NumericFieldWriter implements FieldWriter +{ + /** + * Indicator byte denoting that the numeric value succeeding it is null. This is used in the primitive + * writers. NULL_BYTE < NOT_NULL_BYTE to preserve the ordering while doing byte comparison + */ + public static final byte NULL_BYTE = 0x00; + + /** + * Indicator byte denoting that the numeric value succeeding it is not null. This is used in the primitive + * writers + */ + public static final byte NOT_NULL_BYTE = 0x01; + + /** + * Indicator byte denoting that the numeric value succeeding it is null. This is used while writing the individual + * elements writers of an array. ARRAY_ELEMENT_NULL_BYTE < ARRAY_ELEMENT_NOT_NULL_BYTE to preserve the ordering + * while doing byte comparison + */ + public static final byte ARRAY_ELEMENT_NULL_BYTE = 0x01; + + /** + * Indicator byte denoting that the numeric value succeeding it is not null. This is used while writing the individual + * elements writers of an array + */ + public static final byte ARRAY_ELEMENT_NOT_NULL_BYTE = 0x02; + + private final BaseNullableColumnValueSelector selector; + private final byte nullIndicatorByte; + private final byte notNullIndicatorByte; + + public NumericFieldWriter( + final BaseNullableColumnValueSelector selector, + final boolean forArray + ) + { + this.selector = selector; + if (!forArray) { + this.nullIndicatorByte = NULL_BYTE; + this.notNullIndicatorByte = NOT_NULL_BYTE; + } else { + this.nullIndicatorByte = ARRAY_ELEMENT_NULL_BYTE; + this.notNullIndicatorByte = ARRAY_ELEMENT_NOT_NULL_BYTE; + } + } + + @Override + public long writeTo(WritableMemory memory, long position, long maxSize) + { + int size = getNumericSizeBytes() + Byte.BYTES; + + if (maxSize < size) { + return -1; + } + + // Using isNull() since this is a primitive type + if (selector.isNull()) { + memory.putByte(position, nullIndicatorByte); + writeNullToMemory(memory, position + Byte.BYTES); + } else { + memory.putByte(position, notNullIndicatorByte); + writeSelectorToMemory(memory, position + Byte.BYTES); + } + + return size; + } + + @Override + public void close() + { + // Nothing to do + } + + /** + * @return The size in bytes of the numeric datatype that the implementation of this writer occupies + */ + public abstract int getNumericSizeBytes(); + + /** + * Writes the value pointed by the selector to memory. The caller should ensure that the selector gives out the + * correct primitive type + */ + public abstract void writeSelectorToMemory(WritableMemory memory, long position); + + /** + * Writes the default value for the type to the memory. For long, it is 0L, for double, it is 0.0d etc. Useful mainly + * when the SQL incompatible mode is turned off, and maintains the fact that the size of the numeric field written + * doesn't vary irrespective of whether the value is null + */ + public abstract void writeNullToMemory(WritableMemory memory, long position); +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriterFactory.java b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriterFactory.java new file mode 100644 index 000000000000..a7ae47c91f4e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriterFactory.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import org.apache.druid.segment.ColumnValueSelector; + +/** + * Factory for {@link NumericFieldWriter} + */ +public interface NumericFieldWriterFactory +{ + /** + * Constructs an instance of {@link NumericFieldWriter} given the column selector + */ + NumericFieldWriter get(ColumnValueSelector selector); +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/SettableFieldPointer.java b/processing/src/main/java/org/apache/druid/frame/field/SettableFieldPointer.java new file mode 100644 index 000000000000..d26f84f251de --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/SettableFieldPointer.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +/** + * A simple {@link ReadableFieldPointer} that returns the position that was set on its object. + */ +public class SettableFieldPointer implements ReadableFieldPointer +{ + + long position = 0; + + public void setPosition(long position) + { + this.position = position; + } + + @Override + public long position() + { + return position; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/StringArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/StringArrayFieldReader.java new file mode 100644 index 000000000000..67bed4cb051b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/StringArrayFieldReader.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +/** + * Reads fields written by {@link StringArrayFieldWriter} + * + * @see StringFieldReader for more details on the format that the reader expects + * @see StringFieldReader#StringFieldReader(boolean) for the selector that the reader returns + */ +public class StringArrayFieldReader extends StringFieldReader +{ + StringArrayFieldReader() + { + super(true); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java index 0cce325d2123..6b15804d662a 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java @@ -70,6 +70,11 @@ public class StringFieldReader implements FieldReader { private final boolean asArray; + public StringFieldReader() + { + this(false); + } + /** * Create a string reader. * @@ -77,7 +82,7 @@ public class StringFieldReader implements FieldReader * selectors (potentially multi-value ones). If true, selectors from {@link #makeColumnValueSelector} * behave like string array selectors. */ - StringFieldReader(final boolean asArray) + protected StringFieldReader(final boolean asArray) { this.asArray = asArray; } diff --git a/processing/src/main/java/org/apache/druid/frame/field/TransformUtils.java b/processing/src/main/java/org/apache/druid/frame/field/TransformUtils.java new file mode 100644 index 000000000000..f882f0443568 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/TransformUtils.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +/** + * Utility methods to map the primitive numeric types into an equi-wide byte representation, such that the + * given byte sequence preserves the ordering of the original type when done byte comparison. + * Checkout {@link org.apache.druid.frame.read.FrameReaderUtils#compareMemoryToByteArrayUnsigned} for how this byte + * comparison is performed. + */ +public class TransformUtils +{ + /** + * Transforms a double into a form where it can be compared as unsigned bytes without decoding. + */ + public static long transformFromDouble(final double n) + { + final long bits = Double.doubleToLongBits(n); + final long mask = ((bits & Long.MIN_VALUE) >> 11) | Long.MIN_VALUE; + return Long.reverseBytes(bits ^ mask); + } + + /** + * Inverse of {@link #transformFromDouble}. + */ + public static double detransformToDouble(final long bits) + { + final long reversedBits = Long.reverseBytes(bits); + final long mask = (((reversedBits ^ Long.MIN_VALUE) & Long.MIN_VALUE) >> 11) | Long.MIN_VALUE; + return Double.longBitsToDouble(reversedBits ^ mask); + } + + /** + * Transforms a long into a form where it can be compared as unsigned bytes without decoding. + */ + public static long transformFromLong(final long n) + { + // Must flip the first (sign) bit so comparison-as-bytes works. + return Long.reverseBytes(n ^ Long.MIN_VALUE); + } + + /** + * Reverse the {@link #transformFromLong(long)} function. + */ + public static long detransformToLong(final long bits) + { + return Long.reverseBytes(bits) ^ Long.MIN_VALUE; + } + + /** + * Transforms a float into a form where it can be compared as unsigned bytes without decoding. + */ + public static int transformFromFloat(final float n) + { + final int bits = Float.floatToIntBits(n); + final int mask = ((bits & Integer.MIN_VALUE) >> 8) | Integer.MIN_VALUE; + return Integer.reverseBytes(bits ^ mask); + } + + /** + * Inverse of {@link #transformFromFloat(float)}. + */ + public static float detransformToFloat(final int bits) + { + final int reversedBits = Integer.reverseBytes(bits); + final int mask = (((reversedBits ^ Integer.MIN_VALUE) & Integer.MIN_VALUE) >> 8) | Integer.MIN_VALUE; + return Float.intBitsToFloat(reversedBits ^ mask); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessorExecutor.java b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessorExecutor.java index 20f9f7f4fe3e..02161d5ad5ad 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessorExecutor.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessorExecutor.java @@ -35,10 +35,10 @@ import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.processor.manager.ProcessorManager; import org.apache.druid.java.util.common.Either; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.logger.Logger; import javax.annotation.Nullable; @@ -53,7 +53,6 @@ import java.util.Set; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutorService; -import java.util.function.BiFunction; import java.util.stream.Collectors; /** @@ -399,21 +398,15 @@ void doProcessorCleanup() throws IOException * Runs a sequence of processors and returns a future that resolves when execution is complete. Returns a value * accumulated using the provided {@code accumulateFn}. * - * @param processors sequence of processors to run - * @param initialResult initial value for result accumulation. If there are no processors at all, this - * is returned. - * @param accumulateFn result accumulator. Applied in a critical section, so it should be something - * that executes quickly. It gets {@code initialResult} for the initial accumulation. + * @param processorManager processors to run * @param maxOutstandingProcessors maximum number of processors to run at once * @param bouncer additional limiter on outstanding processors, beyond maxOutstandingProcessors. * Useful when there is some finite resource being shared against multiple different * calls to this method. * @param cancellationId optional cancellation id for {@link #runFully}. */ - public ListenableFuture runAllFully( - final Sequence> processors, - final ResultType initialResult, - final BiFunction accumulateFn, + public ListenableFuture runAllFully( + final ProcessorManager processorManager, final int maxOutstandingProcessors, final Bouncer bouncer, @Nullable final String cancellationId @@ -421,10 +414,8 @@ public ListenableFuture runAllFully( { // Logic resides in a separate class in order to keep this one simpler. return new RunAllFullyWidget<>( - processors, + processorManager, this, - initialResult, - accumulateFn, maxOutstandingProcessors, bouncer, cancellationId diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java index 3380c7688309..2e21bbc312bf 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FrameProcessors.java @@ -177,11 +177,11 @@ public static void closeAll( } for (WritableFrameChannel channel : Lists.reverse(writableFrameChannels)) { - closer.register(channel::close); + closer.register(channel); } for (ReadableFrameChannel channel : Lists.reverse(readableFrameChannels)) { - closer.register(channel::close); + closer.register(channel); } closer.close(); diff --git a/processing/src/main/java/org/apache/druid/frame/processor/RunAllFullyWidget.java b/processing/src/main/java/org/apache/druid/frame/processor/RunAllFullyWidget.java index 67724cdf8042..a1a1c0f87120 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/RunAllFullyWidget.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/RunAllFullyWidget.java @@ -27,21 +27,21 @@ import com.google.common.util.concurrent.SettableFuture; import com.google.errorprone.annotations.concurrent.GuardedBy; import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.frame.processor.manager.ProcessorAndCallback; +import org.apache.druid.frame.processor.manager.ProcessorManager; import org.apache.druid.java.util.common.Either; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Yielder; -import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.utils.CloseableUtils; import javax.annotation.Nullable; import java.util.ArrayDeque; import java.util.Collections; import java.util.IdentityHashMap; +import java.util.Optional; import java.util.Queue; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiFunction; /** * Helper for {@link FrameProcessorExecutor#runAllFully}. See the javadoc for that method for details about the @@ -57,29 +57,23 @@ public class RunAllFullyWidget { private static final Logger log = new Logger(RunAllFullyWidget.class); - private final Sequence> processors; + private final ProcessorManager processorManager; private final FrameProcessorExecutor exec; - private final ResultType initialResult; - private final BiFunction accumulateFn; private final int maxOutstandingProcessors; private final Bouncer bouncer; @Nullable private final String cancellationId; RunAllFullyWidget( - Sequence> processors, + ProcessorManager processorManager, FrameProcessorExecutor exec, - ResultType initialResult, - BiFunction accumulateFn, int maxOutstandingProcessors, Bouncer bouncer, @Nullable String cancellationId ) { - this.processors = processors; + this.processorManager = processorManager; this.exec = exec; - this.initialResult = initialResult; - this.accumulateFn = accumulateFn; this.maxOutstandingProcessors = maxOutstandingProcessors; this.bouncer = bouncer; this.cancellationId = cancellationId; @@ -87,21 +81,25 @@ public class RunAllFullyWidget ListenableFuture run() { - final Yielder> processorYielder; + final ListenableFuture>> nextProcessorFuture; try { - processorYielder = Yielders.each(processors); + nextProcessorFuture = processorManager.next(); + if (nextProcessorFuture.isDone() && !nextProcessorFuture.get().isPresent()) { + // Save some time and return immediately. + final ResultType retVal = processorManager.result(); + processorManager.close(); + return Futures.immediateFuture(retVal); + } } catch (Throwable e) { + CloseableUtils.closeAndSuppressExceptions(processorManager, e::addSuppressed); return Futures.immediateFailedFuture(e); } - if (processorYielder.isDone()) { - return Futures.immediateFuture(initialResult); - } - // This single RunAllFullyRunnable will be submitted to the executor "maxOutstandingProcessors" times. - final RunAllFullyRunnable runnable = new RunAllFullyRunnable(processorYielder); + // It runs concurrently with itself. + final RunAllFullyRunnable runnable = new RunAllFullyRunnable(nextProcessorFuture); for (int i = 0; i < maxOutstandingProcessors; i++) { exec.getExecutorService().submit(runnable); @@ -117,27 +115,37 @@ private class RunAllFullyRunnable implements Runnable private final Object runAllFullyLock = new Object(); @GuardedBy("runAllFullyLock") - Yielder> processorYielder; - - @GuardedBy("runAllFullyLock") - ResultType currentResult = null; - - @GuardedBy("runAllFullyLock") - boolean seenFirstResult = false; + ListenableFuture>> nextProcessorFuture; + /** + * Number of processors currently outstanding. Final cleanup is done when there are no more processors in + * {@link #processorManager}, and when this reaches zero. + */ @GuardedBy("runAllFullyLock") - int outstandingProcessors = 0; + int outstandingProcessors; + /** + * Currently outstanding futures from {@link FrameProcessorExecutor#runFully}. Used for cancellation. + */ @GuardedBy("runAllFullyLock") Set> outstandingFutures = Collections.newSetFromMap(new IdentityHashMap<>()); + /** + * Tickets from a {@link Bouncer} that are available for use by the next instance of this class to run. + */ @Nullable // nulled out by cleanup() @GuardedBy("runAllFullyLock") - Queue bouncerQueue = new ArrayDeque<>(); + Queue bouncerTicketQueue = new ArrayDeque<>(); + + /** + * Whether {@link #cleanup()} has executed. + */ + @GuardedBy("runAllFullyLock") + boolean didCleanup; - private RunAllFullyRunnable(final Yielder> processorYielder) + private RunAllFullyRunnable(final ListenableFuture>> nextProcessorFuture) { - this.processorYielder = processorYielder; + this.nextProcessorFuture = nextProcessorFuture; this.finishedFuture = exec.registerCancelableFuture(SettableFuture.create(), false, cancellationId); this.finishedFuture.addListener( () -> { @@ -165,18 +173,47 @@ private RunAllFullyRunnable(final Yielder> processor @Override public void run() { - final FrameProcessor nextProcessor; + final ProcessorAndCallback nextProcessor; Bouncer.Ticket nextTicket = null; synchronized (runAllFullyLock) { - if (finished.get() != null) { - cleanupIfNoMoreProcessors(); - return; - } else if (!processorYielder.isDone()) { - assert bouncerQueue != null; + try { + if (finished.get() != null) { + cleanupIfNoMoreProcessors(); + return; + } else { + if (!nextProcessorFuture.isDone()) { + final ListenableFuture>> futureRef = nextProcessorFuture; + // Wait for readability. + // Note: this code effectively runs *all* outstanding RunAllFullyRunnable when the future resolves, even + // if only a single processor is available to be run. Still correct, but may be wasteful in situations + // where a processor manager blocks frequently. + futureRef.addListener( + () -> { + if (!futureRef.isCancelled()) { + exec.getExecutorService().submit(RunAllFullyRunnable.this); + } + }, + Execs.directExecutor() + ); + return; + } + + final Optional> maybeNextProcessor = nextProcessorFuture.get(); + + if (!maybeNextProcessor.isPresent()) { + // Finished. + if (outstandingProcessors == 0) { + finished.compareAndSet(null, Either.value(processorManager.result())); + cleanupIfNoMoreProcessors(); + } + return; + } + + // Next processor is ready to run. Let's do it. + assert bouncerTicketQueue != null; - try { - final Bouncer.Ticket ticketFromQueue = bouncerQueue.poll(); + final Bouncer.Ticket ticketFromQueue = bouncerTicketQueue.poll(); if (ticketFromQueue != null) { nextTicket = ticketFromQueue; @@ -187,6 +224,7 @@ public void run() if (ticketFuture.isDone() && !ticketFuture.isCancelled()) { nextTicket = FutureUtils.getUncheckedImmediately(ticketFuture); } else { + // No ticket available. Run again when there's a ticket. ticketFuture.addListener( () -> { if (!ticketFuture.isCancelled()) { @@ -198,7 +236,7 @@ public void run() ticket.giveBack(); return; } else { - bouncerQueue.add(ticket); + bouncerTicketQueue.add(ticket); } } exec.getExecutorService().submit(RunAllFullyRunnable.this); @@ -212,25 +250,25 @@ public void run() } assert outstandingProcessors < maxOutstandingProcessors; - nextProcessor = processorYielder.get(); - processorYielder = processorYielder.next(null); + nextProcessor = maybeNextProcessor.get(); + nextProcessorFuture = processorManager.next(); outstandingProcessors++; } - catch (Throwable e) { - if (nextTicket != null) { - nextTicket.giveBack(); - } - finished.compareAndSet(null, Either.error(e)); - cleanupIfNoMoreProcessors(); - return; + } + catch (Throwable e) { + if (nextTicket != null) { + nextTicket.giveBack(); } - } else { + finished.compareAndSet(null, Either.error(e)); + cleanupIfNoMoreProcessors(); return; } assert nextTicket != null; + assert nextProcessor != null; + final ListenableFuture future = exec.runFully( - FrameProcessors.withBaggage(nextProcessor, nextTicket::giveBack), + FrameProcessors.withBaggage(nextProcessor.processor(), nextTicket::giveBack), cancellationId ); @@ -244,25 +282,17 @@ public void run() public void onSuccess(T result) { final boolean isDone; - ResultType retVal = null; try { synchronized (runAllFullyLock) { outstandingProcessors--; outstandingFutures.remove(future); - if (!seenFirstResult) { - currentResult = accumulateFn.apply(initialResult, result); - seenFirstResult = true; - } else { - currentResult = accumulateFn.apply(currentResult, result); - } + isDone = outstandingProcessors == 0 + && nextProcessorFuture.isDone() + && !nextProcessorFuture.get().isPresent(); - isDone = outstandingProcessors == 0 && processorYielder.isDone(); - - if (isDone) { - retVal = currentResult; - } + nextProcessor.onComplete(result); } } catch (Throwable e) { @@ -276,7 +306,7 @@ public void onSuccess(T result) } if (isDone) { - finished.compareAndSet(null, Either.value(retVal)); + finished.compareAndSet(null, Either.value(processorManager.result())); synchronized (runAllFullyLock) { cleanupIfNoMoreProcessors(); @@ -310,7 +340,7 @@ public void onFailure(Throwable t) @GuardedBy("runAllFullyLock") private void cleanupIfNoMoreProcessors() { - if (outstandingProcessors == 0 && finished.get() != null) { + if (outstandingProcessors == 0 && finished.get() != null && !didCleanup) { cleanup(); } } @@ -324,33 +354,42 @@ private void cleanup() assert finished.get() != null; assert outstandingProcessors == 0; + Throwable caught = null; + try { - if (bouncerQueue != null) { + if (bouncerTicketQueue != null) { // Drain ticket queue, return everything. Bouncer.Ticket ticket; - while ((ticket = bouncerQueue.poll()) != null) { + while ((ticket = bouncerTicketQueue.poll()) != null) { ticket.giveBack(); } - bouncerQueue = null; + bouncerTicketQueue = null; } - if (processorYielder != null) { - processorYielder.close(); - processorYielder = null; - } + processorManager.close(); } catch (Throwable e) { - // No point throwing, since our caller is just a future callback. - log.noStackTrace().warn(e, "Exception encountered while cleaning up from runAllFully"); + caught = e; } finally { + didCleanup = true; + // Set finishedFuture after all cleanup is done. if (finished.get().isValue()) { - finishedFuture.set(finished.get().valueOrThrow()); + if (caught != null) { + // Propagate exception caught during cleanup. + finishedFuture.setException(caught); + } else { + finishedFuture.set(finished.get().valueOrThrow()); + } } else { - finishedFuture.setException(finished.get().error()); + final Throwable t = finished.get().error(); + if (caught != null) { + t.addSuppressed(caught); + } + finishedFuture.setException(t); } } } diff --git a/processing/src/main/java/org/apache/druid/frame/processor/manager/AccumulatingProcessorManager.java b/processing/src/main/java/org/apache/druid/frame/processor/manager/AccumulatingProcessorManager.java new file mode 100644 index 000000000000..aa010808a99d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/processor/manager/AccumulatingProcessorManager.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.processor.manager; + +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.java.util.common.ISE; + +import java.util.Optional; +import java.util.function.BiFunction; + +/** + * Processor manager that wraps another {@link ProcessorManager} and accumulates a result. + */ +public class AccumulatingProcessorManager implements ProcessorManager +{ + private final ProcessorManager delegate; + private final BiFunction accumulateFn; + private R currentResult; + + public AccumulatingProcessorManager( + ProcessorManager delegate, + R initialResult, + BiFunction accumulateFn + ) + { + this.delegate = delegate; + this.currentResult = Preconditions.checkNotNull(initialResult, "initialResult"); + this.accumulateFn = accumulateFn; + } + + @Override + public ListenableFuture>> next() + { + if (currentResult == null) { + throw new ISE("Closed"); + } + + return FutureUtils.transform( + delegate.next(), + nextProcessor -> nextProcessor.map( + retVal -> new ProcessorAndCallback<>( + retVal.processor(), + r -> { + currentResult = accumulateFn.apply(currentResult, r); + retVal.onComplete(r); + } + ) + ) + ); + } + + @Override + public R result() + { + return Preconditions.checkNotNull(currentResult, "currentResult"); + } + + @Override + public void close() + { + currentResult = null; + delegate.close(); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/processor/manager/ProcessorAndCallback.java b/processing/src/main/java/org/apache/druid/frame/processor/manager/ProcessorAndCallback.java new file mode 100644 index 000000000000..ec6a56409be3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/processor/manager/ProcessorAndCallback.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.processor.manager; + +import com.google.common.base.Preconditions; +import org.apache.druid.frame.processor.FrameProcessor; + +import javax.annotation.Nullable; +import java.util.function.Consumer; + +/** + * Processor and success callback returned by {@link ProcessorManager#next()}. + */ +public class ProcessorAndCallback +{ + private final FrameProcessor processor; + private final Consumer callback; + + public ProcessorAndCallback(FrameProcessor processor, @Nullable Consumer callback) + { + this.processor = Preconditions.checkNotNull(processor, "processor"); + this.callback = callback; + } + + public FrameProcessor processor() + { + return processor; + } + + public void onComplete(final T resultObject) + { + if (callback != null) { + callback.accept(resultObject); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/processor/manager/ProcessorManager.java b/processing/src/main/java/org/apache/druid/frame/processor/manager/ProcessorManager.java new file mode 100644 index 000000000000..83d7c96ea939 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/processor/manager/ProcessorManager.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.processor.manager; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.processor.FrameProcessorExecutor; +import org.apache.druid.java.util.common.guava.Sequence; + +import java.io.Closeable; +import java.util.Optional; +import java.util.function.BiFunction; + +/** + * Used by {@link FrameProcessorExecutor#runAllFully} to manage the launching of processors. Processors returned by + * this class may run concurrently with each other. + * + * This interface allows for simple sequences of processors, such as {@link ProcessorManagers#of(Sequence)}. It also + * allows for situations where later processors depend on the results of earlier processors. (The result of earlier + * processors are made available to the manager through {@link ProcessorAndCallback#onComplete(Object)}.) + * + * Implementations do not need to be thread-safe. + * + * @param return type of {@link org.apache.druid.frame.processor.FrameProcessor} created by this manager + * @param result type of this manager; see {@link #result()} + */ +public interface ProcessorManager extends Closeable +{ + /** + * Returns the next processor that should be run, along with a callback. The callback is called when the processor + * completes successfully, along with the result of the processor. If the processor fails, the callback is not called. + * + * The callback is called in a thread-safe manner: it will never be called concurrently with another callback, or + * concurrently with a call to "next" or {@link #close()}. To ensure this, {@link FrameProcessorExecutor#runAllFully} + * synchronizes executions of callbacks for the same processor manager. Therefore, it is important that the callbacks + * executed quickly. + * + * This method returns a future, so it allows for logic where the construction of later processors depends on the + * results of earlier processors. + * + * Returns an empty Optional if there are no more processors to run. + * + * Behavior of this method is undefined if called after {@link #close()}. + * + * @throws java.util.NoSuchElementException if a prior call to this method had returned an empty Optional + */ + ListenableFuture>> next(); + + /** + * Called after all procesors are done, prior to {@link #close()}, to retrieve the result of this computation. + * + * Behavior of this method is undefined if called after {@link #close()}, or if called prior to {@link #next()} + * returning an empty {@link Optional}, or if called prior to all callbacks from {@link #next()} having been called. + */ + R result(); + + /** + * Called when all processors are done, or when one has failed. + * + * This method releases all resources associated with this manager. After calling this method, callers must call no + * other methods. + */ + @Override + void close(); + + /** + * Returns an {@link AccumulatingProcessorManager} that wraps this manager and accumulates a result, to be returned + * by its {@link #result()} method. + * + * @param initialResult initial accumulated value + * @param accumulateFn accumulation function, will be provided the current accumulated value and the incremental + * new value. + * @param result type of the returned manager + */ + default ProcessorManager withAccumulation( + R2 initialResult, + BiFunction accumulateFn + ) + { + return new AccumulatingProcessorManager<>(this, initialResult, accumulateFn); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/processor/manager/ProcessorManagers.java b/processing/src/main/java/org/apache/druid/frame/processor/manager/ProcessorManagers.java new file mode 100644 index 000000000000..f2469712dc4d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/processor/manager/ProcessorManagers.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.processor.manager; + +import com.google.common.collect.Iterators; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; + +import java.util.function.Supplier; + +/** + * Utility functions for creating {@link ProcessorManager}. + */ +public class ProcessorManagers +{ + private ProcessorManagers() + { + // No instantiation. + } + + /** + * Manager with zero processors. Returns the number of processors run. + */ + public static ProcessorManager none() + { + return of(Sequences.empty()); + } + + /** + * Manager with processors derived from a {@link Sequence}. Returns the number of processors run. + */ + public static ProcessorManager of(final Sequence> processors) + { + return new SequenceProcessorManager<>(processors); + } + + /** + * Manager with processors derived from an {@link Iterable}. Returns the number of processors run. + */ + public static ProcessorManager of(final Iterable> processors) + { + return of(Sequences.simple(processors)); + } + + /** + * Manager with a single processor derived from a {@link Supplier}. Returns the number of processors run. + */ + public static ProcessorManager of(final Supplier> processors) + { + return of(Sequences.simple(() -> Iterators.singletonIterator(processors.get()))); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/processor/manager/SequenceProcessorManager.java b/processing/src/main/java/org/apache/druid/frame/processor/manager/SequenceProcessorManager.java new file mode 100644 index 000000000000..c4276ff5cbf1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/processor/manager/SequenceProcessorManager.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.processor.manager; + +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.utils.CloseableUtils; + +import java.util.NoSuchElementException; +import java.util.Optional; + +/** + * Processor manager based on a {@link Sequence}. Returns the number of processors run. + */ +public class SequenceProcessorManager> implements ProcessorManager +{ + private final Sequence

sequence; + private Yielder

yielder; + private boolean done; + private long numProcessors; + + SequenceProcessorManager(final Sequence

sequence) + { + this.sequence = sequence; + } + + @Override + public ListenableFuture>> next() + { + initializeYielderIfNeeded(); + + if (done) { + if (yielder == null) { + // Already closed. + throw new IllegalStateException(); + } else { + // Not closed yet, but done. + throw new NoSuchElementException(); + } + } else if (yielder.isDone()) { + done = true; + return Futures.immediateFuture(Optional.empty()); + } else { + final P retVal; + try { + retVal = Preconditions.checkNotNull(yielder.get(), "processor"); + yielder = yielder.next(null); + } + catch (Throwable e) { + // Some problem with yielder.get() or yielder.next(null). Close the yielder and mark us as done. + done = true; + CloseableUtils.closeAndSuppressExceptions(yielder, e::addSuppressed); + yielder = null; + throw e; + } + + return Futures.immediateFuture(Optional.of(new ProcessorAndCallback<>(retVal, r -> numProcessors++))); + } + } + + @Override + public Long result() + { + return numProcessors; + } + + @Override + public void close() + { + done = true; + + if (yielder != null) { + // Clean up. + CloseableUtils.closeAndWrapExceptions(yielder); + yielder = null; + } + } + + /** + * Initialize {@link #yielder} if needed. + */ + private void initializeYielderIfNeeded() + { + if (!done && yielder == null) { + yielder = Yielders.each(sequence); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java index e5cc20746b49..8ddf99325d39 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java @@ -74,6 +74,7 @@ private FrameReader( * If the columnType is null, we store the data as {@link ColumnType#NESTED_DATA}. This can be done if we know that * the data that we receive can be serded generically using the nested data. It is currently used in the brokers to * store the data with unknown types into frames. + * * @param signature signature used to generate the reader */ public static FrameReader create(final RowSignature signature) @@ -96,8 +97,8 @@ public static FrameReader create(final RowSignature signature) signature.getColumnName(columnNumber) ); - columnReaders.add(FrameColumnReaders.create(columnNumber, columnType)); fieldReaders.add(FieldReaders.create(signature.getColumnName(columnNumber), columnType)); + columnReaders.add(FrameColumnReaders.create(signature.getColumnName(columnNumber), columnNumber, columnType)); } return new FrameReader(signature, columnReaders, fieldReaders); @@ -110,7 +111,7 @@ public RowSignature signature() /** * Returns capabilities for a particular column in a particular frame. - * + *

* Preferred over {@link RowSignature#getColumnCapabilities(String)} when reading a particular frame, because this * method has more insight into what's actually going on with that specific frame (nulls, multivalue, etc). The * RowSignature version is based solely on type. @@ -151,7 +152,7 @@ public CursorFactory makeCursorFactory(final Frame frame) /** * Create a {@link FrameComparisonWidget} for the given frame. - * + *

* Only possible for frames of type {@link org.apache.druid.frame.FrameType#ROW_BASED}. The provided * sortColumns must be a prefix of {@link #signature()}. */ diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java index 4caaeaae87b6..98218819ce13 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java @@ -19,10 +19,15 @@ package org.apache.druid.frame.read.columnar; -import org.apache.druid.java.util.common.UOE; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; +/** + * Creates {@link FrameColumnReader} corresponding to a given column type and number. + * + * Returns a dummy {@link UnsupportedColumnTypeFrameColumnReader} if the column type is not supported or unknown. + * Calling any method of the dummy reader will throw with relevant error message. + */ public class FrameColumnReaders { private FrameColumnReaders() @@ -30,7 +35,11 @@ private FrameColumnReaders() // No instantiation. } - public static FrameColumnReader create(final int columnNumber, final ColumnType columnType) + public static FrameColumnReader create( + final String columnName, + final int columnNumber, + final ColumnType columnType + ) { switch (columnType.getType()) { case LONG: @@ -51,11 +60,12 @@ public static FrameColumnReader create(final int columnNumber, final ColumnType case ARRAY: if (columnType.getElementType().getType() == ValueType.STRING) { return new StringFrameColumnReader(columnNumber, true); + } else { + return new UnsupportedColumnTypeFrameColumnReader(columnName, columnType); } - // Fall through to error for other array types default: - throw new UOE("Unsupported column type [%s]", columnType); + return new UnsupportedColumnTypeFrameColumnReader(columnName, columnType); } } } diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/UnsupportedColumnTypeFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/UnsupportedColumnTypeFrameColumnReader.java new file mode 100644 index 000000000000..06f356128247 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/UnsupportedColumnTypeFrameColumnReader.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.read.columnar; + +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.write.UnsupportedColumnTypeException; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.segment.column.ColumnType; + +import javax.annotation.Nullable; + +/** + * Dummy reader for unsupported types. Throws {@link UnsupportedColumnTypeException} if we try to call any method of the + * frame reader + */ +public class UnsupportedColumnTypeFrameColumnReader implements FrameColumnReader +{ + + private final String columnName; + @Nullable + private final ColumnType columnType; + + UnsupportedColumnTypeFrameColumnReader(String columnName, @Nullable ColumnType columnType) + { + this.columnName = columnName; + this.columnType = columnType; + } + + @Override + public Column readRACColumn(Frame frame) + { + throw new UnsupportedColumnTypeException(columnName, columnType); + } + + @Override + public ColumnPlus readColumn(Frame frame) + { + throw new UnsupportedColumnTypeException(columnName, columnType); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java b/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java index 577443cedd92..ac9574d7f052 100644 --- a/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java +++ b/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java @@ -33,6 +33,8 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.data.ComparableIntArray; +import org.apache.druid.segment.data.ComparableList; import org.apache.druid.segment.data.ComparableStringArray; import org.apache.druid.segment.data.IndexedInts; @@ -132,7 +134,7 @@ public static List getUtf8ByteBuffersFromStringSelector( * Retrieves UTF-8 byte buffers from a {@link ColumnValueSelector}, which is expected to be the kind of * selector you get for an {@code ARRAY} column. * - * Null strings are returned as {@link #NULL_STRING_MARKER_ARRAY}. + * Null strings are returned as {@code null}. * * If the entire array returned by {@link BaseObjectColumnValueSelector#getObject()} is null, returns either * null or {@link #NULL_STRING_MARKER_ARRAY} depending on the value of "useNullArrays". @@ -172,6 +174,48 @@ public static List getUtf8ByteBuffersFromStringArraySelector( return retVal; } + /** + * Retrieves a numeric list from a Java object, given that the object is an instance of something that can be returned + * from {@link ColumnValueSelector#getObject()} of valid numeric array selectors representations + * + * While {@link BaseObjectColumnValueSelector} specifies that only instances of {@code Object[]} can be returned from + * the numeric array selectors, this method also handles a few more cases which can be encountered if the selector is + * directly implemented on top of the group by stuff + */ + @Nullable + public static List getNumericArrayFromObject(Object row) + { + if (row == null) { + return null; + } else if (row instanceof Number) { + return Collections.singletonList((Number) row); + } + + final List retVal = new ArrayList<>(); + + if (row instanceof List) { + for (int i = 0; i < ((List) row).size(); i++) { + retVal.add((Number) ((List) row).get(i)); + } + } else if (row instanceof Object[]) { + for (Object value : (Object[]) row) { + retVal.add((Number) value); + } + } else if (row instanceof ComparableList) { + for (Object value : ((ComparableList) row).getDelegate()) { + retVal.add((Number) value); + } + } else if (row instanceof ComparableIntArray) { + for (int value : ((ComparableIntArray) row).getDelegate()) { + retVal.add(value); + } + } else { + throw new ISE("Unexpected type %s found", row.getClass().getName()); + } + + return retVal; + } + /** * Checks the provided signature for any disallowed field names. Returns any that are found. */ diff --git a/processing/src/main/java/org/apache/druid/java/util/common/Unit.java b/processing/src/main/java/org/apache/druid/java/util/common/Unit.java new file mode 100644 index 000000000000..1229f351bde7 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/java/util/common/Unit.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.java.util.common; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; + +import java.util.Collections; +import java.util.Map; + +/** + * A type that has exactly one value. (Not counting null.) + * + * Jackson-serializable. + */ +public final class Unit +{ + private static final Unit INSTANCE = new Unit(); + + private Unit() + { + // Singleton. + } + + @JsonCreator + public static Unit instance() + { + return INSTANCE; + } + + @JsonValue + private Map asMap() + { + return Collections.emptyMap(); + } +} diff --git a/processing/src/main/java/org/apache/druid/java/util/common/granularity/PeriodGranularity.java b/processing/src/main/java/org/apache/druid/java/util/common/granularity/PeriodGranularity.java index 4f9acefd2849..ba9ba02feb9e 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/granularity/PeriodGranularity.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/granularity/PeriodGranularity.java @@ -250,7 +250,7 @@ private long truncate(long t) long tt = chronology.years().add(origin, y); // always round down to the previous period (for timestamps prior to origin) if (t < tt) { - t = chronology.years().add(tt, -years); + t = chronology.years().add(origin, y - years); } else { t = tt; } @@ -268,7 +268,7 @@ private long truncate(long t) long tt = chronology.months().add(origin, m); // always round down to the previous period (for timestamps prior to origin) if (t < tt) { - t = chronology.months().add(tt, -months); + t = chronology.months().add(origin, m - months); } else { t = tt; } @@ -287,7 +287,7 @@ private long truncate(long t) long tt = chronology.weeks().add(origin, w); // always round down to the previous period (for timestamps prior to origin) if (t < tt) { - t = chronology.weeks().add(tt, -weeks); + t = chronology.weeks().add(origin, w - weeks); } else { t = tt; } @@ -308,7 +308,7 @@ private long truncate(long t) long tt = chronology.days().add(origin, d); // always round down to the previous period (for timestamps prior to origin) if (t < tt) { - t = chronology.days().add(tt, -days); + t = chronology.days().add(origin, d - days); } else { t = tt; } diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index a7af13ed76cf..77734edf0252 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -292,6 +292,77 @@ public Set getVirtualColumnCandidates() .collect(Collectors.toSet()); } + @Override + public Function createSegmentMapFunction( + Query query, + AtomicLong cpuTimeAccumulator + ) + { + return createSegmentMapFunctionInternal( + analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), + analysis.getPreJoinableClauses(), + cpuTimeAccumulator, + analysis.getBaseQuery().orElse(query) + ); + } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + DataSource current = newSource; + DimFilter joinBaseFilter = analysis.getJoinBaseTableFilter().orElse(null); + + for (final PreJoinableClause clause : analysis.getPreJoinableClauses()) { + current = JoinDataSource.create( + current, + clause.getDataSource(), + clause.getPrefix(), + clause.getCondition(), + clause.getJoinType(), + joinBaseFilter, + this.joinableFactoryWrapper + ); + joinBaseFilter = null; + } + return current; + } + + @Override + public byte[] getCacheKey() + { + final List clauses = analysis.getPreJoinableClauses(); + if (clauses.isEmpty()) { + throw new IAE("No join clauses to build the cache key for data source [%s]", this); + } + + final CacheKeyBuilder keyBuilder; + keyBuilder = new CacheKeyBuilder(JoinableFactoryWrapper.JOIN_OPERATION); + if (analysis.getJoinBaseTableFilter().isPresent()) { + keyBuilder.appendCacheable(analysis.getJoinBaseTableFilter().get()); + } + for (PreJoinableClause clause : clauses) { + final Optional bytes = + joinableFactoryWrapper.getJoinableFactory() + .computeJoinCacheKey(clause.getDataSource(), clause.getCondition()); + if (!bytes.isPresent()) { + // Encountered a data source which didn't support cache yet + log.debug("skipping caching for join since [%s] does not support caching", clause.getDataSource()); + return new byte[]{}; + } + keyBuilder.appendByteArray(bytes.get()); + keyBuilder.appendString(clause.getCondition().getOriginalExpression()); + keyBuilder.appendString(clause.getPrefix()); + keyBuilder.appendString(clause.getJoinType().name()); + } + return keyBuilder.build(); + } + + @Override + public DataSourceAnalysis getAnalysis() + { + return analysis; + } + @Override public boolean equals(Object o) { @@ -329,6 +400,11 @@ public String toString() '}'; } + private DataSourceAnalysis getAnalysisForDataSource() + { + final Triple> flattened = flattenJoin(this); + return new DataSourceAnalysis(flattened.first, null, flattened.second, flattened.third); + } /** * Creates a Function that maps base segments to {@link HashJoinSegment} if needed (i.e. if the number of join @@ -343,7 +419,7 @@ public String toString() * {@link DataSourceAnalysis} and "query" is the original * query from the end user. */ - public Function createSegmentMapFn( + private Function createSegmentMapFunctionInternal( @Nullable final Filter baseFilter, final List clauses, final AtomicLong cpuTimeAccumulator, @@ -413,85 +489,6 @@ public Function createSegmentMapFn( ); } - - @Override - public Function createSegmentMapFunction( - Query query, - AtomicLong cpuTimeAccumulator - ) - { - final Function segmentMapFn = createSegmentMapFn( - analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), - analysis.getPreJoinableClauses(), - cpuTimeAccumulator, - analysis.getBaseQuery().orElse(query) - ); - return segmentMapFn; - } - - @Override - public DataSource withUpdatedDataSource(DataSource newSource) - { - DataSource current = newSource; - DimFilter joinBaseFilter = analysis.getJoinBaseTableFilter().orElse(null); - - for (final PreJoinableClause clause : analysis.getPreJoinableClauses()) { - current = JoinDataSource.create( - current, - clause.getDataSource(), - clause.getPrefix(), - clause.getCondition(), - clause.getJoinType(), - joinBaseFilter, - this.joinableFactoryWrapper - ); - joinBaseFilter = null; - } - return current; - } - - @Override - public byte[] getCacheKey() - { - final List clauses = analysis.getPreJoinableClauses(); - if (clauses.isEmpty()) { - throw new IAE("No join clauses to build the cache key for data source [%s]", this); - } - - final CacheKeyBuilder keyBuilder; - keyBuilder = new CacheKeyBuilder(JoinableFactoryWrapper.JOIN_OPERATION); - if (analysis.getJoinBaseTableFilter().isPresent()) { - keyBuilder.appendCacheable(analysis.getJoinBaseTableFilter().get()); - } - for (PreJoinableClause clause : clauses) { - final Optional bytes = - joinableFactoryWrapper.getJoinableFactory() - .computeJoinCacheKey(clause.getDataSource(), clause.getCondition()); - if (!bytes.isPresent()) { - // Encountered a data source which didn't support cache yet - log.debug("skipping caching for join since [%s] does not support caching", clause.getDataSource()); - return new byte[]{}; - } - keyBuilder.appendByteArray(bytes.get()); - keyBuilder.appendString(clause.getCondition().getOriginalExpression()); - keyBuilder.appendString(clause.getPrefix()); - keyBuilder.appendString(clause.getJoinType().name()); - } - return keyBuilder.build(); - } - - private DataSourceAnalysis getAnalysisForDataSource() - { - final Triple> flattened = flattenJoin(this); - return new DataSourceAnalysis(flattened.first, null, flattened.second, flattened.third); - } - - @Override - public DataSourceAnalysis getAnalysis() - { - return analysis; - } - /** * Flatten a datasource into two parts: the left-hand side datasource (the 'base' datasource), and a list of join * clauses, if any. diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 4bb73d845ec4..912ecb1ac322 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -333,17 +333,26 @@ public static SegmentAnalysis mergeAnalyses( for (Map.Entry entry : analysis.getAggregators().entrySet()) { final String aggregatorName = entry.getKey(); final AggregatorFactory aggregator = entry.getValue(); - AggregatorFactory merged = aggregators.get(aggregatorName); - if (merged != null) { - try { - merged = merged.getMergingFactory(aggregator); - } - catch (AggregatorFactoryNotMergeableException e) { + final boolean isMergedYet = aggregators.containsKey(aggregatorName); + AggregatorFactory merged; + + if (!isMergedYet) { + merged = aggregator; + } else { + merged = aggregators.get(aggregatorName); + + if (merged != null && aggregator != null) { + try { + merged = merged.getMergingFactory(aggregator); + } + catch (AggregatorFactoryNotMergeableException e) { + merged = null; + } + } else { merged = null; } - } else { - merged = aggregator; } + aggregators.put(aggregatorName, merged); } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java index 97237ef77a2c..0eefa288df2f 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java @@ -70,7 +70,7 @@ public Column findColumn(String name) .getColumnType(columnIndex) .orElseThrow(() -> new ISE("just got the id, why is columnType not there?")); - colCache.put(name, FrameColumnReaders.create(columnIndex, columnType).readRACColumn(frame)); + colCache.put(name, FrameColumnReaders.create(name, columnIndex, columnType).readRACColumn(frame)); } } return colCache.get(name); diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java index 3eea9193cc37..49793fba4163 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java @@ -57,6 +57,8 @@ public static ColumnType ofType(TypeSignature type) switch (type.getElementType().getType()) { case LONG: return ColumnType.LONG_ARRAY; + case FLOAT: + return ColumnType.FLOAT_ARRAY; case DOUBLE: return ColumnType.DOUBLE_ARRAY; case STRING: diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java index a7de99905c39..3e2b46d5f681 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java @@ -298,7 +298,7 @@ private List createColumnReaders(RowSignature rowSignature) "Type for column [%s]", rowSignature.getColumnName(columnNumber) ); - columnReaders.add(FrameColumnReaders.create(columnNumber, columnType)); + columnReaders.add(FrameColumnReaders.create(rowSignature.getColumnName(columnNumber), columnNumber, columnType)); } return columnReaders; diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java index 0e6165848520..c7c569545e15 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java @@ -28,9 +28,9 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Comparators; -import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.DimensionHandler; -import org.apache.druid.segment.NestedCommonFormatColumnHandler; +import org.apache.druid.segment.NestedDataColumnHandlerV4; +import org.apache.druid.segment.NestedDataColumnSchema; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; @@ -168,13 +168,13 @@ public ColumnType getLogicalType() @Override public DimensionHandler getColumnHandler(String columnName) { - return new NestedCommonFormatColumnHandler(columnName); + return new NestedDataColumnHandlerV4(columnName); } @Override public DimensionSchema getColumnSchema(String columnName) { - return new AutoTypeColumnSchema(columnName); + return new NestedDataColumnSchema(columnName, 4); } @Override diff --git a/processing/src/test/java/org/apache/druid/common/UnitTest.java b/processing/src/test/java/org/apache/druid/common/UnitTest.java new file mode 100644 index 000000000000..1de704600e88 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/common/UnitTest.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.common; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.java.util.common.Unit; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class UnitTest +{ + @Test + public void testSerde() throws Exception + { + final ObjectMapper mapper = TestHelper.makeJsonMapper(); + Assert.assertSame( + Unit.instance(), + mapper.readValue(mapper.writeValueAsBytes(Unit.instance()), Unit.class) + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java new file mode 100644 index 000000000000..6381138f62db --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import com.google.common.collect.ImmutableList; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.quality.Strictness; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public class DoubleArrayFieldReaderTest extends InitializedNullHandlingTest +{ + private static final long MEMORY_POSITION = 1; + + @Rule + public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); + + @Mock + public ColumnValueSelector writeSelector; + + private WritableMemory memory; + private FieldWriter fieldWriter; + + //CHECKSTYLE.OFF: Regexp + private static final Object[] DOUBLES_ARRAY_1 = new Object[]{ + Double.MAX_VALUE, + Double.MIN_VALUE, + null, + Double.POSITIVE_INFINITY, + Double.NEGATIVE_INFINITY, + Double.MIN_NORMAL, + null, + 0.0d, + 1.234234d, + Double.NaN, + -1.344234d, + 129123.123123, + -21312213.33, + null, + 1111.0, + 23.0, + null, + }; + + private static final Object[] DOUBLES_ARRAY_2 = new Object[]{ + null, + Double.MAX_VALUE, + 12.234234d, + -21.344234d, + Double.POSITIVE_INFINITY, + null, + Double.MIN_VALUE, + 129123.123123, + null, + 0.0d, + Double.MIN_NORMAL, + 1111.0, + Double.NaN, + Double.NEGATIVE_INFINITY, + null, + -2133.33, + 23.0, + null + }; + //CHECKSTYLE.ON: Regexp + + private static final List DOUBLES_LIST_1; + private static final List DOUBLES_LIST_2; + + static { + DOUBLES_LIST_1 = Arrays.stream(DOUBLES_ARRAY_1).map(val -> (Double) val).collect(Collectors.toList()); + DOUBLES_LIST_2 = Arrays.stream(DOUBLES_ARRAY_2).map(val -> (Double) val).collect(Collectors.toList()); + } + + @Before + public void setUp() + { + memory = WritableMemory.allocate(1000); + fieldWriter = NumericArrayFieldWriter.getDoubleArrayFieldWriter(writeSelector); + } + + @After + public void tearDown() + { + fieldWriter.close(); + } + + @Test + public void test_isNull_null() + { + writeToMemory(null, MEMORY_POSITION); + Assert.assertTrue(new DoubleArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_aValue() + { + writeToMemory(DOUBLES_ARRAY_1, MEMORY_POSITION); + Assert.assertFalse(new DoubleArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_emptyArray() + { + writeToMemory(new Object[]{}, MEMORY_POSITION); + Assert.assertFalse(new DoubleArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_arrayWithSingleNullElement() + { + writeToMemory(new Object[]{null}, MEMORY_POSITION); + Assert.assertFalse(new DoubleArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_makeColumnValueSelector_null() + { + writeToMemory(null, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + Assert.assertTrue(readSelector.isNull()); + } + + @Test + public void test_makeColumnValueSelector_aValue() + { + writeToMemory(DOUBLES_ARRAY_1, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(DOUBLES_LIST_1, readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_multipleValues() + { + long sz = writeToMemory(DOUBLES_ARRAY_1, MEMORY_POSITION); + writeToMemory(DOUBLES_ARRAY_2, MEMORY_POSITION + sz); + IndexArrayFieldPointer pointer = new IndexArrayFieldPointer(ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz)); + + + final ColumnValueSelector readSelector = new DoubleArrayFieldReader().makeColumnValueSelector(memory, pointer); + + pointer.setPointer(0); + assertResults(DOUBLES_LIST_1, readSelector.getObject()); + + pointer.setPointer(1); + assertResults(DOUBLES_LIST_2, readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_emptyArray() + { + writeToMemory(new Object[]{}, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(Collections.emptyList(), readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_arrayWithSingleNullElement() + { + writeToMemory(new Object[]{null}, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(Collections.singletonList(null), readSelector.getObject()); + } + + private long writeToMemory(final Object value, final long initialPosition) + { + Mockito.when(writeSelector.getObject()).thenReturn(value); + + long bytesWritten = fieldWriter.writeTo(memory, initialPosition, memory.getCapacity() - initialPosition); + if (bytesWritten < 0) { + throw new ISE("Could not write"); + } + return bytesWritten; + } + + private void assertResults(List expected, Object actual) + { + if (expected == null) { + Assert.assertNull(actual); + } + Assert.assertTrue(actual instanceof Object[]); + List actualList = new ArrayList<>(); + for (Object val : (Object[]) actual) { + actualList.add((Double) val); + } + + Assert.assertEquals(expected, actualList); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java index 2393a6a7c3c9..8736c73276cc 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java @@ -59,7 +59,7 @@ public class DoubleFieldReaderTest extends InitializedNullHandlingTest public void setUp() { memory = WritableMemory.allocate(1000); - fieldWriter = new DoubleFieldWriter(writeSelector); + fieldWriter = DoubleFieldWriter.forPrimitive(writeSelector); } @After @@ -72,14 +72,14 @@ public void tearDown() public void test_isNull_defaultOrNull() { writeToMemory(NullHandling.defaultDoubleValue()); - Assert.assertEquals(NullHandling.sqlCompatible(), new DoubleFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertEquals(NullHandling.sqlCompatible(), DoubleFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_aValue() { writeToMemory(5.1d); - Assert.assertFalse(new DoubleFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(DoubleFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION)); } @Test @@ -88,7 +88,7 @@ public void test_makeColumnValueSelector_defaultOrNull() writeToMemory(NullHandling.defaultDoubleValue()); final ColumnValueSelector readSelector = - new DoubleFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + DoubleFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(!NullHandling.replaceWithDefault(), readSelector.isNull()); @@ -103,7 +103,7 @@ public void test_makeColumnValueSelector_aValue() writeToMemory(5.1d); final ColumnValueSelector readSelector = - new DoubleFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + DoubleFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(5.1d, readSelector.getObject()); } @@ -114,7 +114,7 @@ public void test_makeDimensionSelector_defaultOrNull() writeToMemory(NullHandling.defaultDoubleValue()); final DimensionSelector readSelector = - new DoubleFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + DoubleFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -148,7 +148,7 @@ public void test_makeDimensionSelector_aValue() writeToMemory(5.1d); final DimensionSelector readSelector = - new DoubleFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + DoubleFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -175,7 +175,7 @@ public void test_makeDimensionSelector_aValue_extractionFn() writeToMemory(20.5d); final DimensionSelector readSelector = - new DoubleFieldReader().makeDimensionSelector( + DoubleFieldReader.forPrimitive().makeDimensionSelector( memory, new ConstantFieldPointer(MEMORY_POSITION), new SubstringDimExtractionFn(1, null) diff --git a/processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java new file mode 100644 index 000000000000..e61e40db1cb1 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import com.google.common.collect.ImmutableList; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.quality.Strictness; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public class FloatArrayFieldReaderTest extends InitializedNullHandlingTest +{ + private static final long MEMORY_POSITION = 1; + + @Rule + public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); + + @Mock + public ColumnValueSelector writeSelector; + + private WritableMemory memory; + private FieldWriter fieldWriter; + + //CHECKSTYLE.OFF: Regexp + private static final Object[] FLOATS_ARRAY_1 = new Object[]{ + null, + Float.MIN_VALUE, + Float.MAX_VALUE, + Float.MIN_NORMAL, + null, + Float.POSITIVE_INFINITY, + Float.NEGATIVE_INFINITY, + null, + 0.0f, + 1.234234f, + -1.344234f, + Float.NaN, + 129123.123123f, + null, + -21312213.33f, + 1111.0f, + null, + 23.0f + }; + + private static final Object[] FLOATS_ARRAY_2 = new Object[]{ + null, + Float.MAX_VALUE, + null, + Float.POSITIVE_INFINITY, + -0.0f, + 0.0f, + -1.234234f, + Float.MIN_VALUE, + 1.344234333f, + -129123.123123f, + 21312213.33f, + Float.NEGATIVE_INFINITY, + -1111.0f, + 1223.0f, + Float.MIN_NORMAL, + null, + Float.NaN, + null + }; + //CHECKSTYLE.ON: Regexp + + private static final List FLOATS_LIST_1; + private static final List FLOATS_LIST_2; + + static { + FLOATS_LIST_1 = Arrays.stream(FLOATS_ARRAY_1).map(val -> (Float) val).collect(Collectors.toList()); + FLOATS_LIST_2 = Arrays.stream(FLOATS_ARRAY_2).map(val -> (Float) val).collect(Collectors.toList()); + } + + @Before + public void setUp() + { + memory = WritableMemory.allocate(1000); + fieldWriter = NumericArrayFieldWriter.getFloatArrayFieldWriter(writeSelector); + } + + @After + public void tearDown() + { + fieldWriter.close(); + } + + @Test + public void test_isNull_null() + { + writeToMemory(null, MEMORY_POSITION); + Assert.assertTrue(new FloatArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_aValue() + { + writeToMemory(FLOATS_ARRAY_1, MEMORY_POSITION); + Assert.assertFalse(new FloatArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_emptyArray() + { + writeToMemory(new Object[]{}, MEMORY_POSITION); + Assert.assertFalse(new FloatArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_arrayWithSingleNullElement() + { + writeToMemory(new Object[]{null}, MEMORY_POSITION); + Assert.assertFalse(new FloatArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_makeColumnValueSelector_null() + { + writeToMemory(null, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + Assert.assertTrue(readSelector.isNull()); + } + + @Test + public void test_makeColumnValueSelector_aValue() + { + writeToMemory(FLOATS_ARRAY_1, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(FLOATS_LIST_1, readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_multipleValues() + { + long sz = writeToMemory(FLOATS_ARRAY_1, MEMORY_POSITION); + writeToMemory(FLOATS_ARRAY_2, MEMORY_POSITION + sz); + IndexArrayFieldPointer pointer = new IndexArrayFieldPointer(ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz)); + + + final ColumnValueSelector readSelector = new FloatArrayFieldReader().makeColumnValueSelector(memory, pointer); + + pointer.setPointer(0); + assertResults(FLOATS_LIST_1, readSelector.getObject()); + + pointer.setPointer(1); + assertResults(FLOATS_LIST_2, readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_emptyArray() + { + writeToMemory(new Object[]{}, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(Collections.emptyList(), readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_arrayWithSingleNullElement() + { + writeToMemory(new Object[]{null}, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(Collections.singletonList(null), readSelector.getObject()); + } + + private long writeToMemory(final Object value, final long initialPosition) + { + Mockito.when(writeSelector.getObject()).thenReturn(value); + + long bytesWritten = fieldWriter.writeTo(memory, initialPosition, memory.getCapacity() - initialPosition); + if (bytesWritten < 0) { + throw new ISE("Could not write"); + } + return bytesWritten; + } + + private void assertResults(List expected, Object actual) + { + if (expected == null) { + Assert.assertNull(actual); + } + Assert.assertTrue(actual instanceof Object[]); + List actualList = new ArrayList<>(); + for (Object val : (Object[]) actual) { + actualList.add((Float) val); + } + + Assert.assertEquals(expected, actualList); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java index e85225b99877..441858f0fd67 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java @@ -59,7 +59,7 @@ public class FloatFieldReaderTest extends InitializedNullHandlingTest public void setUp() { memory = WritableMemory.allocate(1000); - fieldWriter = new FloatFieldWriter(writeSelector); + fieldWriter = FloatFieldWriter.forPrimitive(writeSelector); } @After @@ -72,14 +72,14 @@ public void tearDown() public void test_isNull_defaultOrNull() { writeToMemory(NullHandling.defaultFloatValue()); - Assert.assertEquals(NullHandling.sqlCompatible(), new FloatFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertEquals(NullHandling.sqlCompatible(), FloatFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_aValue() { writeToMemory(5.1f); - Assert.assertFalse(new FloatFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(FloatFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION)); } @Test @@ -88,7 +88,7 @@ public void test_makeColumnValueSelector_defaultOrNull() writeToMemory(NullHandling.defaultFloatValue()); final ColumnValueSelector readSelector = - new FloatFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + FloatFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(!NullHandling.replaceWithDefault(), readSelector.isNull()); @@ -103,7 +103,7 @@ public void test_makeColumnValueSelector_aValue() writeToMemory(5.1f); final ColumnValueSelector readSelector = - new FloatFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + FloatFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(5.1f, readSelector.getObject()); } @@ -114,7 +114,7 @@ public void test_makeDimensionSelector_defaultOrNull() writeToMemory(NullHandling.defaultFloatValue()); final DimensionSelector readSelector = - new FloatFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + FloatFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -148,7 +148,7 @@ public void test_makeDimensionSelector_aValue() writeToMemory(5.1f); final DimensionSelector readSelector = - new FloatFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + FloatFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -175,7 +175,7 @@ public void test_makeDimensionSelector_aValue_extractionFn() writeToMemory(20.5f); final DimensionSelector readSelector = - new FloatFieldReader().makeDimensionSelector( + FloatFieldReader.forPrimitive().makeDimensionSelector( memory, new ConstantFieldPointer(MEMORY_POSITION), new SubstringDimExtractionFn(1, null) diff --git a/processing/src/test/java/org/apache/druid/frame/field/IndexArrayFieldPointer.java b/processing/src/test/java/org/apache/druid/frame/field/IndexArrayFieldPointer.java new file mode 100644 index 000000000000..1e115f48e3c5 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/field/IndexArrayFieldPointer.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import it.unimi.dsi.fastutil.longs.LongArrayList; + +import java.util.List; + +/** + * Stores the memory locations in an array, and spits out the value pointed to by the memory location by pointer, + * which is settable by the user + */ +public class IndexArrayFieldPointer implements ReadableFieldPointer +{ + private final LongArrayList indices; + private int pointer = 0; + + public IndexArrayFieldPointer(final List indices) + { + this.indices = new LongArrayList(indices); + } + + private int numIndices() + { + return indices.size(); + } + + public void setPointer(int newPointer) + { + assert newPointer >= 0 && newPointer < numIndices(); + this.pointer = newPointer; + } + + @Override + public long position() + { + return indices.getLong(pointer); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/field/LongArrayFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/LongArrayFieldReaderTest.java new file mode 100644 index 000000000000..aa34cd6afaf3 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/field/LongArrayFieldReaderTest.java @@ -0,0 +1,211 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import com.google.common.collect.ImmutableList; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.quality.Strictness; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public class LongArrayFieldReaderTest extends InitializedNullHandlingTest +{ + private static final long MEMORY_POSITION = 1; + + @Rule + public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); + + @Mock + public ColumnValueSelector writeSelector; + + private WritableMemory memory; + private FieldWriter fieldWriter; + + private static final Object[] LONGS_ARRAY_1 = new Object[]{ + Long.MIN_VALUE, + Long.MAX_VALUE, + null, + 0L, + 123L, + -123L + }; + + private static final Object[] LONGS_ARRAY_2 = new Object[]{ + null, + 234L, + Long.MAX_VALUE, + null, + Long.MIN_VALUE, + 0L, + -234L, + null + }; + + private static final List LONGS_LIST_1; + private static final List LONGS_LIST_2; + + static { + LONGS_LIST_1 = Arrays.stream(LONGS_ARRAY_1).map(val -> (Long) val).collect(Collectors.toList()); + LONGS_LIST_2 = Arrays.stream(LONGS_ARRAY_2).map(val -> (Long) val).collect(Collectors.toList()); + } + + @Before + public void setUp() + { + memory = WritableMemory.allocate(1000); + fieldWriter = NumericArrayFieldWriter.getLongArrayFieldWriter(writeSelector); + } + + @After + public void tearDown() + { + fieldWriter.close(); + } + + @Test + public void test_isNull_null() + { + writeToMemory(null, MEMORY_POSITION); + Assert.assertTrue(new LongArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_aValue() + { + writeToMemory(LONGS_ARRAY_1, MEMORY_POSITION); + Assert.assertFalse(new LongArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_emptyArray() + { + writeToMemory(new Object[]{}, MEMORY_POSITION); + Assert.assertFalse(new LongArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_arrayWithSingleNullElement() + { + writeToMemory(new Object[]{null}, MEMORY_POSITION); + Assert.assertFalse(new LongArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_makeColumnValueSelector_null() + { + writeToMemory(null, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + Assert.assertTrue(readSelector.isNull()); + } + + @Test + public void test_makeColumnValueSelector_aValue() + { + writeToMemory(LONGS_ARRAY_1, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(LONGS_LIST_1, readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_multipleValues() + { + long sz = writeToMemory(LONGS_ARRAY_1, MEMORY_POSITION); + writeToMemory(LONGS_ARRAY_2, MEMORY_POSITION + sz); + IndexArrayFieldPointer pointer = new IndexArrayFieldPointer(ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz)); + + + final ColumnValueSelector readSelector = new LongArrayFieldReader().makeColumnValueSelector(memory, pointer); + + pointer.setPointer(0); + assertResults(LONGS_LIST_1, readSelector.getObject()); + + pointer.setPointer(1); + assertResults(LONGS_LIST_2, readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_emptyArray() + { + writeToMemory(new Object[]{}, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(Collections.emptyList(), readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_arrayWithSingleNullElement() + { + writeToMemory(new Object[]{null}, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(Collections.singletonList(null), readSelector.getObject()); + } + + private long writeToMemory(final Object value, final long initialPosition) + { + Mockito.when(writeSelector.getObject()).thenReturn(value); + + long bytesWritten = fieldWriter.writeTo(memory, initialPosition, memory.getCapacity() - initialPosition); + if (bytesWritten < 0) { + throw new ISE("Could not write"); + } + return bytesWritten; + } + + private void assertResults(List expected, Object actual) + { + if (expected == null) { + Assert.assertNull(actual); + } + Assert.assertTrue(actual instanceof Object[]); + List actualList = new ArrayList<>(); + for (Object val : (Object[]) actual) { + actualList.add((Long) val); + } + + Assert.assertEquals(expected, actualList); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java index 06e6c42d9e70..7e73a7cfdf03 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java @@ -59,7 +59,7 @@ public class LongFieldReaderTest extends InitializedNullHandlingTest public void setUp() { memory = WritableMemory.allocate(1000); - fieldWriter = new LongFieldWriter(writeSelector); + fieldWriter = LongFieldWriter.forPrimitive(writeSelector); } @After @@ -72,14 +72,14 @@ public void tearDown() public void test_isNull_defaultOrNull() { writeToMemory(NullHandling.defaultLongValue()); - Assert.assertEquals(NullHandling.sqlCompatible(), new LongFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertEquals(NullHandling.sqlCompatible(), LongFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_aValue() { writeToMemory(5L); - Assert.assertFalse(new LongFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(LongFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION)); } @Test @@ -88,7 +88,7 @@ public void test_makeColumnValueSelector_defaultOrNull() writeToMemory(NullHandling.defaultLongValue()); final ColumnValueSelector readSelector = - new LongFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + LongFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(!NullHandling.replaceWithDefault(), readSelector.isNull()); @@ -103,7 +103,7 @@ public void test_makeColumnValueSelector_aValue() writeToMemory(5L); final ColumnValueSelector readSelector = - new LongFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + LongFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(5L, readSelector.getObject()); } @@ -114,7 +114,7 @@ public void test_makeDimensionSelector_defaultOrNull() writeToMemory(NullHandling.defaultLongValue()); final DimensionSelector readSelector = - new LongFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + LongFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -148,7 +148,7 @@ public void test_makeDimensionSelector_aValue() writeToMemory(5L); final DimensionSelector readSelector = - new LongFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + LongFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -175,7 +175,7 @@ public void test_makeDimensionSelector_aValue_extractionFn() writeToMemory(25L); final DimensionSelector readSelector = - new LongFieldReader().makeDimensionSelector( + LongFieldReader.forPrimitive().makeDimensionSelector( memory, new ConstantFieldPointer(MEMORY_POSITION), new SubstringDimExtractionFn(1, null) diff --git a/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java index 97370187522e..bde3f77480ee 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java @@ -81,24 +81,24 @@ public void tearDown() public void test_isNull_nullValue() { writeToMemory(Collections.singletonList(null)); - Assert.assertTrue(new StringFieldReader(false).isNull(memory, MEMORY_POSITION)); - Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertTrue(new StringFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_twoNullValues() { writeToMemory(Arrays.asList(null, null)); - Assert.assertFalse(new StringFieldReader(false).isNull(memory, MEMORY_POSITION)); - Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_nullRow() { writeToMemory(null); - Assert.assertTrue(new StringFieldReader(false).isNull(memory, MEMORY_POSITION)); - Assert.assertTrue(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertTrue(new StringFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertTrue(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test @@ -107,33 +107,33 @@ public void test_isNull_emptyString() writeToMemory(Collections.singletonList("")); Assert.assertEquals( NullHandling.replaceWithDefault(), - new StringFieldReader(false).isNull(memory, MEMORY_POSITION) + new StringFieldReader().isNull(memory, MEMORY_POSITION) ); - Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_aValue() { writeToMemory(Collections.singletonList("foo")); - Assert.assertFalse(new StringFieldReader(false).isNull(memory, MEMORY_POSITION)); - Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_multiString() { writeToMemory(ImmutableList.of("foo", "bar")); - Assert.assertFalse(new StringFieldReader(false).isNull(memory, MEMORY_POSITION)); - Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_multiStringIncludingNulls() { writeToMemory(Arrays.asList(null, "bar")); - Assert.assertFalse(new StringFieldReader(false).isNull(memory, MEMORY_POSITION)); - Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test @@ -142,9 +142,9 @@ public void test_makeColumnValueSelector_singleString_notArray() writeToMemory(Collections.singletonList("foo")); final ColumnValueSelector readSelector = - new StringFieldReader(false).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); final ColumnValueSelector readSelectorAsArray = - new StringFieldReader(true).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals("foo", readSelector.getObject()); Assert.assertArrayEquals(new Object[]{"foo"}, (Object[]) readSelectorAsArray.getObject()); @@ -156,9 +156,9 @@ public void test_makeColumnValueSelector_multiString() writeToMemory(ImmutableList.of("foo", "bar")); final ColumnValueSelector readSelector = - new StringFieldReader(false).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); final ColumnValueSelector readSelectorAsArray = - new StringFieldReader(true).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(ImmutableList.of("foo", "bar"), readSelector.getObject()); Assert.assertArrayEquals(new Object[]{"foo", "bar"}, (Object[]) readSelectorAsArray.getObject()); @@ -170,9 +170,9 @@ public void test_makeColumnValueSelector_null() writeToMemory(Collections.singletonList(null)); final ColumnValueSelector readSelector = - new StringFieldReader(false).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); final ColumnValueSelector readSelectorAsArray = - new StringFieldReader(true).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertNull(readSelector.getObject()); Assert.assertArrayEquals(new Object[]{null}, (Object[]) readSelectorAsArray.getObject()); @@ -184,9 +184,9 @@ public void test_makeColumnValueSelector_empty() writeToMemory(Collections.emptyList()); final ColumnValueSelector readSelector = - new StringFieldReader(false).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); final ColumnValueSelector readSelectorAsArray = - new StringFieldReader(true).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertNull(readSelector.getObject()); Assert.assertArrayEquals(ObjectArrays.EMPTY_ARRAY, (Object[]) readSelectorAsArray.getObject()); @@ -199,7 +199,7 @@ public void test_makeDimensionSelector_multiString_asArray() final IllegalStateException e = Assert.assertThrows( IllegalStateException.class, - () -> new StringFieldReader(true).makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null) + () -> new StringArrayFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null) ); MatcherAssert.assertThat( @@ -214,7 +214,7 @@ public void test_makeDimensionSelector_multiString() writeToMemory(ImmutableList.of("foo", "bar")); final DimensionSelector readSelector = - new StringFieldReader(false).makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + new StringFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -244,7 +244,7 @@ public void test_makeDimensionSelector_multiString_withExtractionFn() writeToMemory(ImmutableList.of("foo", "bar")); final DimensionSelector readSelector = - new StringFieldReader(false).makeDimensionSelector( + new StringFieldReader().makeDimensionSelector( memory, new ConstantFieldPointer(MEMORY_POSITION), new SubstringDimExtractionFn(1, null) diff --git a/processing/src/test/java/org/apache/druid/frame/field/TransformUtilsTest.java b/processing/src/test/java/org/apache/druid/frame/field/TransformUtilsTest.java new file mode 100644 index 000000000000..276e598ad367 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/field/TransformUtilsTest.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.field; + +import com.google.common.collect.ImmutableList; +import org.apache.datasketches.memory.WritableMemory; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class TransformUtilsTest +{ + + private final WritableMemory lhsMemory = WritableMemory.allocate(10); + private final WritableMemory rhsMemory = WritableMemory.allocate(10); + + private static final long MEMORY_LOCATION = 0; + + @Test + public void doubleTestWithoutNaN() + { + //CHECKSTYLE.OFF: Regexp + List values = + ImmutableList.of( + Double.MAX_VALUE, + Double.MIN_VALUE, + Double.POSITIVE_INFINITY, + Double.NEGATIVE_INFINITY, + Double.MIN_NORMAL, + 0.0d, + 1.234234d, + -1.344234d, + 129123.123123, + -21312213.33, + 1111.0, + 23.0 + ); + //CHECKSTYLE.ON: Regexp + + for (double value : values) { + Assert.assertEquals( + value, + TransformUtils.detransformToDouble(TransformUtils.transformFromDouble(value)), + 0.0 + ); + + } + + for (int lhsIndex = 0; lhsIndex < values.size(); ++lhsIndex) { + for (int rhsIndex = lhsIndex; rhsIndex < values.size(); ++rhsIndex) { + double lhs = values.get(lhsIndex); + double rhs = values.get(rhsIndex); + lhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromDouble(lhs)); + rhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromDouble(rhs)); + int byteCmp = byteComparison(Double.BYTES); + + if (byteCmp < 0) { + Assert.assertTrue(lhs < rhs); + } else if (byteCmp == 0) { + Assert.assertEquals(lhs, rhs, 0.0); + } else { + Assert.assertTrue(lhs > rhs); + } + + } + } + } + + @Test + public void longTest() + { + List values = + ImmutableList.of( + Long.MIN_VALUE, + Long.MAX_VALUE, + 0L, + 123L, + -123L + ); + + for (long value : values) { + Assert.assertEquals( + value, + TransformUtils.detransformToLong(TransformUtils.transformFromLong(value)) + ); + + } + + for (int lhsIndex = 0; lhsIndex < values.size(); ++lhsIndex) { + for (int rhsIndex = lhsIndex; rhsIndex < values.size(); ++rhsIndex) { + long lhs = values.get(lhsIndex); + long rhs = values.get(rhsIndex); + lhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromLong(lhs)); + rhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromLong(rhs)); + int byteCmp = byteComparison(Long.BYTES); + + if (byteCmp < 0) { + Assert.assertTrue(lhs < rhs); + } else if (byteCmp == 0) { + Assert.assertEquals(lhs, rhs); + } else { + Assert.assertTrue(lhs > rhs); + } + } + } + } + + @Test + public void floatTestWithoutNaN() + { + //CHECKSTYLE.OFF: Regexp + List values = + ImmutableList.of( + Float.MIN_VALUE, + Float.MAX_VALUE, + Float.MIN_NORMAL, + Float.POSITIVE_INFINITY, + Float.NEGATIVE_INFINITY, + 0.0f, + 1.234234f, + -1.344234f, + 129123.123123f, + -21312213.33f, + 1111.0f, + 23.0f + ); + //CHECKSTYLE.ON: Regexp + + for (float value : values) { + Assert.assertEquals( + value, + TransformUtils.detransformToFloat(TransformUtils.transformFromFloat(value)), + 0.0 + ); + + } + + for (int lhsIndex = 0; lhsIndex < values.size(); ++lhsIndex) { + for (int rhsIndex = lhsIndex; rhsIndex < values.size(); ++rhsIndex) { + float lhs = values.get(lhsIndex); + float rhs = values.get(rhsIndex); + lhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromFloat(lhs)); + rhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromFloat(rhs)); + int byteCmp = byteComparison(Long.BYTES); + + if (byteCmp < 0) { + Assert.assertTrue(lhs < rhs); + } else if (byteCmp == 0) { + Assert.assertEquals(lhs, rhs, 0.0); + } else { + Assert.assertTrue(lhs > rhs); + } + } + } + } + + private int byteComparison(int numBytes) + { + for (int i = 0; i < numBytes; ++i) { + byte lhsByte = lhsMemory.getByte(MEMORY_LOCATION + i); + byte rhsByte = rhsMemory.getByte(MEMORY_LOCATION + i); + final int cmp = (lhsByte & 0xFF) - (rhsByte & 0xFF); + if (cmp != 0) { + return cmp; + } + } + return 0; + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/processor/RunAllFullyWidgetTest.java b/processing/src/test/java/org/apache/druid/frame/processor/RunAllFullyWidgetTest.java index cd13bcd00d9e..b82aed94ef81 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/RunAllFullyWidgetTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/RunAllFullyWidgetTest.java @@ -20,7 +20,10 @@ package org.apache.druid.frame.processor; import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; import com.google.errorprone.annotations.concurrent.GuardedBy; import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.druid.frame.Frame; @@ -29,12 +32,14 @@ import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.ReadableNilFrameChannel; import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.processor.manager.ProcessorAndCallback; +import org.apache.druid.frame.processor.manager.ProcessorManager; +import org.apache.druid.frame.processor.manager.ProcessorManagers; import org.apache.druid.frame.processor.test.ChompingFrameProcessor; import org.apache.druid.frame.processor.test.FailingFrameProcessor; import org.apache.druid.frame.processor.test.SleepyFrameProcessor; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.TestIndex; @@ -54,8 +59,10 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -64,6 +71,8 @@ public class RunAllFullyWidgetTest extends FrameProcessorExecutorTest.BaseFrameP { private final int bouncerPoolSize; private final int maxOutstandingProcessors; + private final boolean delayed; + private final AtomicLong closed = new AtomicLong(); private Bouncer bouncer; @@ -73,14 +82,16 @@ public class RunAllFullyWidgetTest extends FrameProcessorExecutorTest.BaseFrameP @GuardedBy("this") private int concurrentNow = 0; - public RunAllFullyWidgetTest(int numThreads, int bouncerPoolSize, int maxOutstandingProcessors) + public RunAllFullyWidgetTest(int numThreads, int bouncerPoolSize, int maxOutstandingProcessors, boolean delayed) { super(numThreads); this.bouncerPoolSize = bouncerPoolSize; this.maxOutstandingProcessors = maxOutstandingProcessors; + this.delayed = delayed; } - @Parameterized.Parameters(name = "numThreads = {0}, bouncerPoolSize = {1}, maxOutstandingProcessors = {2}") + @Parameterized.Parameters(name = + "numThreads = {0}, bouncerPoolSize = {1}, maxOutstandingProcessors = {2}, delayed = {3}") public static Collection constructorFeeder() { final List constructors = new ArrayList<>(); @@ -88,7 +99,9 @@ public static Collection constructorFeeder() for (int numThreads : new int[]{1, 3, 12}) { for (int bouncerPoolSize : new int[]{1, 3, 12, Integer.MAX_VALUE}) { for (int maxOutstandingProcessors : new int[]{1, 3, 12}) { - constructors.add(new Object[]{numThreads, bouncerPoolSize, maxOutstandingProcessors}); + for (boolean delayed : new boolean[]{false, true}) { + constructors.add(new Object[]{numThreads, bouncerPoolSize, maxOutstandingProcessors, delayed}); + } } } } @@ -121,17 +134,16 @@ public void tearDown() throws Exception MatcherAssert.assertThat(concurrentHighWatermark, Matchers.lessThanOrEqualTo(maxOutstandingProcessors)); } - Assert.assertEquals(0, bouncer.getCurrentCount()); - Assert.assertEquals(bouncerPoolSize, bouncer.getMaxCount()); + Assert.assertEquals("Bouncer current running count", 0, bouncer.getCurrentCount()); + Assert.assertEquals("Bouncer max pool size", bouncerPoolSize, bouncer.getMaxCount()); + Assert.assertEquals("Encountered single close (from ensureClose)", 1, closed.get()); } @Test - public void test_runAllFully_emptySequence() throws Exception + public void test_runAllFully_emptyChannel() throws Exception { - final ListenableFuture future = exec.runAllFully( - Sequences.empty(), - "xyzzy", - (s1, s2) -> s1 + s2, + final ListenableFuture future = exec.runAllFully( + possiblyDelay(ensureClose(ProcessorManagers.none().withAccumulation("xyzzy", (s1, s2) -> s1 + s2))), maxOutstandingProcessors, bouncer, null @@ -143,7 +155,7 @@ public void test_runAllFully_emptySequence() throws Exception @Test public void test_runAllFully_fiftyThousandProcessors() throws Exception { - final int numProcessors = 50_000; + final int numProcessors = 100; // Doesn't matter what's in this frame. final Frame frame = @@ -154,33 +166,29 @@ public void test_runAllFully_fiftyThousandProcessors() throws Exception .toList() ); - final Sequence> processors = Sequences.simple( - () -> - IntStream.range(0, numProcessors) - .mapToObj( - i -> { - final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal(); - - try { - channel.writable().write(frame); - channel.writable().close(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - - return new ConcurrencyTrackingFrameProcessor<>( - new ChompingFrameProcessor(Collections.singletonList(channel.readable())) - ); - } - ) - .iterator() - ); + final ProcessorManager processors = ProcessorManagers.of( + Iterables.transform( + IntStream.range(0, numProcessors)::iterator, + i -> { + final BlockingQueueFrameChannel channel = BlockingQueueFrameChannel.minimal(); + + try { + channel.writable().write(frame); + channel.writable().close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + + return new ConcurrencyTrackingFrameProcessor<>( + new ChompingFrameProcessor(Collections.singletonList(channel.readable())) + ); + } + ) + ).withAccumulation(0L, Long::sum); final ListenableFuture future = exec.runAllFully( - processors, - 0L, - Long::sum, + possiblyDelay(ensureClose(processors)), maxOutstandingProcessors, bouncer, null @@ -193,22 +201,24 @@ public void test_runAllFully_fiftyThousandProcessors() throws Exception public void test_runAllFully_failing() { final ListenableFuture future = exec.runAllFully( - Sequences.simple( - () -> IntStream.generate(() -> 0) // Infinite stream - .mapToObj( - i -> - new ConcurrencyTrackingFrameProcessor<>( - new FailingFrameProcessor( - ReadableNilFrameChannel.INSTANCE, - BlockingQueueFrameChannel.minimal().writable(), - 0 - ) - ) - ) - .iterator() + possiblyDelay( + ensureClose( + ProcessorManagers.of( + Iterables.transform( + IntStream.generate(() -> 0)::iterator /* Infinite stream */, + i -> + new ConcurrencyTrackingFrameProcessor<>( + new FailingFrameProcessor( + ReadableNilFrameChannel.INSTANCE, + BlockingQueueFrameChannel.minimal().writable(), + 0 + ) + ) + + ) + ).withAccumulation(0L, Long::sum) + ) ), - 0L, - Long::sum, maxOutstandingProcessors, bouncer, null @@ -227,15 +237,48 @@ public void test_runAllFully_failing() public void test_runAllFully_errorAccumulateFn() { final ListenableFuture future = exec.runAllFully( - Sequences.simple( - () -> IntStream.range(0, 100) - .mapToObj(i -> new ChompingFrameProcessor(Collections.emptyList())) - .iterator() + possiblyDelay( + ensureClose( + ProcessorManagers.of( + Iterables.transform( + IntStream.range(0, 100)::iterator, + i -> new ChompingFrameProcessor(Collections.emptyList()) + ) + ).withAccumulation( + 0L, + (x, y) -> { + throw new ISE("error!"); + } + ) + ) + ), + maxOutstandingProcessors, + bouncer, + null + ); + + final ExecutionException e = Assert.assertThrows(ExecutionException.class, future::get); + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(IllegalStateException.class)); + MatcherAssert.assertThat(e.getCause(), ThrowableMessageMatcher.hasMessage(CoreMatchers.equalTo("error!"))); + } + + @Test + public void test_runAllFully_errorChannelFirstElement() + { + final ListenableFuture future = exec.runAllFully( + possiblyDelay( + new ThrowOnNextProcessorManager<>( + ensureClose( + ProcessorManagers.of( + Iterables.transform( + IntStream.generate(() -> 0)::iterator /* Infinite stream */, + i -> new ChompingFrameProcessor(Collections.emptyList()) + ) + ).withAccumulation(0L, Long::sum) + ), + 0 + ) ), - 0L, - (x, y) -> { - throw new ISE("error!"); - }, maxOutstandingProcessors, bouncer, null @@ -247,20 +290,22 @@ public void test_runAllFully_errorAccumulateFn() } @Test - public void test_runAllFully_errorSequenceFirstElement() + public void test_runAllFully_errorChannelSecondElement() { final ListenableFuture future = exec.runAllFully( - Sequences.simple( - () -> IntStream.generate(() -> 0) // Infinite stream - .>mapToObj( - i -> { - throw new ISE("error!"); - } - ) - .iterator() + possiblyDelay( + new ThrowOnNextProcessorManager<>( + ensureClose( + ProcessorManagers.of( + Iterables.transform( + IntStream.generate(() -> 0)::iterator /* Infinite stream */, + i -> new ChompingFrameProcessor(Collections.emptyList()) + ) + ).withAccumulation(0L, Long::sum) + ), + 1 + ) ), - 0L, - Long::sum, maxOutstandingProcessors, bouncer, null @@ -272,24 +317,22 @@ public void test_runAllFully_errorSequenceFirstElement() } @Test - public void test_runAllFully_errorSequenceSecondElement() + public void test_runAllFully_errorChannelHundredthElement() { final ListenableFuture future = exec.runAllFully( - Sequences.simple( - () -> IntStream.range(0, 101) - .>mapToObj( - i -> { - if (i != 2) { - return new ChompingFrameProcessor(Collections.emptyList()); - } else { - throw new ISE("error!"); - } - } - ) - .iterator() + possiblyDelay( + new ThrowOnNextProcessorManager<>( + ensureClose( + ProcessorManagers.of( + Iterables.transform( + IntStream.generate(() -> 0)::iterator /* Infinite stream */, + i -> new ChompingFrameProcessor(Collections.emptyList()) + ) + ).withAccumulation(0L, Long::sum) + ), + 100 + ) ), - 0L, - Long::sum, maxOutstandingProcessors, bouncer, null @@ -301,24 +344,50 @@ public void test_runAllFully_errorSequenceSecondElement() } @Test - public void test_runAllFully_errorSequenceHundredthElement() + public void test_runAllFully_errorChannelClose() { final ListenableFuture future = exec.runAllFully( - Sequences.simple( - () -> IntStream.range(0, 101) - .mapToObj( - i -> { - if (i != 100) { - return new ChompingFrameProcessor(Collections.emptyList()); - } else { - throw new ISE("error!"); - } - } - ) - .iterator() + possiblyDelay( + new ThrowOnCloseProcessorManager<>( + ensureClose( + ProcessorManagers.of( + Iterables.transform( + IntStream.range(0, 101)::iterator, + i -> new ChompingFrameProcessor(Collections.emptyList()) + ) + ).withAccumulation(0L, Long::sum) + ) + ) + ), + maxOutstandingProcessors, + bouncer, + null + ); + + final ExecutionException e = Assert.assertThrows(ExecutionException.class, future::get); + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(IllegalStateException.class)); + MatcherAssert.assertThat(e.getCause(), ThrowableMessageMatcher.hasMessage(CoreMatchers.equalTo("error!"))); + } + + @Test + public void test_runAllFully_errorChannelSecondElementAndClose() + { + final ListenableFuture future = exec.runAllFully( + possiblyDelay( + new ThrowOnCloseProcessorManager<>( + new ThrowOnNextProcessorManager<>( + ensureClose( + ProcessorManagers.of( + Iterables.transform( + IntStream.range(0, 101)::iterator, + i -> new ChompingFrameProcessor(Collections.emptyList()) + ) + ).withAccumulation(0L, Long::sum) + ), + 1 + ) + ) ), - 0L, - Long::sum, maxOutstandingProcessors, bouncer, null @@ -341,9 +410,12 @@ public void test_runAllFully_futureCancel() throws InterruptedException .collect(Collectors.toList()); final ListenableFuture future = exec.runAllFully( - Sequences.simple(processors).map(ConcurrencyTrackingFrameProcessor::new), - 0L, - Long::sum, + possiblyDelay( + ensureClose( + ProcessorManagers.of(Sequences.simple(processors).map(ConcurrencyTrackingFrameProcessor::new)) + .withAccumulation(0L, Long::sum) + ) + ), maxOutstandingProcessors, bouncer, "xyzzy" @@ -364,6 +436,22 @@ public void test_runAllFully_futureCancel() throws InterruptedException Assert.assertEquals(0, exec.cancelableProcessorCount()); } + /** + * Wrap in {@link DelayedProcessorManager} if {@link #delayed} is set. + */ + private ProcessorManager possiblyDelay(final ProcessorManager processorManager) + { + return delayed ? new DelayedProcessorManager<>(processorManager) : processorManager; + } + + /** + * Ensure that the provided processor manager is closed once, and only once. Must be called once per test case. + */ + private ProcessorManager ensureClose(final ProcessorManager processorManager) + { + return new EnsureCloseProcessorManager<>(processorManager); + } + /** * FrameProcessor wrapper that updates {@link #concurrentNow}, {@link #concurrentHighWatermark} to enable * verification of concurrency controls. @@ -425,4 +513,158 @@ public void cleanup() throws IOException } } } + + private class EnsureCloseProcessorManager implements ProcessorManager + { + private final ProcessorManager delegate; + + public EnsureCloseProcessorManager(ProcessorManager delegate) + { + this.delegate = delegate; + } + + @Override + public ListenableFuture>> next() + { + return delegate.next(); + } + + @Override + public R result() + { + return delegate.result(); + } + + @Override + public void close() + { + closed.getAndIncrement(); + delegate.close(); + } + } + + /** + * Processor manager that throws an error on the Nth element. + */ + private static class ThrowOnNextProcessorManager implements ProcessorManager + { + private final ProcessorManager delegate; + private int i; + + public ThrowOnNextProcessorManager(final ProcessorManager delegate, final int i) + { + this.delegate = delegate; + this.i = i; + } + + @Override + public ListenableFuture>> next() + { + if (i == 0) { + throw new ISE("error!"); + } else { + i--; + return delegate.next(); + } + } + + @Override + public R result() + { + return delegate.result(); + } + + @Override + public void close() + { + delegate.close(); + } + } + + /** + * Processor manager that throws an error on {@link #close()}. + */ + private static class ThrowOnCloseProcessorManager implements ProcessorManager + { + private final ProcessorManager delegate; + + public ThrowOnCloseProcessorManager(ProcessorManager delegate) + { + this.delegate = delegate; + } + + @Override + public ListenableFuture>> next() + { + return delegate.next(); + } + + @Override + public R result() + { + return delegate.result(); + } + + @Override + public void close() + { + delegate.close(); + throw new ISE("error!"); + } + } + + /** + * Processor manager that effectively delays future resolution by deferring it through {@link #exec}. + * Especially useful on single-threaded test cases. This helps us ensure that things work when channels don't have + * processors immediately ready upon a call to {@link FrameProcessorExecutor#runAllFully}. + */ + private class DelayedProcessorManager implements ProcessorManager + { + private final ProcessorManager delegate; + + public DelayedProcessorManager(ProcessorManager delegate) + { + this.delegate = delegate; + } + + @Override + public ListenableFuture>> next() + { + final ListenableFuture>> future = delegate.next(); + final SettableFuture>> retVal = SettableFuture.create(); + + Futures.addCallback( + future, + new FutureCallback>>() + { + @Override + public void onSuccess(Optional> result) + { + retVal.set(result); + } + + @Override + public void onFailure(Throwable t) + { + retVal.setException(t); + } + }, + exec.getExecutorService() + ); + + return retVal; + } + + @Override + public R result() + { + return delegate.result(); + } + + @Override + public void close() + { + delegate.close(); + } + } } diff --git a/processing/src/test/java/org/apache/druid/frame/processor/manager/SequenceProcessorManagerTest.java b/processing/src/test/java/org/apache/druid/frame/processor/manager/SequenceProcessorManagerTest.java new file mode 100644 index 000000000000..59aa8d651ee0 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/processor/manager/SequenceProcessorManagerTest.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.frame.processor.manager; + +import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.ListenableFuture; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.ReturnOrAwait; +import org.apache.druid.java.util.common.Unit; +import org.apache.druid.java.util.common.guava.Sequences; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; + +public class SequenceProcessorManagerTest +{ + @Test + public void test_empty() throws Exception + { + final AtomicLong closed = new AtomicLong(); + + try (final SequenceProcessorManager> manager = + new SequenceProcessorManager<>( + Sequences.>empty() + .withBaggage(closed::getAndIncrement))) { + final ListenableFuture>> future = manager.next(); + Assert.assertTrue(future.isDone()); + Assert.assertFalse(future.get().isPresent()); + } + + Assert.assertEquals(1, closed.get()); + } + + @Test + public void test_one() throws Exception + { + final NilFrameProcessor processor = new NilFrameProcessor(); + final AtomicLong closed = new AtomicLong(); + + try (final SequenceProcessorManager> manager = + new SequenceProcessorManager<>( + Sequences.>simple(Collections.singleton(processor)) + .withBaggage(closed::getAndIncrement))) { + // First element. + ListenableFuture>> future = manager.next(); + Assert.assertTrue(future.isDone()); + Assert.assertTrue(future.get().isPresent()); + Assert.assertSame(processor, future.get().get().processor()); + + // End of sequence. + future = manager.next(); + Assert.assertTrue(future.isDone()); + Assert.assertFalse(future.get().isPresent()); + } + + Assert.assertEquals(1, closed.get()); + } + + @Test + public void test_two() throws Exception + { + final NilFrameProcessor processor0 = new NilFrameProcessor(); + final NilFrameProcessor processor1 = new NilFrameProcessor(); + final AtomicLong closed = new AtomicLong(); + + try (final SequenceProcessorManager> manager = + new SequenceProcessorManager<>( + Sequences.>simple(ImmutableList.of(processor0, processor1)) + .withBaggage(closed::getAndIncrement))) { + // First element. + ListenableFuture>> future = manager.next(); + Assert.assertTrue(future.isDone()); + Assert.assertTrue(future.get().isPresent()); + Assert.assertSame(processor0, future.get().get().processor()); + + // Second element. + future = manager.next(); + Assert.assertTrue(future.isDone()); + Assert.assertTrue(future.get().isPresent()); + Assert.assertSame(processor1, future.get().get().processor()); + + // End of sequence. + future = manager.next(); + Assert.assertTrue(future.isDone()); + Assert.assertFalse(future.get().isPresent()); + + // One more, should throw because there's nothing left. + Assert.assertThrows( + NoSuchElementException.class, + manager::next + ); + } + + Assert.assertEquals(1, closed.get()); + } + + @Test + public void test_empty_closeThenNext() + { + final AtomicLong closed = new AtomicLong(); + + final SequenceProcessorManager> manager = + new SequenceProcessorManager<>( + Sequences.>empty() + .withBaggage(closed::getAndIncrement)); + manager.close(); + + // IllegalStateException instead of NoSuchElementException because the problem is that we are closed. + Assert.assertThrows( + IllegalStateException.class, + manager::next + ); + + // Sequence is not closed because it never started iterating. + Assert.assertEquals(0, closed.get()); + } + + private static class NilFrameProcessor implements FrameProcessor + { + @Override + public List inputChannels() + { + return Collections.emptyList(); + } + + @Override + public List outputChannels() + { + return Collections.emptyList(); + } + + @Override + public ReturnOrAwait runIncrementally(IntSet readableInputs) + { + throw new UnsupportedOperationException(); + } + + @Override + public void cleanup() + { + // Do nothing. + } + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java index 342ccbefc48c..31b24825f959 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java @@ -153,6 +153,33 @@ public void test_long() testWithDataset(FrameWriterTestData.TEST_LONGS); } + @Test + public void test_arrayLong() + { + // ARRAY can't be read or written for columnar frames, therefore skip the check if it encounters those + // parameters + Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR); + testWithDataset(FrameWriterTestData.TEST_ARRAYS_LONG); + } + + @Test + public void test_arrayFloat() + { + // ARRAY can't be read or written for columnar frames, therefore skip the check if it encounters those + // parameters + Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR); + testWithDataset(FrameWriterTestData.TEST_ARRAYS_FLOAT); + } + + @Test + public void test_arrayDouble() + { + // ARRAY can't be read or written for columnar frames, therefore skip the check if it encounters those + // parameters + Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR); + testWithDataset(FrameWriterTestData.TEST_ARRAYS_DOUBLE); + } + @Test public void test_float() { @@ -226,6 +253,14 @@ public void test_typePairs() // Test all possible arrangements of two different types. for (final FrameWriterTestData.Dataset dataset1 : FrameWriterTestData.DATASETS) { for (final FrameWriterTestData.Dataset dataset2 : FrameWriterTestData.DATASETS) { + if (dataset1.getType().isArray() && dataset1.getType().getElementType().isNumeric() + || dataset2.getType().isArray() && dataset2.getType().getElementType().isNumeric()) { + if (inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR) { + // Skip the check if any of the dataset is a numerical array and any of the input or the output frame type + // is COLUMNAR. + continue; + } + } final RowSignature signature = makeSignature(Arrays.asList(dataset1, dataset2)); final Sequence> rowSequence = unsortAndMakeRows(Arrays.asList(dataset1, dataset2)); @@ -265,6 +300,7 @@ public void test_typePairs() public void test_insufficientWriteCapacity() { // Test every possible capacity, up to the amount required to write all items from every list. + Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR); final RowSignature signature = makeSignature(FrameWriterTestData.DATASETS); final Sequence> rowSequence = unsortAndMakeRows(FrameWriterTestData.DATASETS); final int totalRows = rowSequence.toList().size(); diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java index 3017f5b9ed48..a52c4d5efdd8 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java @@ -136,6 +136,24 @@ public class FrameWriterTestData ).sorted(Comparators.naturalNullsFirst()).collect(Collectors.toList()) ); + public static final Dataset TEST_ARRAYS_LONG = new Dataset<>( + ColumnType.LONG_ARRAY, + Arrays.asList( + null, + ObjectArrays.EMPTY_ARRAY, + new Object[]{null}, + new Object[]{null, 6L, null, 5L, null}, + new Object[]{null, 6L, null, 5L, NullHandling.defaultLongValue()}, + new Object[]{null, 6L, null, 5L, 0L, -1L}, + new Object[]{null, 6L, null, 5L, 0L, -1L, Long.MIN_VALUE}, + new Object[]{null, 6L, null, 5L, 0L, -1L, Long.MAX_VALUE}, + new Object[]{5L}, + new Object[]{5L, 6L}, + new Object[]{5L, 6L, null}, + new Object[]{Long.MAX_VALUE, Long.MIN_VALUE} + ) + ); + public static final Dataset TEST_FLOATS = new Dataset<>( ColumnType.FLOAT, Stream.of( @@ -158,6 +176,28 @@ public class FrameWriterTestData ).sorted(Comparators.naturalNullsFirst()).collect(Collectors.toList()) ); + //CHECKSTYLE.OFF: Regexp + public static final Dataset TEST_ARRAYS_FLOAT = new Dataset<>( + ColumnType.FLOAT_ARRAY, + Arrays.asList( + null, + ObjectArrays.EMPTY_ARRAY, + new Object[]{null}, + new Object[]{null, 6.2f, null, 5.1f, null}, + new Object[]{null, 6.2f, null, 5.1f, NullHandling.defaultFloatValue()}, + new Object[]{null, 6.2f, null, 5.7f, 0.0f, -1.0f}, + new Object[]{null, 6.2f, null, 5.7f, 0.0f, -1.0f, Float.MIN_VALUE}, + new Object[]{null, 6.2f, null, 5.7f, 0.0f, -1.0f, Float.MAX_VALUE}, + new Object[]{Float.NEGATIVE_INFINITY, Float.MIN_VALUE}, + new Object[]{5.7f}, + new Object[]{5.7f, 6.2f}, + new Object[]{5.7f, 6.2f, null}, + new Object[]{Float.MAX_VALUE, Float.MIN_VALUE}, + new Object[]{Float.POSITIVE_INFINITY, Float.MIN_VALUE} + ) + ); + //CHECKSTYLE.ON: Regexp + public static final Dataset TEST_DOUBLES = new Dataset<>( ColumnType.DOUBLE, Stream.of( @@ -180,6 +220,28 @@ public class FrameWriterTestData ).sorted(Comparators.naturalNullsFirst()).collect(Collectors.toList()) ); + //CHECKSTYLE.OFF: Regexp + public static final Dataset TEST_ARRAYS_DOUBLE = new Dataset<>( + ColumnType.DOUBLE_ARRAY, + Arrays.asList( + null, + ObjectArrays.EMPTY_ARRAY, + new Object[]{null}, + new Object[]{null, 6.2d, null, 5.1d, null}, + new Object[]{null, 6.2d, null, 5.1d, NullHandling.defaultDoubleValue()}, + new Object[]{null, 6.2d, null, 5.7d, 0.0d, -1.0d}, + new Object[]{null, 6.2d, null, 5.7d, 0.0d, -1.0d, Double.MIN_VALUE}, + new Object[]{null, 6.2d, null, 5.7d, 0.0d, -1.0d, Double.MAX_VALUE}, + new Object[]{Double.NEGATIVE_INFINITY, Double.MIN_VALUE}, + new Object[]{5.7d}, + new Object[]{5.7d, 6.2d}, + new Object[]{5.7d, 6.2d, null}, + new Object[]{Double.MAX_VALUE, Double.MIN_VALUE}, + new Object[]{Double.POSITIVE_INFINITY, Double.MIN_VALUE} + ) + ); + //CHECKSTYLE.ON: Regexp + public static final Dataset TEST_COMPLEX = new Dataset<>( HyperUniquesAggregatorFactory.TYPE, Arrays.asList( @@ -200,6 +262,9 @@ public class FrameWriterTestData .add(TEST_STRINGS_SINGLE_VALUE) .add(TEST_STRINGS_MULTI_VALUE) .add(TEST_ARRAYS_STRING) + .add(TEST_ARRAYS_LONG) + .add(TEST_ARRAYS_FLOAT) + .add(TEST_ARRAYS_DOUBLE) .add(TEST_COMPLEX) .build(); diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java index 42ed468ec8c3..9d359eed05e0 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java @@ -75,42 +75,6 @@ public void test_columnar() Assert.assertEquals(ALLOCATOR_CAPACITY, factory.allocatorCapacity()); } - @Test - public void test_rowBased_unsupportedSortingColumnType() - { - final UnsupportedColumnTypeException e = Assert.assertThrows( - UnsupportedColumnTypeException.class, - () -> FrameWriters.makeFrameWriterFactory( - FrameType.ROW_BASED, - new ArenaMemoryAllocatorFactory(ALLOCATOR_CAPACITY), - RowSignature.builder().add("x", ColumnType.LONG_ARRAY).build(), - Collections.singletonList(new KeyColumn("x", KeyOrder.ASCENDING)) - ) - ); - - Assert.assertEquals("x", e.getColumnName()); - Assert.assertEquals(ColumnType.LONG_ARRAY, e.getColumnType()); - } - - @Test - public void test_rowBased_unsupportedNonSortingColumnType() - { - final FrameWriterFactory factory = FrameWriters.makeFrameWriterFactory( - FrameType.ROW_BASED, - new ArenaMemoryAllocatorFactory(ALLOCATOR_CAPACITY), - RowSignature.builder().add("x", ColumnType.LONG_ARRAY).build(), - Collections.emptyList() - ); - - final UnsupportedColumnTypeException e = Assert.assertThrows( - UnsupportedColumnTypeException.class, - () -> factory.newFrameWriter(new AllNullColumnSelectorFactory()) - ); - - Assert.assertEquals("x", e.getColumnName()); - Assert.assertEquals(ColumnType.LONG_ARRAY, e.getColumnType()); - } - @Test public void test_columnar_unsupportedColumnType() { diff --git a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java index 639a7c529ed1..dbb952fbf1cf 100644 --- a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java +++ b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java @@ -356,6 +356,93 @@ public void testPeriodDaylightSaving() hour.bucketStart(DateTimes.of("2012-11-04T03:30:00-08:00")) ) ); + + final PeriodGranularity p7days = new PeriodGranularity( + new Period("P7D"), + DateTimes.of("2022-03-24T02:35:00.000-07:00"), + tz + ); + assertSameDateTime( + Lists.newArrayList( + new DateTime("2022-03-03T02:35:00.000-08:00", tz), + new DateTime("2022-03-10T02:35:00.000-08:00", tz), + new DateTime("2022-03-24T02:35:00.000-07:00", tz), + new DateTime("2022-03-31T02:35:00.000-07:00", tz) + + ), + Lists.newArrayList( + p7days.bucketStart(DateTimes.of("2022-03-04T02:35:00.000-08:00")), + p7days.bucketStart(DateTimes.of("2022-03-16T02:35:00.000-07:00")), + p7days.bucketStart(DateTimes.of("2022-03-26T02:35:00.000-07:00")), + p7days.bucketStart(DateTimes.of("2022-03-31T03:35:00.000-07:00")) + ) + ); + + final PeriodGranularity week = new PeriodGranularity( + new Period("P1W"), + DateTimes.of("2022-03-24T02:35:00.000-07:00"), + tz + ); + + assertSameDateTime( + Lists.newArrayList( + new DateTime("2022-03-03T02:35:00.000-08:00", tz), + new DateTime("2022-03-10T02:35:00.000-08:00", tz), + new DateTime("2022-03-24T02:35:00.000-07:00", tz), + new DateTime("2022-03-31T02:35:00.000-07:00", tz) + + ), + Lists.newArrayList( + week.bucketStart(DateTimes.of("2022-03-04T02:35:00.000-08:00")), + week.bucketStart(DateTimes.of("2022-03-16T02:35:00.000-07:00")), + week.bucketStart(DateTimes.of("2022-03-26T02:35:00.000-07:00")), + week.bucketStart(DateTimes.of("2022-03-31T03:35:00.000-07:00")) + ) + ); + + final PeriodGranularity month = new PeriodGranularity( + new Period("P1M"), + DateTimes.of("2022-03-24T02:35:00.000-07:00"), + tz + ); + + assertSameDateTime( + Lists.newArrayList( + new DateTime("2022-02-24T02:35:00.000-08:00", tz), + new DateTime("2022-02-24T02:35:00.000-08:00", tz), + new DateTime("2022-03-24T02:35:00.000-07:00", tz), + new DateTime("2022-03-24T02:35:00.000-07:00", tz) + + ), + Lists.newArrayList( + month.bucketStart(DateTimes.of("2022-03-04T02:35:00.000-08:00")), + month.bucketStart(DateTimes.of("2022-03-16T02:35:00.000-07:00")), + month.bucketStart(DateTimes.of("2022-03-26T02:35:00.000-07:00")), + month.bucketStart(DateTimes.of("2022-03-31T03:35:00.000-07:00")) + ) + ); + + final PeriodGranularity year = new PeriodGranularity( + new Period("P1Y"), + DateTimes.of("2022-03-24T02:35:00.000-07:00"), + tz + ); + + assertSameDateTime( + Lists.newArrayList( + new DateTime("2021-03-24T02:35:00.000-07:00", tz), + new DateTime("2021-03-24T02:35:00.000-07:00", tz), + new DateTime("2022-03-24T02:35:00.000-07:00", tz), + new DateTime("2022-03-24T02:35:00.000-07:00", tz) + + ), + Lists.newArrayList( + year.bucketStart(DateTimes.of("2022-03-04T02:35:00.000-08:00")), + year.bucketStart(DateTimes.of("2022-03-16T02:35:00.000-07:00")), + year.bucketStart(DateTimes.of("2022-03-26T02:35:00.000-07:00")), + year.bucketStart(DateTimes.of("2022-03-31T03:35:00.000-07:00")) + ) + ); } @Test @@ -877,7 +964,7 @@ private void assertSameInterval(List expected, Iterable actu Assert.assertFalse("actualIter not exhausted!?", actualIter.hasNext()); Assert.assertFalse("expectedIter not exhausted!?", expectedIter.hasNext()); } - + @Test public void testTruncateKathmandu() { diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/constant/LongConstantBufferAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/constant/LongConstantBufferAggregatorTest.java index 0608fca85b74..845d2c825b82 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/constant/LongConstantBufferAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/constant/LongConstantBufferAggregatorTest.java @@ -20,7 +20,6 @@ package org.apache.druid.query.aggregation.constant; import org.apache.commons.lang.math.RandomUtils; -import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -38,9 +37,8 @@ public void setup() { randomVal = RandomUtils.nextLong(); aggregator = new LongConstantBufferAggregator(randomVal); - byteBuffer = EasyMock.mock(ByteBuffer.class); - EasyMock.replay(byteBuffer); - EasyMock.verifyUnexpectedCalls(byteBuffer); + // mark byteBuffer null to verify no methods ever get called on it. + byteBuffer = null; } @Test diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/constant/LongConstantVectorAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/constant/LongConstantVectorAggregatorTest.java index f62dd0369c61..6159ea6a28b8 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/constant/LongConstantVectorAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/constant/LongConstantVectorAggregatorTest.java @@ -20,7 +20,6 @@ package org.apache.druid.query.aggregation.constant; import org.apache.commons.lang.math.RandomUtils; -import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -38,9 +37,8 @@ public void setup() { randomVal = RandomUtils.nextLong(); aggregator = new LongConstantVectorAggregator(randomVal); - byteBuffer = EasyMock.mock(ByteBuffer.class); - EasyMock.replay(byteBuffer); - EasyMock.verifyUnexpectedCalls(byteBuffer); + // mark byteBuffer null to verify no methods ever get called on it. + byteBuffer = null; } @Test diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java index 1926e14eae55..4afb202b4331 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java @@ -634,6 +634,44 @@ public void testMergeAggregatorsConflict() ) ); + // Simulate multi-level lenient merge (unmerged first) + Assert.assertEquals( + new SegmentAnalysis( + "dummy_2021-01-01T00:00:00.000Z_2021-01-02T00:00:00.000Z_merged", + null, + new LinkedHashMap<>(), + 0, + 0, + expectedLenient, + null, + null, + null + ), + mergeLenient( + analysis1, + mergeLenient(analysis1, analysis2) + ) + ); + + // Simulate multi-level lenient merge (unmerged second) + Assert.assertEquals( + new SegmentAnalysis( + "dummy_2021-01-01T00:00:00.000Z_2021-01-02T00:00:00.000Z_merged", + null, + new LinkedHashMap<>(), + 0, + 0, + expectedLenient, + null, + null, + null + ), + mergeLenient( + mergeLenient(analysis1, analysis2), + analysis1 + ) + ); + Assert.assertEquals( new SegmentAnalysis( "dummy_2021-01-01T00:00:00.000Z_2021-01-02T00:00:00.000Z_merged", diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java index cb1a8270af7a..2c2a69faf44f 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java @@ -41,7 +41,9 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.IndexableAdapter; +import org.apache.druid.segment.NestedDataColumnHandlerV4; import org.apache.druid.segment.NestedDataColumnIndexerV4; +import org.apache.druid.segment.NestedDataColumnSchema; import org.apache.druid.segment.ObjectColumnSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.SimpleAscendingOffset; @@ -226,6 +228,9 @@ public void testBasicFunctionality() throws IOException final ColumnCapabilities capabilities = holder.getCapabilities(); Assert.assertEquals(ColumnType.NESTED_DATA, capabilities.toColumnType()); Assert.assertTrue(holder.getColumnFormat() instanceof NestedDataComplexTypeSerde.NestedColumnFormatV4); + Assert.assertTrue(holder.getColumnFormat().getColumnHandler("test") instanceof NestedDataColumnHandlerV4); + NestedDataColumnSchema schema = (NestedDataColumnSchema) holder.getColumnFormat().getColumnSchema("test"); + Assert.assertEquals(4, schema.getFormatVersion()); try (NestedDataComplexColumn column = (NestedDataComplexColumn) holder.getColumn()) { smokeTest(column); } diff --git a/services/pom.xml b/services/pom.xml index 42dadd57aa24..3556c8c849e4 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -282,6 +282,12 @@ shade + + false ${project.build.directory}/${project.artifactId}-${project.version}-selfcontained.jar diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index 25e06ca88254..9c41d79070bf 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -1432,17 +1432,14 @@ private WindowOperatorQuery toWindowQuery() if (windowing == null) { return null; } - - final DataSource myDataSource; if (dataSource instanceof TableDataSource) { - // In this case, we first plan a scan query to pull the results up for us before applying the window - myDataSource = new QueryDataSource(toScanQuery()); - } else { - myDataSource = dataSource; + // We need a scan query to pull the results up for us before applying the window + // Returning null here to ensure that the planner generates that alternative + return null; } return new WindowOperatorQuery( - myDataSource, + dataSource, new LegacySegmentSpec(Intervals.ETERNITY), plannerContext.queryContextMap(), windowing.getSignature(), diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java index e486a9588123..07c5544441dc 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java @@ -88,9 +88,9 @@ public class Windowing private static final ImmutableMap KNOWN_WINDOW_FNS = ImmutableMap .builder() .put("LAG", (agg) -> - new WindowOffsetProcessor(agg.getColumn(0), agg.getOutputName(), -agg.getConstantInt(1))) + new WindowOffsetProcessor(agg.getColumn(0), agg.getOutputName(), -agg.getConstantInt(1, 1))) .put("LEAD", (agg) -> - new WindowOffsetProcessor(agg.getColumn(0), agg.getOutputName(), agg.getConstantInt(1))) + new WindowOffsetProcessor(agg.getColumn(0), agg.getOutputName(), agg.getConstantInt(1, 1))) .put("FIRST_VALUE", (agg) -> new WindowFirstProcessor(agg.getColumn(0), agg.getOutputName())) .put("LAST_VALUE", (agg) -> diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 16f642701724..3bb20bb3780e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -1170,7 +1170,7 @@ protected void skipVectorize() skipVectorize = true; } - protected void notMsqCompatible() + protected void msqIncompatible() { msqCompatible = false; } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java index 8fc531d35ab6..e6a669b9c28b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java @@ -87,7 +87,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest @Test public void testSelectConstantArrayExpressionFromTable() { - notMsqCompatible(); testQuery( "SELECT ARRAY[1,2] as arr, dim1 FROM foo LIMIT 1", ImmutableList.of( @@ -169,7 +168,8 @@ public void testSelectNonConstantArrayExpressionFromTable() @Test public void testSelectNonConstantArrayExpressionFromTableForMultival() { - notMsqCompatible(); + // Produces nested string array, that MSQ can't infer from the selector + msqIncompatible(); final String sql = "SELECT ARRAY[CONCAT(dim3, 'word'),'up'] as arr, dim1 FROM foo LIMIT 5"; final Query scanQuery = newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) @@ -209,7 +209,6 @@ public void testSomeArrayFunctionsWithScanQuery() // Yes these outputs are strange sometimes, arrays are in a partial state of existence so end up a bit // stringy for now this is because virtual column selectors are coercing values back to stringish so that // multi-valued string dimensions can be grouped on. - notMsqCompatible(); List expectedResults; if (useDefault) { expectedResults = ImmutableList.of( @@ -389,7 +388,6 @@ public void testSomeArrayFunctionsWithScanQueryNoStringify() // which will still always be stringified to ultimately adhere to the varchar type // as array support increases in the engine this will likely change since using explict array functions should // probably kick it into an array - notMsqCompatible(); List expectedResults; if (useDefault) { expectedResults = ImmutableList.of( @@ -1021,7 +1019,6 @@ public void testArrayOffset() @Test public void testArrayGroupAsLongArray() { - notMsqCompatible(); // Cannot vectorize as we donot have support in native query subsytem for grouping on arrays cannotVectorize(); testQuery( @@ -1073,7 +1070,6 @@ public void testArrayGroupAsDoubleArray() { // Cannot vectorize as we donot have support in native query subsytem for grouping on arrays as keys cannotVectorize(); - notMsqCompatible(); testQuery( "SELECT ARRAY[d1], SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC", QUERY_CONTEXT_NO_STRINGIFY_ARRAY, @@ -1121,7 +1117,6 @@ public void testArrayGroupAsDoubleArray() @Test public void testArrayGroupAsFloatArray() { - notMsqCompatible(); // Cannot vectorize as we donot have support in native query subsytem for grouping on arrays as keys cannotVectorize(); testQuery( @@ -1612,7 +1607,6 @@ public void testArrayAggMultiValue() @Test public void testArrayAggNumeric() { - notMsqCompatible(); cannotVectorize(); testQuery( "SELECT ARRAY_AGG(l1), ARRAY_AGG(DISTINCT l1), ARRAY_AGG(d1), ARRAY_AGG(DISTINCT d1), ARRAY_AGG(f1), ARRAY_AGG(DISTINCT f1) FROM numfoo", @@ -1749,7 +1743,6 @@ public void testArrayAggNumeric() @Test public void testArrayAggQuantile() { - notMsqCompatible(); cannotVectorize(); testQuery( "SELECT ARRAY_QUANTILE(ARRAY_AGG(l1), 0.9) FROM numfoo", @@ -1793,7 +1786,9 @@ public void testArrayAggQuantile() @Test public void testArrayAggArrays() { - notMsqCompatible(); + // Produces nested array - ARRAY>, which frame writers don't support. A way to get this query + // to run would be to use nested columns. + msqIncompatible(); cannotVectorize(); testQuery( "SELECT ARRAY_AGG(ARRAY[l1, l2]), ARRAY_AGG(DISTINCT ARRAY[l1, l2]) FROM numfoo", @@ -1890,7 +1885,6 @@ public void testArrayAggArrays() @Test public void testArrayConcatAggArrays() { - notMsqCompatible(); cannotVectorize(); testQuery( "SELECT ARRAY_CONCAT_AGG(ARRAY[l1, l2]), ARRAY_CONCAT_AGG(DISTINCT ARRAY[l1, l2]) FROM numfoo", @@ -2039,7 +2033,6 @@ public void testArrayAggExpression() public void testArrayAggMaxBytes() { cannotVectorize(); - notMsqCompatible(); testQuery( "SELECT ARRAY_AGG(l1, 128), ARRAY_AGG(DISTINCT l1, 128) FROM numfoo", ImmutableList.of( @@ -2239,7 +2232,6 @@ public void testArrayAggGroupByArrayAggFromSubquery() @Test public void testArrayAggGroupByArrayAggOfLongsFromSubquery() { - notMsqCompatible(); requireMergeBuffers(3); cannotVectorize(); testQuery( @@ -2379,7 +2371,6 @@ public void testArrayAggGroupByArrayAggOfStringsFromSubquery() @Test public void testArrayAggGroupByArrayAggOfDoubleFromSubquery() { - notMsqCompatible(); requireMergeBuffers(3); cannotVectorize(); testQuery( @@ -2897,7 +2888,6 @@ public void testUnnestTwiceWithFiltersAndExpressions() @Test public void testUnnestThriceWithFiltersOnDimAndUnnestCol() { - notMsqCompatible(); cannotVectorize(); String sql = " SELECT dimZipf, dim3_unnest1, dim3_unnest2, dim3_unnest3 FROM \n" + " ( SELECT * FROM \n" @@ -2996,7 +2986,6 @@ public void testUnnestThriceWithFiltersOnDimAndUnnestCol() @Test public void testUnnestThriceWithFiltersOnDimAndAllUnnestColumns() { - notMsqCompatible(); cannotVectorize(); String sql = " SELECT dimZipf, dim3_unnest1, dim3_unnest2, dim3_unnest3 FROM \n" + " ( SELECT * FROM \n" diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index bb1660f856a9..d1300ff19b24 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -695,7 +695,7 @@ public void testFilterAndGroupByLookupUsingJoinOperatorBackwards(Map queryContext) { // MSQ does not support UNION ALL. - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize JOIN operator. cannotVectorize(); @@ -1918,7 +1918,7 @@ public void testCommaJoinTableLookupTableMismatchedTypes(Map que // Regression test for https://github.com/apache/druid/issues/9646. // Empty-dataset aggregation queries in MSQ return an empty row, rather than a single row as SQL requires. - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize JOIN operator. cannotVectorize(); @@ -1974,7 +1974,7 @@ public void testCommaJoinTableLookupTableMismatchedTypes(Map que public void testJoinTableLookupTableMismatchedTypesWithoutComma(Map queryContext) { // Empty-dataset aggregation queries in MSQ return an empty row, rather than a single row as SQL requires. - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize JOIN operator. cannotVectorize(); @@ -2396,7 +2396,7 @@ public void testSelectOnLookupUsingLeftJoinOperator(Map queryCon public void testSelectOnLookupUsingRightJoinOperator(Map queryContext) { // MSQ refuses to do RIGHT join with broadcast. - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT dim1, lookyloo.*\n" @@ -2438,7 +2438,7 @@ public void testSelectOnLookupUsingRightJoinOperator(Map queryCo public void testSelectOnLookupUsingFullJoinOperator(Map queryContext) { // MSQ refuses to do FULL join with broadcast. - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT dim1, m1, cnt, lookyloo.*\n" @@ -3751,7 +3751,7 @@ public void testInnerJoinSubqueryWithSelectorFilter(Map queryCon { if (sortBasedJoin) { // Cannot handle the [l1.k = 'abc'] condition. - notMsqCompatible(); + msqIncompatible(); } // Cannot vectorize due to 'concat' expression. @@ -4172,7 +4172,7 @@ public void testSemiJoinWithOuterTimeExtractAggregateWithOrderBy() public void testJoinOnMultiValuedColumnShouldThrowException(Map queryContext) { // MSQ throws a slightly different error than QueryException. - notMsqCompatible(); + msqIncompatible(); final String query = "SELECT dim3, l.v from druid.foo f inner join lookup.lookyloo l on f.dim3 = l.k\n"; @@ -4189,7 +4189,7 @@ public void testJoinOnMultiValuedColumnShouldThrowException(Map public void testUnionAllTwoQueriesLeftQueryIsJoin(Map queryContext) { // MSQ does not support UNION ALL. - notMsqCompatible(); + msqIncompatible(); // Fully removing the join allows this query to vectorize. if (!isRewriteJoinToFilter(queryContext)) { @@ -4233,7 +4233,7 @@ public void testUnionAllTwoQueriesLeftQueryIsJoin(Map queryConte public void testUnionAllTwoQueriesRightQueryIsJoin(Map queryContext) { // MSQ does not support UNION ALL. - notMsqCompatible(); + msqIncompatible(); // Fully removing the join allows this query to vectorize. if (!isRewriteJoinToFilter(queryContext)) { @@ -4276,7 +4276,7 @@ public void testUnionAllTwoQueriesRightQueryIsJoin(Map queryCont public void testUnionAllTwoQueriesBothQueriesAreJoin() { // MSQ does not support UNION ALL. - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); testQuery( @@ -4982,7 +4982,7 @@ public void testVirtualColumnOnMVFilterJoinExpression(Map queryC // IndexedTableDimensionSelector. In native, this doesn't happen, because we don't have as much type information, // and we end up using IndexedTableColumnValueSelector instead. This is really a problem with // IndexedTableColumnSelectorFactory: it assumes strings are not multi-valued, even though they might be. - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN druid.numfoo as foo2 " @@ -5038,7 +5038,7 @@ public void testVirtualColumnOnMVFilterMultiJoinExpression(Map q // IndexedTableDimensionSelector. In native, this doesn't happen, because we don't have as much type information, // and we end up using IndexedTableColumnValueSelector instead. This is really a problem with // IndexedTableColumnSelectorFactory: it assumes strings are not multi-valued, even though they might be. - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN " diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 35f85c5fbf2b..d67ed60c5527 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -141,7 +141,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testInformationSchemaSchemata() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT DISTINCT SCHEMA_NAME FROM INFORMATION_SCHEMA.SCHEMATA", ImmutableList.of(), @@ -158,7 +158,7 @@ public void testInformationSchemaSchemata() @Test public void testInformationSchemaTables() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE, IS_JOINABLE, IS_BROADCAST\n" + "FROM INFORMATION_SCHEMA.TABLES\n" @@ -237,7 +237,7 @@ public void testInformationSchemaTables() @Test public void testInformationSchemaColumnsOnTable() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n" + "FROM INFORMATION_SCHEMA.COLUMNS\n" @@ -259,7 +259,7 @@ public void testInformationSchemaColumnsOnTable() @Test public void testInformationSchemaColumnsOnForbiddenTable() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n" + "FROM INFORMATION_SCHEMA.COLUMNS\n" @@ -290,7 +290,7 @@ public void testInformationSchemaColumnsOnForbiddenTable() @Test public void testInformationSchemaColumnsOnView() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n" + "FROM INFORMATION_SCHEMA.COLUMNS\n" @@ -305,7 +305,7 @@ public void testInformationSchemaColumnsOnView() @Test public void testInformationSchemaColumnsOnAnotherView() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n" + "FROM INFORMATION_SCHEMA.COLUMNS\n" @@ -322,7 +322,7 @@ public void testInformationSchemaColumnsOnAnotherView() @Test public void testCannotInsertWithNativeEngine() { - notMsqCompatible(); + msqIncompatible(); final DruidException e = Assert.assertThrows( DruidException.class, () -> testQuery( @@ -341,7 +341,7 @@ public void testCannotInsertWithNativeEngine() @Test public void testCannotReplaceWithNativeEngine() { - notMsqCompatible(); + msqIncompatible(); final DruidException e = Assert.assertThrows( DruidException.class, () -> testQuery( @@ -360,7 +360,7 @@ public void testCannotReplaceWithNativeEngine() @Test public void testAggregatorsOnInformationSchemaColumns() { - notMsqCompatible(); + msqIncompatible(); // Not including COUNT DISTINCT, since it isn't supported by BindableAggregate, and so it can't work. testQuery( "SELECT\n" @@ -381,7 +381,7 @@ public void testAggregatorsOnInformationSchemaColumns() @Test public void testFilterAggregatorFunctionsOnInformationSchemaRoutines() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + " COUNT(*)\n" @@ -397,7 +397,7 @@ public void testFilterAggregatorFunctionsOnInformationSchemaRoutines() @Test public void testFilterScalarFunctionsOnInformationSchemaRoutines() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + " COUNT(*)\n" @@ -413,7 +413,7 @@ public void testFilterScalarFunctionsOnInformationSchemaRoutines() @Test public void testNonExistentSchemaOnInformationSchemaRoutines() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT *\n" + "FROM INFORMATION_SCHEMA.ROUTINES\n" @@ -426,7 +426,7 @@ public void testNonExistentSchemaOnInformationSchemaRoutines() @Test public void testTopNLimitWrapping() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { expected = ImmutableList.of( @@ -631,7 +631,7 @@ public void testGroupBySingleColumnDescendingNoTopN() @Test public void testEarliestAggregators() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT " @@ -678,7 +678,7 @@ public void testEarliestAggregators() @Test public void testLatestVectorAggregators() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT " + "LATEST(cnt), LATEST(cnt + 1), LATEST(m1), LATEST(m1+1) " @@ -712,7 +712,7 @@ public void testLatestVectorAggregators() @Test public void testLatestAggregators() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT " @@ -857,7 +857,7 @@ public void testAnyAggregatorsOnHeapNumericNulls() @Test public void testLatestVectorAggregatorsOnTimeExpression() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT \n" + " LATEST_BY(m1, MILLIS_TO_TIMESTAMP(BITWISE_SHIFT_RIGHT(TIMESTAMP_TO_MILLIS(__time), 3)))\n" @@ -928,7 +928,7 @@ public void testAnyAggregatorsOffHeapNumericNulls() @Test public void testPrimitiveLatestInSubquery() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT SUM(val1), SUM(val2), SUM(val3) FROM (SELECT dim2, LATEST(m1) AS val1, LATEST(cnt) AS val2, LATEST(m2) AS val3 FROM foo GROUP BY dim2)", ImmutableList.of( @@ -976,7 +976,7 @@ public void testPrimitiveLatestInSubquery() @Test public void testPrimitiveLatestInSubqueryGroupBy() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT dim2, LATEST(m1) AS val1 FROM foo GROUP BY dim2", ImmutableList.of( @@ -1095,7 +1095,7 @@ public void testStringLatestByGroupByWithAlwaysFalseCondition() @Test public void testPrimitiveEarliestInSubquery() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT SUM(val1), SUM(val2), SUM(val3) FROM (SELECT dim2, EARLIEST(m1) AS val1, EARLIEST(cnt) AS val2, EARLIEST(m2) AS val3 FROM foo GROUP BY dim2)", @@ -1303,7 +1303,7 @@ public void testPrimitiveAnyInSubquery() @Test public void testStringEarliestSingleStringDim() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT dim2, EARLIEST(dim1,10) AS val FROM foo GROUP BY dim2", ImmutableList.of( @@ -1419,7 +1419,7 @@ public void testStringAnyInSubquery() @Test public void testEarliestAggregatorsNumericNulls() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT EARLIEST(l1), EARLIEST(d1), EARLIEST(f1) FROM druid.numfoo", @@ -1447,7 +1447,7 @@ public void testEarliestAggregatorsNumericNulls() @Test public void testLatestAggregatorsNumericNull() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT LATEST(l1), LATEST(d1), LATEST(f1) FROM druid.numfoo", ImmutableList.of( @@ -1478,7 +1478,7 @@ public void testLatestAggregatorsNumericNull() @Test public void testFirstLatestAggregatorsSkipNulls() { - notMsqCompatible(); + msqIncompatible(); final DimFilter filter; if (useDefault) { @@ -1592,7 +1592,7 @@ public void testAnyAggregatorsSkipNullsWithFilter() @Test public void testOrderByEarliestFloat() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { @@ -1639,7 +1639,7 @@ public void testOrderByEarliestFloat() @Test public void testOrderByEarliestDouble() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { @@ -1686,7 +1686,7 @@ public void testOrderByEarliestDouble() @Test public void testOrderByEarliestLong() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { @@ -1733,7 +1733,7 @@ public void testOrderByEarliestLong() @Test public void testOrderByLatestFloat() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { expected = ImmutableList.of( @@ -1780,7 +1780,7 @@ public void testOrderByLatestFloat() @Test public void testOrderByLatestDouble() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { expected = ImmutableList.of( @@ -1826,7 +1826,7 @@ public void testOrderByLatestDouble() @Test public void testOrderByLatestLong() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { expected = ImmutableList.of( @@ -1884,7 +1884,7 @@ public void testOrderByAnyFloat() ); } else { // Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved - notMsqCompatible(); + msqIncompatible(); expected = ImmutableList.of( new Object[]{"2", 0.0f}, @@ -1935,7 +1935,7 @@ public void testOrderByAnyDouble() ); } else { // Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved - notMsqCompatible(); + msqIncompatible(); expected = ImmutableList.of( new Object[]{"2", 0.0}, @@ -1985,7 +1985,7 @@ public void testOrderByAnyLong() ); } else { // Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved - notMsqCompatible(); + msqIncompatible(); expected = ImmutableList.of( new Object[]{"2", 0L}, @@ -2066,7 +2066,7 @@ public void testGroupByOrdinal() @Ignore("Disabled since GROUP BY alias can confuse the validator; see DruidConformance::isGroupByAlias") public void testGroupByAndOrderByAlias() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT cnt AS theCnt, COUNT(*) FROM druid.foo GROUP BY theCnt ORDER BY theCnt ASC", ImmutableList.of( @@ -2449,7 +2449,7 @@ public void testHavingOnExactCountDistinct() @Test public void testExactCountDistinctWithFilter() { - notMsqCompatible(); + msqIncompatible(); final String sqlQuery = "SELECT COUNT(DISTINCT foo.dim1) FILTER(WHERE foo.cnt = 1), SUM(foo.cnt) FROM druid.foo"; // When useApproximateCountDistinct=false and useGroupingSetForExactDistinct=false, planning fails due // to a bug in the Calcite's rule (AggregateExpandDistinctAggregatesRule) @@ -2814,7 +2814,7 @@ public void testTopNWithSelectAndOrderByProjections() @Test public void testUnionAllQueries() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COUNT(*) FROM foo UNION ALL SELECT SUM(cnt) FROM foo UNION ALL SELECT COUNT(*) FROM foo", ImmutableList.of( @@ -2848,7 +2848,7 @@ public void testUnionAllQueries() @Test public void testUnionAllQueriesWithLimit() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT * FROM (" + "SELECT COUNT(*) FROM foo UNION ALL SELECT SUM(cnt) FROM foo UNION ALL SELECT COUNT(*) FROM foo" @@ -2877,7 +2877,7 @@ public void testUnionAllQueriesWithLimit() @Test public void testUnionAllDifferentTablesWithMapping() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + "dim1, dim2, SUM(m1), COUNT(*)\n" @@ -2921,7 +2921,7 @@ public void testUnionAllDifferentTablesWithMapping() @Test public void testJoinUnionAllDifferentTablesWithMapping() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + "dim1, dim2, SUM(m1), COUNT(*)\n" @@ -2985,7 +2985,7 @@ public void testUnionAllTablesColumnCountMismatch() @Test public void testUnionAllTablesColumnTypeMismatchFloatLong() { - notMsqCompatible(); + msqIncompatible(); // "m1" has a different type in foo and foo2 (float vs long), but this query is OK anyway because they can both // be implicitly cast to double. @@ -3095,7 +3095,7 @@ public void testUnionAllTablesWhenCastAndMappingIsRequired() @Test public void testUnionAllSameTableTwice() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + "dim1, dim2, SUM(m1), COUNT(*)\n" @@ -3139,7 +3139,7 @@ public void testUnionAllSameTableTwice() @Test public void testUnionAllSameTableTwiceWithSameMapping() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + "dim1, dim2, SUM(m1), COUNT(*)\n" @@ -3197,7 +3197,7 @@ public void testUnionAllSameTableTwiceWithDifferentMapping() @Test public void testUnionAllSameTableThreeTimes() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + "dim1, dim2, SUM(m1), COUNT(*)\n" @@ -3302,7 +3302,7 @@ public void testUnionAllThreeTablesColumnCountMismatch3() @Test public void testUnionAllSameTableThreeTimesWithSameMapping() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + "dim1, dim2, SUM(m1), COUNT(*)\n" @@ -3673,7 +3673,7 @@ public void testNullDoubleTopN() { // Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved if (NullHandling.sqlCompatible()) { - notMsqCompatible(); + msqIncompatible(); } List expected; if (useDefault) { @@ -3717,7 +3717,7 @@ public void testNullFloatTopN() { // Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved if (NullHandling.sqlCompatible()) { - notMsqCompatible(); + msqIncompatible(); } List expected; if (useDefault) { @@ -3761,7 +3761,7 @@ public void testNullLongTopN() { // Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved if (NullHandling.sqlCompatible()) { - notMsqCompatible(); + msqIncompatible(); } List expected; if (useDefault) { @@ -4199,7 +4199,7 @@ public void testColumnIsNull() @Test public void testGroupingWithNullInFilter() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COUNT(*) FROM foo WHERE dim1 IN (NULL)", ImmutableList.of( @@ -4224,7 +4224,7 @@ public void testGroupingWithNullInFilter() @Test public void testGroupByNothingWithLiterallyFalseFilter() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COUNT(*), MAX(cnt) FROM druid.foo WHERE 1 = 0", ImmutableList.of( @@ -4250,7 +4250,7 @@ public void testGroupByNothingWithLiterallyFalseFilter() @Test public void testGroupByNothingWithImpossibleTimeFilter() { - notMsqCompatible(); + msqIncompatible(); // Regression test for https://github.com/apache/druid/issues/7671 testQuery( @@ -4321,7 +4321,7 @@ public void testGroupByOneColumnWithLiterallyFalseFilter() @Test public void testGroupByWithFilterMatchingNothing() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COUNT(*), MAX(cnt) FROM druid.foo WHERE dim1 = 'foobar'", ImmutableList.of( @@ -4669,7 +4669,7 @@ public void testCountStarWithLongColumnFiltersForceRange() @Test public void testCountStarWithLongColumnFiltersOnFloatLiterals() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COUNT(*) FROM druid.foo WHERE cnt > 1.1 and cnt < 100000001.0", ImmutableList.of( @@ -5659,7 +5659,7 @@ public void testCountStarWithDegenerateFilter() public void testCountStarWithNotOfDegenerateFilter() { - notMsqCompatible(); + msqIncompatible(); // HashJoinSegmentStorageAdapter is not vectorizable cannotVectorize(); @@ -5689,7 +5689,7 @@ public void testCountStarWithNotOfDegenerateFilter() @Test public void testUnplannableQueries() { - notMsqCompatible(); + msqIncompatible(); // All of these queries are unplannable because they rely on features Druid doesn't support. // This test is here to confirm that we don't fall back to Calcite's interpreter or enumerable implementation. // It's also here so when we do support these features, we can have "real" tests for these queries. @@ -5798,7 +5798,7 @@ public void testIsNotDistinctFromLiteral() @Test public void testArrayAggQueryOnComplexDatatypes() { - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); testQuery( "SELECT ARRAY_AGG(unique_dim1) FROM druid.foo", @@ -5961,7 +5961,7 @@ public void testCountStarWithTimeInIntervalFilterLosAngeles() @Test public void testCountStarWithTimeInIntervalFilterInvalidInterval() { - notMsqCompatible(); + msqIncompatible(); testQueryThrows( "SELECT COUNT(*) FROM druid.foo " + "WHERE TIME_IN_INTERVAL(__time, '2000-01-01/X')", @@ -5978,7 +5978,7 @@ public void testCountStarWithTimeInIntervalFilterInvalidInterval() @Test public void testCountStarWithTimeInIntervalFilterNonLiteral() { - notMsqCompatible(); + msqIncompatible(); testQueryThrows( "SELECT COUNT(*) FROM druid.foo " + "WHERE TIME_IN_INTERVAL(__time, dim1)", @@ -6559,7 +6559,7 @@ public void testTimeseriesWithTimeFilterOnLongColumnUsingMillisToTimestamp() @Test public void testCountDistinct() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT SUM(cnt), COUNT(distinct dim2), COUNT(distinct unique_dim1) FROM druid.foo", ImmutableList.of( @@ -6684,7 +6684,7 @@ public void testApproxCountDistinctWhenHllDisabled() { if (NullHandling.sqlCompatible()) { // Empty string isn't counted properly; see https://github.com/apache/druid/issues/13950 - notMsqCompatible(); + msqIncompatible(); } // When HLL is disabled, APPROX_COUNT_DISTINCT is still approximate. @@ -6722,7 +6722,7 @@ public void testApproxCountDistinctBuiltin() { if (NullHandling.sqlCompatible()) { // Empty string isn't counted properly; see https://github.com/apache/druid/issues/13950 - notMsqCompatible(); + msqIncompatible(); } testQuery( @@ -6921,7 +6921,7 @@ public void testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJo @Test public void testMultipleExactCountDistinctWithGroupingUsingGroupingSets() { - notMsqCompatible(); + msqIncompatible(); requireMergeBuffers(4); testQuery( PLANNER_CONFIG_NO_HLL.withOverrides( @@ -7004,7 +7004,7 @@ public void testMultipleExactCountDistinctWithGroupingUsingGroupingSets() @Test public void testApproxCountDistinct() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -7333,7 +7333,7 @@ public void testExactCountDistinctUsingSubquery() @Test public void testExactCountDistinctUsingSubqueryOnUnionAllTables() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + " SUM(cnt),\n" @@ -7485,7 +7485,7 @@ public void testQueryWithMoreThanMaxNumericInFilter() // skip in sql compatible mode, this plans to an OR filter with equality filter children... return; } - notMsqCompatible(); + msqIncompatible(); expectedException.expect(UOE.class); expectedException.expectMessage( "The number of values in the IN clause for [dim6] in query exceeds configured maxNumericFilter limit of [2] for INs. Cast [3] values of IN clause to String"); @@ -7746,7 +7746,7 @@ public void testCountDistinctArithmetic() { if (NullHandling.sqlCompatible()) { // Empty string isn't counted properly; see https://github.com/apache/druid/issues/13950 - notMsqCompatible(); + msqIncompatible(); } testQuery( @@ -8485,7 +8485,7 @@ public void testFilterOnTimeExtractWithMultipleDays() @Test public void testFilterOnTimeExtractWithVariousTimeUnits() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -8548,7 +8548,7 @@ public void testFilterOnTimeExtractWithVariousTimeUnits() @Test public void testFilterOnTimeFloorMisaligned() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COUNT(*) FROM druid.foo " + "WHERE floor(__time TO month) = TIMESTAMP '2000-01-01 00:00:01'", @@ -8595,7 +8595,7 @@ public void testGroupByFloor() @Test public void testQueryWithSelectProjectAndIdentityProjectDoesNotRename() { - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); skipVectorize(); requireMergeBuffers(3); @@ -9068,7 +9068,7 @@ public void testCountDistinctOfLookup() @Test public void testGroupByExpressionFromLookup() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize direct queries on lookup tables. cannotVectorize(); @@ -9454,7 +9454,7 @@ public void testTimeseriesLosAngelesUsingTimeFloorConnectionLosAngeles() @Test public void testTimeseriesDontSkipEmptyBuckets() { - notMsqCompatible(); + msqIncompatible(); // Tests that query context parameters are passed through to the underlying query engine. Long defaultVal = NullHandling.replaceWithDefault() ? 0L : null; testQuery( @@ -9594,7 +9594,7 @@ public void testTimeseriesDescending() @Test public void testTimeseriesEmptyResultsAggregatorDefaultValues() { - notMsqCompatible(); + msqIncompatible(); // timeseries with all granularity have a single group, so should return default results for given aggregators testQuery( "SELECT\n" @@ -9700,7 +9700,7 @@ public void testTimeseriesEmptyResultsAggregatorDefaultValues() @Test public void testTimeseriesEmptyResultsAggregatorDefaultValuesNonVectorized() { - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); skipVectorize(); // timeseries with all granularity have a single group, so should return default results for given aggregators @@ -10016,7 +10016,7 @@ public void testGroupByAggregatorDefaultValues() @Test public void testGroupByAggregatorDefaultValuesNonVectorized() { - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); skipVectorize(); testQuery( @@ -10641,7 +10641,7 @@ public void testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim() @Test public void testGroupingSets() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -10706,7 +10706,7 @@ public void testGroupingSets() @Test public void testGroupingAggregatorDifferentOrder() { - notMsqCompatible(); + msqIncompatible(); requireMergeBuffers(3); testQuery( @@ -10770,7 +10770,7 @@ public void testGroupingAggregatorDifferentOrder() @Test public void testGroupingAggregatorWithPostAggregator() { - notMsqCompatible(); + msqIncompatible(); List resultList; if (NullHandling.sqlCompatible()) { resultList = ImmutableList.of( @@ -10829,7 +10829,7 @@ public void testGroupingAggregatorWithPostAggregator() @Test public void testGroupingSetsWithNumericDimension() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT cnt, COUNT(*)\n" + "FROM foo\n" @@ -10860,7 +10860,7 @@ public void testGroupingSetsWithNumericDimension() @Test public void testGroupByRollup() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -10919,7 +10919,7 @@ public void testGroupByRollup() @Test public void testGroupByRollupDifferentOrder() { - notMsqCompatible(); + msqIncompatible(); // Like "testGroupByRollup", but the ROLLUP exprs are in the reverse order. testQuery( "SELECT dim2, gran, SUM(cnt)\n" @@ -10975,7 +10975,7 @@ public void testGroupByRollupDifferentOrder() @Test public void testGroupByCube() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -11037,7 +11037,7 @@ public void testGroupByCube() @Test public void testGroupingSetsWithDummyDimension() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -11099,7 +11099,7 @@ public void testGroupingSetsWithDummyDimension() @Test public void testGroupingSetsNoSuperset() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -11156,7 +11156,7 @@ public void testGroupingSetsNoSuperset() @Test public void testGroupingSetsWithOrderByDimension() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT dim2, gran, SUM(cnt)\n" + "FROM (SELECT FLOOR(__time TO MONTH) AS gran, COALESCE(dim2, '') dim2, cnt FROM druid.foo) AS x\n" @@ -11227,7 +11227,7 @@ public void testGroupingSetsWithOrderByDimension() @Test public void testGroupingSetsWithOrderByAggregator() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -11296,7 +11296,7 @@ public void testGroupingSetsWithOrderByAggregator() @Test public void testGroupingSetsWithOrderByAggregatorWithLimit() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -11562,7 +11562,7 @@ public void testProjectAfterSort2() + " org.apache.calcite.sql.validate.SqlValidatorException: Column 'dim1' is ambiguous") public void testProjectAfterSort3() { - notMsqCompatible(); + msqIncompatible(); testQuery( "select dim1 from (select dim1, dim1, count(*) cnt from druid.foo group by dim1, dim1 order by cnt)", ImmutableList.of( @@ -11601,7 +11601,7 @@ public void testProjectAfterSort3() @Test public void testProjectAfterSort3WithoutAmbiguity() { - notMsqCompatible(); + msqIncompatible(); // This query is equivalent to the one in testProjectAfterSort3 but renames the second grouping column // to avoid the ambiguous name exception. The inner sort is also optimized out in Calcite 1.21. testQuery( @@ -12150,7 +12150,7 @@ public void testRequireTimeConditionLogicalValuePositive() @Test public void testRequireTimeConditionSimpleQueryNegative() { - notMsqCompatible(); + msqIncompatible(); expectedException.expect(CannotBuildQueryException.class); expectedException.expectMessage("__time column"); @@ -12171,7 +12171,7 @@ public void testRequireTimeConditionSimpleQueryNegative() @Test public void testRequireTimeConditionSubQueryNegative() { - notMsqCompatible(); + msqIncompatible(); expectedException.expect(CannotBuildQueryException.class); expectedException.expectMessage("__time column"); @@ -12191,7 +12191,7 @@ public void testRequireTimeConditionSubQueryNegative() @Test public void testRequireTimeConditionSemiJoinNegative() { - notMsqCompatible(); + msqIncompatible(); expectedException.expect(CannotBuildQueryException.class); expectedException.expectMessage("__time column"); @@ -12277,7 +12277,7 @@ public void testFilterLongDimension() @Test public void testTrigonometricFunction() { - notMsqCompatible(); + msqIncompatible(); testQuery( PLANNER_CONFIG_DEFAULT, QUERY_CONTEXT_DEFAULT, @@ -12576,7 +12576,7 @@ public void testLeftRightStringOperators() @Test public void testQueryContextOuterLimit() { - notMsqCompatible(); + msqIncompatible(); Map outerLimitContext = new HashMap<>(QUERY_CONTEXT_DEFAULT); outerLimitContext.put(PlannerContext.CTX_SQL_OUTER_LIMIT, 4); @@ -12887,7 +12887,7 @@ public void testTimeStampAddConversion() @Test public void testGroupingSetsWithLimit() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT dim2, gran, SUM(cnt)\n" + "FROM (SELECT FLOOR(__time TO MONTH) AS gran, COALESCE(dim2, '') dim2, cnt FROM druid.foo) AS x\n" @@ -12952,7 +12952,7 @@ public void testGroupingSetsWithLimit() @Test public void testGroupingSetsWithLimitOrderByGran() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -14276,7 +14276,7 @@ public void testGreatestFunctionForNumberWithIsNull() @Test public void testGreatestFunctionForStringWithIsNull() { - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); String query = "SELECT l1, LATEST(GREATEST(dim1, dim2)) IS NULL FROM druid.numfoo GROUP BY l1"; @@ -14367,7 +14367,7 @@ public void testSubqueryTypeMismatchWithLiterals() public void testTimeseriesQueryWithEmptyInlineDatasourceAndGranularity() { // TODO(gianm): this test does not actually test the below thing, b/c the timestamp_floor got baked in - notMsqCompatible(); + msqIncompatible(); //msqCompatible(); // the SQL query contains an always FALSE filter ('bar' = 'baz'), which optimizes the query to also remove time @@ -14457,7 +14457,7 @@ public void testComplexDecode() @Test public void testComplexDecodeAgg() { - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); testQuery( "SELECT APPROX_COUNT_DISTINCT_BUILTIN(COMPLEX_DECODE_BASE64('hyperUnique',PARSE_JSON(TO_JSON_STRING(unique_dim1)))) from druid.foo", diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java index 7b42596d81a3..6ee790e1f582 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java @@ -54,6 +54,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assume.assumeThat; +import static org.junit.Assume.assumeTrue; /** * These tests are file-based, look in resources -> calcite/tests/window for the set of test specifications. @@ -101,6 +102,7 @@ public CalciteWindowQueryTest( @SuppressWarnings("unchecked") public void windowQueryTest() throws IOException { + assumeTrue("These tests are only run in sqlCompatible mode!", NullHandling.sqlCompatible()); final Function stringManipulator; if (NullHandling.sqlCompatible()) { stringManipulator = s -> "".equals(s) ? null : s; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java index 963e1e0b23bc..f9d82e71dd7e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java @@ -321,10 +321,15 @@ public static Pair> getResults( public static class VerifyResults implements QueryVerifyStep { protected final BaseExecuteQuery execStep; + protected final boolean verifyRowSignature; - public VerifyResults(BaseExecuteQuery execStep) + public VerifyResults( + BaseExecuteQuery execStep, + boolean verifyRowSignature + ) { this.execStep = execStep; + this.verifyRowSignature = verifyRowSignature; } @Override @@ -346,7 +351,9 @@ private void verifyResults(QueryResults queryResults) } QueryTestBuilder builder = execStep.builder(); - builder.expectedResultsVerifier.verifyRowSignature(queryResults.signature); + if (verifyRowSignature) { + builder.expectedResultsVerifier.verifyRowSignature(queryResults.signature); + } builder.expectedResultsVerifier.verify(builder.sql, results); } } @@ -667,7 +674,9 @@ public QueryTestRunner(QueryTestBuilder builder) verifySteps.add(new VerifyNativeQueries(finalExecStep)); } if (builder.expectedResultsVerifier != null) { - verifySteps.add(new VerifyResults(finalExecStep)); + // Don't verify the row signature when MSQ is running, since the broker receives the task id, and the signature + // would be {TASK:STRING} instead of the expected results signature + verifySteps.add(new VerifyResults(finalExecStep, !config.isRunningMSQ())); } if (!builder.customVerifications.isEmpty()) { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRuleTest.java index 73054f506a29..ee2e4273f83b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRuleTest.java @@ -20,7 +20,6 @@ package org.apache.druid.sql.calcite.rule; import com.google.common.collect.ImmutableList; -import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexLiteral; @@ -33,6 +32,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.sql.calcite.planner.DruidTypeSystem; import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.junit.Assert; @@ -44,10 +44,8 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -import org.mockito.ArgumentMatchers; import org.mockito.Mockito; -import java.lang.reflect.Field; import java.math.BigDecimal; @RunWith(Enclosed.class) @@ -56,7 +54,7 @@ public class DruidLogicalValuesRuleTest private static final PlannerContext DEFAULT_CONTEXT = Mockito.mock(PlannerContext.class); @RunWith(Parameterized.class) - public static class GetValueFromLiteralSimpleTypesTest + public static class GetValueFromLiteralSimpleTypesTest extends InitializedNullHandlingTest { @Parameters(name = "{1}, {2}") public static Iterable constructorFeeder() @@ -88,7 +86,7 @@ public GetValueFromLiteralSimpleTypesTest(Comparable val, SqlTypeName sqlType @Test public void testGetValueFromLiteral() { - final RexLiteral literal = makeLiteral(val, sqlTypeName, javaType); + final RexLiteral literal = Mockito.spy(makeLiteral(val, sqlTypeName, javaType)); final Object fromLiteral = DruidLogicalValuesRule.getValueFromLiteral(literal, DEFAULT_CONTEXT); Assert.assertSame(javaType, fromLiteral.getClass()); Assert.assertEquals(val, fromLiteral); @@ -97,20 +95,11 @@ public void testGetValueFromLiteral() private static RexLiteral makeLiteral(Comparable val, SqlTypeName typeName, Class javaType) { - RelDataType dataType = Mockito.mock(RelDataType.class); - Mockito.when(dataType.getSqlTypeName()).thenReturn(typeName); - RexLiteral literal = Mockito.mock(RexLiteral.class); - try { - Field field = literal.getClass().getSuperclass().getDeclaredField("value"); - field.setAccessible(true); - field.set(literal, val); - } - catch (Exception e) { - Assert.fail("Unable to mock the literal for test.\nException: " + e); - } - Mockito.when(literal.getType()).thenReturn(dataType); - Mockito.when(literal.getValueAs(ArgumentMatchers.any())).thenReturn(javaType.cast(val)); - return literal; + return (RexLiteral) new RexBuilder(DruidTypeSystem.TYPE_FACTORY).makeLiteral( + typeName == SqlTypeName.DECIMAL && val != null ? new BigDecimal(String.valueOf(val)) : val, + DruidTypeSystem.TYPE_FACTORY.createSqlType(typeName), + false + ); } } diff --git a/sql/src/test/resources/calcite/tests/window/lead_lag.sqlTest b/sql/src/test/resources/calcite/tests/window/lead_lag.sqlTest new file mode 100644 index 000000000000..1ce080b08b69 --- /dev/null +++ b/sql/src/test/resources/calcite/tests/window/lead_lag.sqlTest @@ -0,0 +1,50 @@ +type: "operatorValidation" + +sql: | + SELECT + dim1, + LAG(dim1,2) OVER (), + LAG(dim1) OVER (), + LAG(dim1,0) OVER (), + LEAD(dim1,0) OVER (), + LEAD(dim1) OVER (), + LEAD(dim1,2) OVER () + FROM foo + WHERE length(dim1) > 1 + GROUP BY dim1 + +expectedOperators: + - type: "naivePartition" + partitionColumns: [] + - type: "window" + processor: + type: "composing" + processors: + - type: "offset" + inputColumn: "d0" + outputColumn: "w0" + offset: -2 + - type: "offset" + inputColumn: "d0" + outputColumn: "w1" + offset: -1 + - type: "offset" + inputColumn: "d0" + outputColumn: "w2" + offset: 0 + - type: "offset" + inputColumn: "d0" + outputColumn: "w3" + offset: 0 + - type: "offset" + inputColumn: "d0" + outputColumn: "w4" + offset: 1 + - type: "offset" + inputColumn: "d0" + outputColumn: "w5" + offset: 2 +expectedResults: + - ["10.1",null,null,"10.1","10.1","abc","def"] + - ["abc",null,"10.1","abc","abc","def",null] + - ["def","10.1","abc","def","def",null,null] diff --git a/sql/src/test/resources/calcite/tests/window/no_grouping.sqlTest b/sql/src/test/resources/calcite/tests/window/no_grouping.sqlTest new file mode 100644 index 000000000000..5e8c31e69fff --- /dev/null +++ b/sql/src/test/resources/calcite/tests/window/no_grouping.sqlTest @@ -0,0 +1,29 @@ +type: "operatorValidation" + +sql: | + SELECT + m1, + COUNT(m1) OVER () cc + FROM druid.foo + +expectedOperators: + - type: "naivePartition" + partitionColumns: [] + - type: "window" + processor: + type: "framedAgg" + frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + aggregations: + - type: "filtered" + aggregator: {"type":"count","name":"w0"} + filter: + type: not + field: {"type":"null","column":"m1"} + name: null +expectedResults: + - [1.0,6] + - [2.0,6] + - [3.0,6] + - [4.0,6] + - [5.0,6] + - [6.0,6]