diff --git a/.github/workflows/reusable-unit-tests.yml b/.github/workflows/reusable-unit-tests.yml index 291cb648b3e..3634da0e200 100644 --- a/.github/workflows/reusable-unit-tests.yml +++ b/.github/workflows/reusable-unit-tests.yml @@ -117,6 +117,28 @@ jobs: MAVEN_PROJECTS: ${{ inputs.maven_projects }} run: ./.github/scripts/unit_tests_script.sh + - name: Check for .hprof files on failure + if: ${{ failure() }} + id: check_for_heap_dump + run: | + if ls ${GITHUB_WORKSPACE}/target/*.hprof 1> /dev/null 2>&1; then + echo "found_hprof=true" >> "$GITHUB_ENV" + else + echo "found_hprof=false" >> "$GITHUB_ENV" + fi + + - name: Collect tarball hprof dumps if they exist on failure + if: ${{ failure() && env.found_hprof == 'true' }} + run: | + tar cvzf ${RUNNER_TEMP}/hprof-dumps.tgz ${GITHUB_WORKSPACE}/target/*.hprof + + - name: Upload hprof dumps to GitHub if they exist on failure + if: ${{ failure() && env.found_hprof == 'true' }} + uses: actions/upload-artifact@master + with: + name: Hprof-${{ inputs.group }} hprof dumps (Compile=jdk${{ inputs.build_jdk }}, Run=jdk${{ inputs.runtime_jdk }}) + path: ${{ runner.temp }}/hprof-dumps.tgz + - name: set outputs on failure id: set_outputs if: ${{ failure() }} diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml index 84b2ddf0d13..f2ffbe247ec 100644 --- a/benchmarks/pom.xml +++ b/benchmarks/pom.xml @@ -27,7 +27,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/DelimitedInputFormatBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/DelimitedInputFormatBenchmark.java index cabd58a4b7e..008dcd9600a 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/DelimitedInputFormatBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/DelimitedInputFormatBenchmark.java @@ -132,7 +132,7 @@ public class DelimitedInputFormatBenchmark @Setup(Level.Trial) public void prepareFormat() { - format = new DelimitedInputFormat(fromHeader ? null : COLUMNS, null, "\t", null, fromHeader, fromHeader ? 0 : 1); + format = new DelimitedInputFormat(fromHeader ? null : COLUMNS, null, "\t", null, fromHeader, fromHeader ? 0 : 1, null); } @Benchmark diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java index d65d13a324d..cda80475f53 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java @@ -36,7 +36,7 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.generator.GeneratorColumnSchema; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -161,9 +161,8 @@ public class ExpressionAggregationBenchmark private double compute(final Function aggregatorFactory) { - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); - - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final BufferAggregator bufferAggregator = aggregatorFactory.apply(cursor.getColumnSelectorFactory()); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java index 06f0952b51f..6672edc5c42 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java @@ -35,7 +35,7 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.generator.GeneratorColumnSchema; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -148,7 +148,9 @@ public class ExpressionFilterBenchmark final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(expressionFilter.toFilter()) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); @@ -166,7 +168,9 @@ public class ExpressionFilterBenchmark final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(nativeFilter.toFilter()) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x"); while (!cursor.isDone()) { diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java index 8d79a611d0e..f4dce839cd5 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java @@ -42,9 +42,8 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; @@ -160,7 +159,8 @@ public class ExpressionSelectorBenchmark ) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); @@ -171,7 +171,8 @@ public class ExpressionSelectorBenchmark @Benchmark public void timeFloorUsingExtractionFn(Blackhole blackhole) { - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor @@ -190,15 +191,15 @@ public class ExpressionSelectorBenchmark @Benchmark public void timeFloorUsingCursor(Blackhole blackhole) { - final StorageAdapter adapter = new QueryableIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final CursorGranularizer granularizer = CursorGranularizer.create( cursor, QueryableIndexTimeBoundaryInspector.create(index), Cursors.getTimeOrdering(index.getOrdering()), Granularities.HOUR, - adapter.getInterval() + index.getDataInterval() ); final Sequence results = Sequences.simple(granularizer.getBucketIterable()) @@ -241,7 +242,8 @@ public class ExpressionSelectorBenchmark ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor.getColumnSelectorFactory().makeDimensionSelector( DefaultDimensionSpec.of("v") @@ -253,7 +255,8 @@ public class ExpressionSelectorBenchmark @Benchmark public void timeFormatUsingExtractionFn(Blackhole blackhole) { - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() @@ -284,7 +287,8 @@ public class ExpressionSelectorBenchmark ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -307,7 +311,8 @@ public class ExpressionSelectorBenchmark ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() @@ -320,7 +325,8 @@ public class ExpressionSelectorBenchmark @Benchmark public void strlenUsingExtractionFn(Blackhole blackhole) { - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() @@ -346,7 +352,8 @@ public class ExpressionSelectorBenchmark ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -368,7 +375,8 @@ public class ExpressionSelectorBenchmark ) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -390,7 +398,8 @@ public class ExpressionSelectorBenchmark ) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -412,7 +421,8 @@ public class ExpressionSelectorBenchmark ) ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -447,7 +457,8 @@ public class ExpressionSelectorBenchmark ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -476,7 +487,8 @@ public class ExpressionSelectorBenchmark ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); @@ -513,7 +525,8 @@ public class ExpressionSelectorBenchmark ) .build(); - try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v"); consumeLong(cursor, selector, blackhole); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java index 5604f7fe883..2e9ffb644d5 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java @@ -35,7 +35,7 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -155,7 +155,7 @@ public class ExpressionVectorSelectorBenchmark .setVirtualColumns(virtualColumns) .build(); final CursorHolder cursorHolder = closer.register( - new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec) + new QueryableIndexCursorFactory(index).makeCursorHolder(buildSpec) ); if (vectorize) { VectorCursor cursor = cursorHolder.asVectorCursor(); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java index 65dde3cb65a..2c0f2690198 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java @@ -46,14 +46,14 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.IndexedInts; @@ -231,8 +231,8 @@ public class FilterPartitionBenchmark @OutputTimeUnit(TimeUnit.MICROSECONDS) public void stringRead(Blackhole blackhole) { - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, null)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, null)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -243,8 +243,8 @@ public class FilterPartitionBenchmark @OutputTimeUnit(TimeUnit.MICROSECONDS) public void longRead(Blackhole blackhole) { - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, null)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, null)) { final Cursor cursor = cursorHolder.asCursor(); readCursorLong(cursor, blackhole); } @@ -255,8 +255,8 @@ public class FilterPartitionBenchmark @OutputTimeUnit(TimeUnit.MICROSECONDS) public void timeFilterNone(Blackhole blackhole) { - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterNone)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, timeFilterNone)) { final Cursor cursor = cursorHolder.asCursor(); readCursorLong(cursor, blackhole); } @@ -267,8 +267,8 @@ public class FilterPartitionBenchmark @OutputTimeUnit(TimeUnit.MICROSECONDS) public void timeFilterHalf(Blackhole blackhole) { - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterHalf)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, timeFilterHalf)) { final Cursor cursor = cursorHolder.asCursor(); readCursorLong(cursor, blackhole); } @@ -279,8 +279,8 @@ public class FilterPartitionBenchmark @OutputTimeUnit(TimeUnit.MICROSECONDS) public void timeFilterAll(Blackhole blackhole) { - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterAll)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, timeFilterAll)) { final Cursor cursor = cursorHolder.asCursor(); readCursorLong(cursor, blackhole); } @@ -293,8 +293,8 @@ public class FilterPartitionBenchmark { Filter filter = new SelectorFilter("dimSequential", "199"); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, filter)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -307,8 +307,8 @@ public class FilterPartitionBenchmark { Filter filter = new NoBitmapSelectorFilter("dimSequential", "199"); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, filter)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -321,8 +321,8 @@ public class FilterPartitionBenchmark { Filter filter = new SelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter(); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, filter)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -335,8 +335,8 @@ public class FilterPartitionBenchmark { Filter filter = new NoBitmapSelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter(); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, filter)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -354,8 +354,8 @@ public class FilterPartitionBenchmark ) ); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, andFilter)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, andFilter)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -370,8 +370,8 @@ public class FilterPartitionBenchmark Filter filter2 = new AndFilter(Arrays.asList(new SelectorFilter("dimMultivalEnumerated2", "Corundum"), new NoBitmapSelectorFilter("dimMultivalEnumerated", "Bar"))); Filter orFilter = new OrFilter(Arrays.asList(filter, filter2)); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, orFilter)) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, orFilter)) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -386,8 +386,8 @@ public class FilterPartitionBenchmark Filter filter2 = new AndFilter(Arrays.asList(new SelectorFilter("dimMultivalEnumerated2", "Corundum"), new NoBitmapSelectorFilter("dimMultivalEnumerated", "Bar"))); Filter orFilter = new OrFilter(Arrays.asList(filter, filter2)); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, Filters.toCnf(orFilter))) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, Filters.toCnf(orFilter))) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -425,8 +425,8 @@ public class FilterPartitionBenchmark )) ); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, dimFilter3.toFilter())) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, dimFilter3.toFilter())) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } @@ -464,16 +464,16 @@ public class FilterPartitionBenchmark )) ); - StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex); - try (final CursorHolder cursorHolder = makeCursorHolder(sa, Filters.toCnf(dimFilter3.toFilter()))) { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex); + try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, Filters.toCnf(dimFilter3.toFilter()))) { final Cursor cursor = cursorHolder.asCursor(); readCursor(cursor, blackhole); } } - private CursorHolder makeCursorHolder(StorageAdapter sa, Filter filter) + private CursorHolder makeCursorHolder(CursorFactory factory, Filter filter) { - return sa.makeCursorHolder( + return factory.makeCursorHolder( CursorBuildSpec.builder() .setFilter(filter) .setInterval(schemaInfo.getDataInterval()) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java index 95d59856395..bbff131e867 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -378,7 +378,6 @@ public class GroupByTypeInterfaceBenchmark final GroupingEngine groupingEngine = new GroupingEngine( druidProcessingConfig, configSupplier, - bufferPool, groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), @@ -387,7 +386,8 @@ public class GroupByTypeInterfaceBenchmark factory = new GroupByQueryRunnerFactory( groupingEngine, - new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool) + new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool), + bufferPool ); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java index 952e9b188fe..e654b033dee 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java @@ -238,7 +238,7 @@ public class IndexedTableJoinCursorBenchmark private CursorHolder makeCursorHolder() { - return hashJoinSegment.asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN); + return hashJoinSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java index c66ef6baf0a..d9a98b467d8 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java @@ -369,7 +369,7 @@ public class JoinAndLookupBenchmark @OutputTimeUnit(TimeUnit.MILLISECONDS) public void baseSegment(Blackhole blackhole) { - try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = baseSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "countryIsoCode")); } @@ -384,7 +384,7 @@ public class JoinAndLookupBenchmark final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(filter) .build(); - try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = baseSegment.asCursorFactory().makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "countryIsoCode")); } @@ -395,7 +395,7 @@ public class JoinAndLookupBenchmark @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinLookupStringKey(Blackhole blackhole) { - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.v")); @@ -411,7 +411,7 @@ public class JoinAndLookupBenchmark final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(filter) .build(); - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.v")); @@ -423,7 +423,7 @@ public class JoinAndLookupBenchmark @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinLookupLongKey(Blackhole blackhole) { - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.v")); @@ -439,7 +439,7 @@ public class JoinAndLookupBenchmark final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(filter) .build(); - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.v")); @@ -451,7 +451,7 @@ public class JoinAndLookupBenchmark @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinIndexedTableLongKey(Blackhole blackhole) { - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.countryName")); @@ -467,7 +467,7 @@ public class JoinAndLookupBenchmark final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(filter) .build(); - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.countryName")); @@ -479,7 +479,7 @@ public class JoinAndLookupBenchmark @OutputTimeUnit(TimeUnit.MILLISECONDS) public void joinIndexedTableStringKey(Blackhole blackhole) { - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.countryName")); @@ -495,7 +495,7 @@ public class JoinAndLookupBenchmark final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(filter) .build(); - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, "c.countryName")); @@ -510,7 +510,7 @@ public class JoinAndLookupBenchmark final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setVirtualColumns(lookupVirtualColumns) .build(); - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME)); @@ -527,7 +527,7 @@ public class JoinAndLookupBenchmark .setFilter(filter) .setVirtualColumns(lookupVirtualColumns) .build(); - try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter() + try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory() .makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME)); @@ -542,7 +542,7 @@ public class JoinAndLookupBenchmark final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setVirtualColumns(lookupVirtualColumns) .build(); - try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = baseSegment.asCursorFactory().makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_NUMBER_TO_NAME)); } @@ -558,7 +558,7 @@ public class JoinAndLookupBenchmark .setVirtualColumns(lookupVirtualColumns) .setFilter(filter) .build(); - try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = baseSegment.asCursorFactory().makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_NUMBER_TO_NAME)); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/compression/LongCompressionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/compression/LongCompressionBenchmark.java index e059c8a7ea7..e76cfc5b425 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/compression/LongCompressionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/compression/LongCompressionBenchmark.java @@ -23,7 +23,7 @@ import com.google.common.base.Supplier; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.MappedByteBufferHandler; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.data.ColumnarLongs; import org.apache.druid.segment.data.CompressedColumnarLongsSupplier; import org.openjdk.jmh.annotations.Benchmark; @@ -118,7 +118,7 @@ public class LongCompressionBenchmark @Benchmark public void readVectorizedContinuous(Blackhole bh) { - long[] vector = new long[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE]; + long[] vector = new long[QueryContexts.DEFAULT_VECTOR_SIZE]; ColumnarLongs columnarLongs = supplier.get(); int count = columnarLongs.size(); for (int i = 0; i < count; i++) { diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java index a8b90e56eea..25f9015de2b 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java @@ -21,6 +21,7 @@ package org.apache.druid.benchmark.frame; import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.guava.FutureUtils; @@ -203,6 +204,7 @@ public class FrameChannelMergerBenchmark private final List sortKey = ImmutableList.of(new KeyColumn(KEY, KeyOrder.ASCENDING)); private List> channelFrames; + private ListeningExecutorService innerExec; private FrameProcessorExecutor exec; private List channels; @@ -226,7 +228,7 @@ public class FrameChannelMergerBenchmark frameReader = FrameReader.create(signature); exec = new FrameProcessorExecutor( - MoreExecutors.listeningDecorator( + innerExec = MoreExecutors.listeningDecorator( Execs.singleThreaded(StringUtils.encodeForFormat(getClass().getSimpleName())) ) ); @@ -284,7 +286,7 @@ public class FrameChannelMergerBenchmark signature ); final Sequence frameSequence = - FrameSequenceBuilder.fromAdapter(segment.asStorageAdapter()) + FrameSequenceBuilder.fromCursorFactory(segment.asCursorFactory()) .allocator(ArenaMemoryAllocator.createOnHeap(10_000_000)) .frameType(FrameType.ROW_BASED) .frames(); @@ -335,8 +337,8 @@ public class FrameChannelMergerBenchmark @TearDown(Level.Trial) public void tearDown() throws Exception { - exec.getExecutorService().shutdownNow(); - if (!exec.getExecutorService().awaitTermination(1, TimeUnit.MINUTES)) { + innerExec.shutdownNow(); + if (!innerExec.awaitTermination(1, TimeUnit.MINUTES)) { throw new ISE("Could not terminate executor after 1 minute"); } } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java index 6f0796f412f..56a41e0fb9b 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java @@ -35,6 +35,7 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.search.ContainsSearchQuerySpec; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.data.IndexedInts; @@ -44,8 +45,8 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexCreator; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.serde.ComplexMetrics; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -147,8 +148,8 @@ public class IncrementalIndexReadBenchmark @OutputTimeUnit(TimeUnit.MICROSECONDS) public void read(Blackhole blackhole) { - IncrementalIndexStorageAdapter sa = new IncrementalIndexStorageAdapter(incIndex); - try (final CursorHolder cursorHolder = makeCursor(sa, null)) { + final CursorFactory cursorFactory = new IncrementalIndexCursorFactory(incIndex); + try (final CursorHolder cursorHolder = makeCursor(cursorFactory, null)) { Cursor cursor = cursorHolder.asCursor(); List selectors = new ArrayList<>(); @@ -183,8 +184,8 @@ public class IncrementalIndexReadBenchmark ) ); - IncrementalIndexStorageAdapter sa = new IncrementalIndexStorageAdapter(incIndex); - try (final CursorHolder cursorHolder = makeCursor(sa, filter)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(incIndex); + try (final CursorHolder cursorHolder = makeCursor(cursorFactory, filter)) { Cursor cursor = cursorHolder.asCursor(); List selectors = new ArrayList<>(); @@ -204,14 +205,14 @@ public class IncrementalIndexReadBenchmark } } - private CursorHolder makeCursor(IncrementalIndexStorageAdapter sa, DimFilter filter) + private CursorHolder makeCursor(CursorFactory factory, DimFilter filter) { CursorBuildSpec.CursorBuildSpecBuilder builder = CursorBuildSpec.builder() .setInterval(schemaInfo.getDataInterval()); if (filter != null) { builder.setFilter(filter.toFilter()); } - return sa.makeCursorHolder(builder.build()); + return factory.makeCursorHolder(builder.build()); } private static DimensionSelector makeDimensionSelector(Cursor cursor, String name) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 24afa1e8477..8e0715e0fe5 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -362,14 +362,13 @@ public class CachingClusteredClientBenchmark final GroupingEngine groupingEngine = new GroupingEngine( processingConfig, configSupplier, - bufferPool, groupByResourcesReservationPool, mapper, mapper, QueryRunnerTestHelper.NOOP_QUERYWATCHER ); final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool); - return new GroupByQueryRunnerFactory(groupingEngine, toolChest); + return new GroupByQueryRunnerFactory(groupingEngine, toolChest, bufferPool); } @TearDown(Level.Trial) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java index e7220cc286d..5ab19b6235f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java @@ -495,7 +495,6 @@ public class GroupByBenchmark final GroupingEngine groupingEngine = new GroupingEngine( druidProcessingConfig, configSupplier, - bufferPool, groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), @@ -504,7 +503,8 @@ public class GroupByBenchmark factory = new GroupByQueryRunnerFactory( groupingEngine, - new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool) + new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool), + bufferPool ); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index 06ea2aa9e19..0e38757f12b 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -57,8 +57,8 @@ import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.column.StringEncodingStrategy; import org.apache.druid.segment.data.FrontCodedIndexed; import org.apache.druid.segment.generator.GeneratorBasicSchemas; @@ -694,8 +694,8 @@ public class SqlBenchmark } else if (STORAGE_FRAME_ROW.equals(storageType)) { walker.add( descriptor, - FrameTestUtil.adapterToFrameSegment( - new QueryableIndexStorageAdapter(index), + FrameTestUtil.cursorFactoryToFrameSegment( + new QueryableIndexCursorFactory(index), FrameType.ROW_BASED, descriptor.getId() ) @@ -703,8 +703,8 @@ public class SqlBenchmark } else if (STORAGE_FRAME_COLUMNAR.equals(storageType)) { walker.add( descriptor, - FrameTestUtil.adapterToFrameSegment( - new QueryableIndexStorageAdapter(index), + FrameTestUtil.cursorFactoryToFrameSegment( + new QueryableIndexCursorFactory(index), FrameType.COLUMNAR, descriptor.getId() ) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java index 2b5db7413a2..977e8e8e42c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java @@ -55,8 +55,8 @@ import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.column.StringEncodingStrategy; import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -159,6 +159,12 @@ public class SqlWindowFunctionsBenchmark { return 3; } + + @Override + public int intermediateComputeSizeBytes() + { + return 200_000_000; + } }; @Setup(Level.Trial) @@ -281,8 +287,8 @@ public class SqlWindowFunctionsBenchmark } else if (STORAGE_FRAME_ROW.equals(storageType)) { walker.add( descriptor, - FrameTestUtil.adapterToFrameSegment( - new QueryableIndexStorageAdapter(index), + FrameTestUtil.cursorFactoryToFrameSegment( + new QueryableIndexCursorFactory(index), FrameType.ROW_BASED, descriptor.getId() ) @@ -290,8 +296,8 @@ public class SqlWindowFunctionsBenchmark } else if (STORAGE_FRAME_COLUMNAR.equals(storageType)) { walker.add( descriptor, - FrameTestUtil.adapterToFrameSegment( - new QueryableIndexStorageAdapter(index), + FrameTestUtil.cursorFactoryToFrameSegment( + new QueryableIndexCursorFactory(index), FrameType.COLUMNAR, descriptor.getId() ) @@ -336,7 +342,8 @@ public class SqlWindowFunctionsBenchmark { final Map context = ImmutableMap.of( PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.MAX_SUBQUERY_BYTES_KEY, "auto" + QueryContexts.MAX_SUBQUERY_BYTES_KEY, "disabled", + QueryContexts.MAX_SUBQUERY_ROWS_KEY, -1 ); try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, sql, context)) { final PlannerResult plannerResult = planner.plan(); @@ -420,4 +427,15 @@ public class SqlWindowFunctionsBenchmark + "GROUP BY dimUniform, dimSequential"; querySql(sql, blackhole); } + + @Benchmark + public void windowWithGroupbyTime(Blackhole blackhole) + { + String sql = "SELECT " + + "SUM(dimSequentialHalfNull) + SUM(dimHyperUnique), " + + "LAG(SUM(dimSequentialHalfNull + dimHyperUnique)) OVER (PARTITION BY dimUniform ORDER BY dimSequential) " + + "FROM foo " + + "GROUP BY __time, dimUniform, dimSequential"; + querySql(sql, blackhole); + } } diff --git a/cloud/aws-common/pom.xml b/cloud/aws-common/pom.xml index a41e21380e3..148a6dec54e 100644 --- a/cloud/aws-common/pom.xml +++ b/cloud/aws-common/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/cloud/gcp-common/pom.xml b/cloud/gcp-common/pom.xml index 3819cf1a04d..d3ba20edd3f 100644 --- a/cloud/gcp-common/pom.xml +++ b/cloud/gcp-common/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/dev/chmod-heap-dumps.sh b/dev/chmod-heap-dumps.sh new file mode 100755 index 00000000000..dbb9582224b --- /dev/null +++ b/dev/chmod-heap-dumps.sh @@ -0,0 +1,19 @@ +#!/bin/bash -eux + +# 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. + +BASE_DIR=$(git rev-parse --show-toplevel) +chmod 644 ${BASE_DIR}/target/*.hprof diff --git a/distribution/pom.xml b/distribution/pom.xml index 136935433e9..5aa71130f77 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -30,7 +30,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT @@ -458,6 +458,8 @@ org.apache.druid.extensions.contrib:druid-spectator-histogram -c org.apache.druid.extensions.contrib:druid-rabbit-indexing-service + -c + org.apache.druid.extensions.contrib:grpc-query diff --git a/docs/assets/druid-architecture.svg b/docs/assets/druid-architecture.svg index 8db6ec25d4a..9d0e67188fd 100644 --- a/docs/assets/druid-architecture.svg +++ b/docs/assets/druid-architecture.svg @@ -16,4 +16,4 @@ ~ specific language governing permissions and limitations ~ under the License. --> - \ No newline at end of file + \ No newline at end of file diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 64797c402c0..5fd31366ad0 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -616,9 +616,10 @@ the [HDFS input source](../ingestion/input-sources.md#hdfs-input-source). You can set the following property to specify permissible protocols for the [HTTP input source](../ingestion/input-sources.md#http-input-source). -|Property|Possible values|Description|Default| -|--------|---------------|-----------|-------| -|`druid.ingestion.http.allowedProtocols`|List of protocols|Allowed protocols for the HTTP input source.|`["http", "https"]`| +|Property| Possible values | Description |Default| +|--------|------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------|-------| +|`druid.ingestion.http.allowedProtocols`| List of protocols | Allowed protocols for the HTTP input source. |`["http", "https"]`| +|`druid.ingestion.http.allowedHeaders`| A list of permitted request headers for the HTTP input source. By default, the list is empty, which means no headers are allowed in the ingestion specification. |`[]`| ### External data access security configuration @@ -1795,6 +1796,7 @@ This strategy can be enabled by setting `druid.query.scheduler.prioritization.st |`druid.query.scheduler.prioritization.periodThreshold`|ISO duration threshold for how old data can be queried before automatically adjusting query priority.|none| |`druid.query.scheduler.prioritization.durationThreshold`|ISO duration threshold for maximum duration a queries interval can span before the priority is automatically adjusted.|none| |`druid.query.scheduler.prioritization.segmentCountThreshold`|Number threshold for maximum number of segments that can take part in a query before its priority is automatically adjusted.|none| +|`druid.query.scheduler.prioritization.segmentRangeThreshold`|ISO duration threshold for maximum segment range a query can span before the priority is automatically adjusted.|none| |`druid.query.scheduler.prioritization.adjustment`|Amount to reduce the priority of queries which cross any threshold.|none| ##### Laning strategies diff --git a/docs/development/extensions-contrib/delta-lake.md b/docs/development/extensions-contrib/delta-lake.md index 503fbfdc55d..88f3a2c77f3 100644 --- a/docs/development/extensions-contrib/delta-lake.md +++ b/docs/development/extensions-contrib/delta-lake.md @@ -51,9 +51,4 @@ java \ -c "org.apache.druid.extensions.contrib:druid-deltalake-extensions:" ``` -See [Loading community extensions](../../configuration/extensions.md#loading-community-extensions) for more information. - -## Known limitations - -This extension relies on the Delta Kernel API and can only read from the latest Delta table snapshot. Ability to read from -arbitrary snapshots is tracked [here](https://github.com/delta-io/delta/issues/2581). \ No newline at end of file +See [Loading community extensions](../../configuration/extensions.md#loading-community-extensions) for more information. \ No newline at end of file diff --git a/docs/development/extensions-core/druid-lookups.md b/docs/development/extensions-core/druid-lookups.md index d6219b8c742..06283ec4d72 100644 --- a/docs/development/extensions-core/druid-lookups.md +++ b/docs/development/extensions-core/druid-lookups.md @@ -31,9 +31,9 @@ This module can be used side to side with other lookup module like the global ca To use this Apache Druid extension, [include](../../configuration/extensions.md#loading-extensions) `druid-lookups-cached-single` in the extensions load list. :::info - If using JDBC, you will need to add your database's client JAR files to the extension's directory. +To use JDBC, you must add your database client JAR files to the extension's directory. For Postgres, the connector JAR is already included. - See the MySQL extension documentation for instructions to obtain [MySQL](./mysql.md#installing-the-mysql-connector-library) or [MariaDB](./mysql.md#alternative-installing-the-mariadb-connector-library) connector libraries. + See the MySQL extension documentation for instructions to obtain [MySQL](./mysql.md#install-mysql-connectorj) or [MariaDB](./mysql.md#install-mariadb-connectorj) connector libraries. Copy or symlink the downloaded file to `extensions/druid-lookups-cached-single` under the distribution root directory. ::: diff --git a/docs/development/extensions-core/mysql.md b/docs/development/extensions-core/mysql.md index bc6012dbb5a..a3678f65056 100644 --- a/docs/development/extensions-core/mysql.md +++ b/docs/development/extensions-core/mysql.md @@ -1,6 +1,6 @@ --- id: mysql -title: "MySQL Metadata Store" +title: "MySQL metadata store" --- -This page provides a reference of Apache Druid® SQL functions in alphabetical order. For more details on a function, refer to the following: -* [Aggregation functions](sql-aggregations.md) -* [Array functions](sql-array-functions.md) -* [JSON functions](sql-json-functions.md) -* [Multi-value string functions](sql-multivalue-string-functions.md) -* [Scalar functions](sql-scalar.md) -* [Window functions](sql-window-functions.md) -The examples on this page use the following example datasources: -* `flight-carriers` using `FlightCarrierOnTime (1 month)` -* `taxi-trips` using `NYC Taxi cabs (3 files)` + +This page provides a reference of all Druid SQL functions in alphabetical order. +Click the linked function type for documentation on a particular function. ## ABS +`ABS()` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Calculates the absolute value of a numeric expression. -* **Syntax:** `ABS()` -* **Function type:** Scalar, numeric - -
Example - -The following example applies the ABS function to the `ArrDelay` column from the `flight-carriers` datasource. - -```sql -SELECT - "ArrDelay" AS "arrival_delay", - ABS("ArrDelay") AS "absolute_arrival_delay" -FROM "flight-carriers" -WHERE "ArrDelay" < 0 -LIMIT 1 -``` -Returns the following: - -| `arrival_delay` | `absolute_arrival_delay` | -| -- | -- | -| `-27` | `27` | -
- -[Learn more](sql-scalar.md#numeric-functions) - ## ACOS -Calculates the arc cosine (arccosine) of a numeric expression. +`ACOS()` -* **Syntax:** `ACOS(expr)` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following example calculates the arc cosine of `0`. - -```sql -SELECT ACOS(0) AS "arc_cosine" -``` -Returns the following: - -| `arc_cosine` | -| -- | -| `1.5707963267948966` | -
- -[Learn more](sql-scalar.md#numeric-functions) +Calculates the arc cosine of a numeric expression. ## ANY_VALUE @@ -107,7 +64,6 @@ Returns any value of the specified expression. Counts distinct values of a regular column or a prebuilt sketch column. -## APPROX_COUNT_DISTINCT_BUILTIN `APPROX_COUNT_DISTINCT_BUILTIN(expr)` **Function type:** [Aggregation](sql-aggregations.md) @@ -295,72 +251,27 @@ Joins all elements of `arr` by the delimiter specified by `str`. ## ASIN -Calculates the arc sine (arcsine) of a numeric expression. +`ASIN()` -* **Syntax:** `ASIN(expr)` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following example calculates the arc sine of `1`. - -```sql -SELECT ASIN(1) AS "arc_sine" -``` -Returns the following: - -| `arc_sine` | -| -- | -| `1.5707963267948966` | -
- -[Learn more](sql-scalar.md#numeric-functions) +Calculates the arc sine of a numeric expression. ## ATAN -Calculates the arc tangent (arctangent) of a numeric expression. +`ATAN()` -* **Syntax:** `ATAN(expr)` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following example calculates the arc tangent of `1`. - -```sql -SELECT ATAN(1) AS "arc_tangent" -``` -Returns the following: - -| `arc_tangent` | -| -- | -| `0.7853981633974483` | -
- -[Learn more](sql-scalar.md#numeric-functions) +Calculates the arc tangent of a numeric expression. ## ATAN2 -Calculates the arc tangent (arctangent) of a specified x and y coordinate. +`ATAN2(, )` -* **Syntax:** `ATAN2(x, y)` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following example calculates the arc tangent of the coordinate `(1, -1)` - -```sql -SELECT ATAN2(1,-1) AS "arc_tangent_2" -``` -Returns the following: - -| `arc_tangent_2` | -| -- | -| `2.356194490192345` | -
- -[Learn more](sql-scalar.md#numeric-functions) +Calculates the arc tangent of the two arguments. ## AVG @@ -396,188 +307,67 @@ Performs a bitwise XOR operation on all input values. ## BITWISE_AND -Returns the bitwise AND between two expressions: `expr1 & expr2`. +`BITWISE_AND(expr1, expr2)` -* **Syntax:** `BITWISE_AND(expr1, expr2)` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following example performs the bitwise AND operation `12 & 10`. - -```sql -SELECT BITWISE_AND(12, 10) AS "bitwise_and" -``` -Returns the following: - -| `bitwise_and` | -| -- | -| 8 | -
- -[Learn more](sql-scalar.md#numeric-functions) +Returns the bitwise AND between the two expressions, that is, `expr1 & expr2`. ## BITWISE_COMPLEMENT -Returns the bitwise complement (bitwise not) for the expression: `~expr`. +`BITWISE_COMPLEMENT(expr)` -* **Syntax:** `BITWISE_COMPLEMENT(expr)` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following example performs the bitwise complement operation `~12`. - -```sql -SELECT BITWISE_COMPLEMENT(12) AS "bitwise_complement" -``` -Returns the following: - -| `bitwise_complement` | -| -- | -| -13 | -
- -[Learn more](sql-scalar.md#numeric-functions) +Returns the bitwise NOT for the expression, that is, `~expr`. ## BITWISE_CONVERT_DOUBLE_TO_LONG_BITS -Converts the bits of an IEEE 754 floating-point double value to long. +`BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(expr)` -* **Syntax:**`BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(expr)` -* **Function type:** Scalar, numeric - -
Example - -The following example returns the IEEE 754 floating-point double representation of `255` as a long. - -```sql -SELECT BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(255) AS "ieee_754_double_to_long" -``` -Returns the following: - -| `ieee_754_double_to_long` | -| -- | -| `4643176031446892544` | -
- -[Learn more](sql-scalar.md#numeric-functions) +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +Converts the bits of an IEEE 754 floating-point double value to a long. ## BITWISE_CONVERT_LONG_BITS_TO_DOUBLE +`BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(expr)` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Converts a long to the IEEE 754 floating-point double specified by the bits stored in the long. -* **Syntax:**`BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(expr)` -* **Function type:** Scalar, numeric - -
Example - -The following example returns the long representation of `4643176031446892544` as an IEEE 754 floating-point double. - -```sql -SELECT BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(4643176031446892544) AS "long_to_ieee_754_double" -``` -Returns the following: - -| `long_to_ieee_754_double` | -| -- | -| `255` | -
- -[Learn more](sql-scalar.md#numeric-functions) - ## BITWISE_OR -Returns the bitwise OR between the two expressions: `expr1 | expr2`. +`BITWISE_OR(expr1, expr2)` -* **Syntax:** `BITWISE_OR(expr1, expr2)` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following example performs the bitwise OR operation `12 | 10`. - -```sql -SELECT BITWISE_OR(12, 10) AS "bitwise_or" -``` -Returns the following: - -| `bitwise_or` | -| -- | -| `14` | -
- -[Learn more](sql-scalar.md#numeric-functions) +Returns the bitwise OR between the two expressions, that is, `expr1 | expr2`. ## BITWISE_SHIFT_LEFT -Returns the bitwise left shift by x positions of an expr: `expr << x`. +`BITWISE_SHIFT_LEFT(expr1, expr2)` -* **Syntax:** `BITWISE_SHIFT_LEFT(expr, x)` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following example performs the bitwise SHIFT operation `2 << 3`. - -```sql -SELECT BITWISE_SHIFT_LEFT(2, 3) AS "bitwise_shift_left" -``` -Returns the following: - -| `bitwise_shift_left` | -| -- | -| `16` | -
- -[Learn more](sql-scalar.md#numeric-functions) +Returns a bitwise left shift of expr1, that is, `expr1 << expr2`. ## BITWISE_SHIFT_RIGHT -Returns the bitwise right shift by x positions of an expr: `expr >> x`. +`BITWISE_SHIFT_RIGHT(expr1, expr2)` -* **Syntax:** `BITWISE_SHIFT_RIGHT(expr, x)` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following example performs the bitwise SHIFT operation `16 >> 3`. - -```sql -SELECT BITWISE_SHIFT_RIGHT(16, 3) AS "bitwise_shift_right" -``` -Returns the following: - -| `bitwise_shift_right` | -| -- | -| `2` | -
- -[Learn more](sql-scalar.md#numeric-functions) +Returns a bitwise right shift of expr1, that is, `expr1 >> expr2`. ## BITWISE_XOR -Returns the bitwise exclusive OR between the two expressions: `expr1 ^ expr2`. +`BITWISE_XOR(expr1, expr2)` -* **Syntax:** `BITWISE_XOR(expr1, expr2)` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following example performs the bitwise XOR operation `12 ^ 10`. - -```sql -SELECT BITWISE_XOR(12, 10) AS "bitwise_xor" -``` -Returns the following: - -| `bitwise_xor` | -| -- | -| `6` | -
- -[Learn more](sql-scalar.md#numeric-functions) +Returns the bitwise exclusive OR between the two expressions, that is, `expr1 ^ expr2`. ## BLOOM_FILTER @@ -597,30 +387,11 @@ Returns true if the expression is contained in a Base64-serialized Bloom filter. ## BTRIM -Trims characters from both the leading and trailing ends of an expression. Defaults `chars` to a space if none is provided. +`BTRIM(, [])` -* **Syntax:** `BTRIM(expr[, chars])` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example trims the `_` characters from both ends of the string expression. - -```sql -SELECT - '___abc___' AS "original_string", - BTRIM('___abc___', '_') AS "trim_both_ends" -``` - -Returns the following: - -| `original_string` | `trim_both_ends` | -| -- | -- | -| `___abc___` | `abc` | - -
- -[Learn more](sql-scalar.md#string-functions) +Trims characters from both the leading and trailing ends of an expression. ## CASE @@ -648,49 +419,28 @@ Rounds up a timestamp by a given time unit. ## CEIL (numeric) +`CEIL()` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Calculates the smallest integer value greater than or equal to the numeric expression. -* **Syntax:** `CEIL()` -* **Function type:** Scalar, numeric - -
Example - -The following example applies the CEIL function to the `fare_amount` column from the `taxi-trips` datasource. - -```sql -SELECT - "fare_amount" AS "fare_amount", - CEIL("fare_amount") AS "ceiling_fare_amount" -FROM "taxi-trips" -LIMIT 1 -``` -Returns the following: - -| `fare_amount` | `ceiling_fare_amount` | -| -- | -- | -| `21.25` | `22` | -
- -[Learn more](sql-scalar.md#numeric-functions) ## CHAR_LENGTH +`CHAR_LENGTH(expr)` + +**Function type:** [Scalar, string](sql-scalar.md#string-functions) + Alias for [`LENGTH`](#length). -* **Syntax:** `CHAR_LENGTH(expr)` -* **Function type:** Scalar, string - -[Learn more](sql-scalar.md#string-functions) - ## CHARACTER_LENGTH +`CHARACTER_LENGTH(expr)` + +**Function type:** [Scalar, string](sql-scalar.md#string-functions) + Alias for [`LENGTH`](#length). -* **Syntax:** `CHARACTER_LENGTH(expr)` -* **Function type:** Scalar, string - -[Learn more](sql-scalar.md#string-functions) - - ## COALESCE `COALESCE(expr, expr, ...)` @@ -701,111 +451,36 @@ Returns the first non-null value. ## CONCAT +`CONCAT(expr, expr...)` + +**Function type:** [Scalar, string](sql-scalar.md#string-functions) + Concatenates a list of expressions. -* **Syntax:** `CONCAT(expr[, expr,...])` -* **Function type:** Scalar, string - -
Example - -The following example concatenates the `OriginCityName` column from `flight-carriers`, the string ` to `, and the `DestCityName` column from `flight-carriers`. - -```sql -SELECT - "OriginCityName" AS "origin_city", - "DestCityName" AS "destination_city", - CONCAT("OriginCityName", ' to ', "DestCityName") AS "concatenate_flight_details" -FROM "flight-carriers" -LIMIT 1 -``` - -Returns the following: - -| `origin_city` | `destination_city` | `concatenate_flight_details` | -| -- | -- | -- | -| `San Juan, PR` | `Washington, DC` | `San Juan, PR to Washington, DC` | - -
- -[Learn more](sql-scalar.md#string-functions) - ## CONTAINS_STRING -Returns `true` if `str` is a substring of `expr`, case-sensitive. Otherwise returns `false`. +`CONTAINS_STRING(, )` -* **Syntax:** `CONTAINS_STRING(expr, str)` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example returns `true` if the `OriginCityName` column from the `flight-carriers` datasource contains the substring `San`. - -```sql -SELECT - "OriginCityName" AS "origin_city", - CONTAINS_STRING("OriginCityName", 'San') AS "contains_string" -FROM "flight-carriers" -LIMIT 2 -``` - -Returns the following: - -| `origin_city` | `contains_string` | -| -- | -- | -| `San Juan, PR` | `true` | -| `Boston, MA` | `false` | - -
- - -[Learn more](sql-scalar.md#string-functions) +Finds whether a string is in a given expression, case-sensitive. ## COS +`COS()` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Calculates the trigonometric cosine of an angle expressed in radians. -* **Syntax:** `COS(expr)` -* **Function type:** Scalar, numeric - -
Example - -The following example calculates the cosine of angle `PI/3` radians. - -```sql -SELECT COS(PI / 3) AS "cosine" -``` -Returns the following: - -| `cosine` | -| -- | -| `0.5000000000000001` | -
- -[Learn more](sql-scalar.md#numeric-functions) - ## COT +`COT()` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Calculates the trigonometric cotangent of an angle expressed in radians. -* **Syntax:** `COT(expr)` -* **Function type:** Scalar, numeric - -
Example - -The following example calculates the cotangent of angle `PI/3` radians. - -```sql -SELECT COT(PI / 3) AS "cotangent" -``` -Returns the following: - -| `cotangent` | -| -- | -| `0.577350269189626` | -
- -[Learn more](sql-scalar.md#numeric-functions) - ## COUNT `COUNT([DISTINCT] expr)` @@ -826,85 +501,28 @@ Returns the cumulative distribution of the current row within the window calcula ## CURRENT_DATE -Returns the current date in UTC time, unless you specify a different timezone in the query context. +`CURRENT_DATE` -* **Syntax:** `CURRENT_DATE` -* **Function type:** Scalar, date and time +**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions) -
Example - -The following example returns the current date. - -```sql -SELECT CURRENT_DATE AS "current_date" -``` - -Returns the following: - -| `current_date` | -| -- | -| `2024-08-14T00:00:00.000Z `| - -
- -[Learn more](sql-scalar.md#date-and-time-functions) +Returns the current date in the connection's time zone. ## CURRENT_TIMESTAMP -Returns the current timestamp in UTC time, unless you specify a different timezone in the query context. +`CURRENT_TIMESTAMP` +**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions) -* **Syntax:** `CURRENT_TIMESTAMP` -* **Function type:** Scalar, date and time - -
Example - -The following example returns the current timestamp. - -```sql -SELECT CURRENT_TIMESTAMP AS "current_timestamp" -``` - -Returns the following: - -| `current_timestamp` | -| -- | -| `2024-08-14T21:30:13.793Z` | - -
- -[Learn more](sql-scalar.md#date-and-time-functions) +Returns the current timestamp in the connection's time zone. ## DATE_TRUNC +`DATE_TRUNC(, )` + +**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions) + Rounds down a timestamp by a given time unit. -* **Syntax:** `DATE_TRUNC(unit, timestamp_expr)` -* **Function type:** Scalar, date and time - -
Example - -The following example truncates a timestamp from the `__time` column from the `taxi-trips` datasource to the most recent `decade`. - -```sql -SELECT - "__time" AS "original_timestamp", - DATE_TRUNC('decade', "__time") AS "truncate_timestamp" -FROM "taxi-trips" -LIMIT 1 -``` - -Returns the following: - -| `original_timestamp` | `truncate_time` | -| -- | -- | -| `2013-08-01T08:14:37.000Z` | `2010-01-01T00:00:00.000Z` | - -
- -[Learn more](sql-scalar.md#date-and-time-functions) - - ## DECODE_BASE64_COMPLEX `DECODE_BASE64_COMPLEX(dataType, expr)` @@ -915,54 +533,21 @@ Decodes a Base64-encoded string into a complex data type, where `dataType` is th ## DECODE_BASE64_UTF8 +`DECODE_BASE64_UTF8(expr)` + +**Function type:** [Scalar, string](sql-scalar.md#string-functions) + + Decodes a Base64-encoded string into a UTF-8 encoded string. -* **Syntax:** `DECODE_BASE64_UTF8(expr)` -* **Function type:** Scalar, string - -
Example - -The following example converts the base64 encoded string `SGVsbG8gV29ybGQhCg==` into an UTF-8 encoded string. - -```sql -SELECT - 'SGVsbG8gV29ybGQhCg==' AS "base64_encoding", - DECODE_BASE64_UTF8('SGVsbG8gV29ybGQhCg==') AS "convert_to_UTF8_encoding" -``` - -Returns the following: - -| `base64_encoding` | `convert_to_UTF8_encoding` | -| -- | -- | -| `SGVsbG8gV29ybGQhCg==` | `Hello World!` | - -
- -[Learn more](sql-scalar.md#string-functions) - ## DEGREES +`DEGREES()` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Converts an angle from radians to degrees. -* **Syntax:** `DEGREES(expr)` -* **Function type:** Scalar, numeric - -
Example - -The following example converts an angle of `PI` radians to degrees - -```sql -SELECT DEGREES(PI) AS "degrees" -``` -Returns the following: - -| `degrees` | -| -- | -| `180` | -
- -[Learn more](sql-scalar.md#numeric-functions) - ## DENSE_RANK `DENSE_RANK()` @@ -973,37 +558,16 @@ Returns the rank for a row within a window without gaps. For example, if two row ## DIV +`DIV(x, y)` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Returns the result of integer division of `x` by `y`. -* **Syntax:** `DIV(x, y)` -* **Function type:** Scalar, numeric - - - :::info - - The `DIV` function is not implemented in Druid versions 30.0.0 or earlier. Consider using [`SAFE_DIVIDE`](./sql-functions.md/#safe_divide) instead. - +The `DIV` function is not implemented in Druid versions 30.0.0 or earlier. Consider using [`SAFE_DIVIDE`](./sql-functions.md#safe_divide) instead. ::: -[Learn more](sql-scalar.md#numeric-functions) - ## DS_CDF `DS_CDF(expr, splitPoint0, splitPoint1, ...)` @@ -1137,27 +701,12 @@ Returns the value of a numeric or string expression corresponding to the earlies ## EXP +`EXP()` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Calculates _e_ raised to the power of the numeric expression. -* **Syntax:** `EXP()` -* **Function type:** Scalar, numeric - -
Example - -The following example calculates _e_ to the power of 1. - -```sql -SELECT EXP(1) AS "exponential" -``` -Returns the following: - -| `exponential` | -| -- | -| `2.7182818284590455` | -
- -[Learn more](sql-scalar.md#numeric-functions) - ## EXTRACT `EXTRACT( FROM )` @@ -1184,56 +733,19 @@ Rounds down a timestamp by a given time unit. ## FLOOR (numeric) -Calculates the largest integer less than or equal to the numeric expression. +`FLOOR()` -* **Syntax:** `FLOOR()` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following example applies the FLOOR function to the `fare_amount` column from the `taxi-trips` datasource. - -```sql -SELECT - "fare_amount" AS "fare_amount", - FLOOR("fare_amount") AS "floor_fare_amount" -FROM "taxi-trips" -LIMIT 1 -``` -Returns the following: - -| `fare_amount` | `floor_fare_amount` | -| -- | -- | -| `21.25` | `21` | -
- -[Learn more](sql-scalar.md#numeric-functions) +Calculates the largest integer value less than or equal to the numeric expression. ## GREATEST -Returns the maximum value from the provided expressions. For information on how Druid interprets the arguments passed into the function, see [Reduction functions](sql-scalar.md#reduction-functions). +`GREATEST([expr1, ...])` -* **Syntax:** `GREATEST([expr1, ...])` -* **Function type:** Scalar, reduction - -
Example - -The following example returns the greatest value between the numeric constant `PI`, the integer number `4`, and the double `-5.0`. Druid interprets these arguments as DOUBLE data type. - -```sql -SELECT GREATEST(PI, 4, -5.0) AS "greatest" -``` - -Returns the following: - -| `greatest` | -| -- | -| `4` | - -
- -[Learn more](sql-scalar.md#reduction-functions) +**Function type:** [Scalar, reduction](sql-scalar.md#reduction-functions) +Returns the maximum value from the provided arguments. ## GROUPING @@ -1277,107 +789,35 @@ Returns a union of HLL sketches. ## HUMAN_READABLE_BINARY_BYTE_FORMAT -Converts an integer byte size into human-readable [IEC](https://en.wikipedia.org/wiki/Binary_prefix) format. +`HUMAN_READABLE_BINARY_BYTE_FORMAT(value[, precision])` -* **Syntax:** `HUMAN_READABLE_BINARY_BYTE_FORMAT(value[, precision])` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - - The following example converts `1000000` into IEC format. - - ```sql - SELECT HUMAN_READABLE_BINARY_BYTE_FORMAT(1000000, 2) AS "iec_format" - ``` - - Returns the following: - - | `iec_format` | - | -- | - | `976.56 KiB` | - -
- -[Learn more](sql-scalar.md#numeric-functions) +Converts an integer byte size into human-readable IEC format. ## HUMAN_READABLE_DECIMAL_BYTE_FORMAT -Converts a byte size into human-readable [SI](https://en.wikipedia.org/wiki/Binary_prefix) format. +`HUMAN_READABLE_DECIMAL_BYTE_FORMAT(value[, precision])` -* **Syntax:** `HUMAN_READABLE_DECIMAL_BYTE_FORMAT(value[, precision])` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following example converts `1000000` into SI format. - -```sql -SELECT HUMAN_READABLE_DECIMAL_BYTE_FORMAT(1000000, 2) AS "si_format" -``` - -Returns the following: - -|`si_format`| -|--| -|`1.00 MB`| - -
- -[Learn more](sql-scalar.md#numeric-functions) +Converts a byte size into human-readable SI format. ## HUMAN_READABLE_DECIMAL_FORMAT +`HUMAN_READABLE_DECIMAL_FORMAT(value[, precision])` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Converts a byte size into human-readable SI format with single-character units. -* **Syntax:** `HUMAN_READABLE_DECIMAL_FORMAT(value[, precision])` -* **Function type:** Scalar, numeric - -
Example - - The following example converts `1000000` into single character SI format. - -```sql -SELECT HUMAN_READABLE_DECIMAL_FORMAT(1000000, 2) AS "single_character_si_format" -``` - -Returns the following: - -|`single_character_si_format`| -|--| -|`1.00 M`| -
- -[Learn more](sql-scalar.md#numeric-functions) - ## ICONTAINS_STRING -Returns `true` if `str` is a substring of `expr`, case-insensitive. Otherwise returns `false`. +`ICONTAINS_STRING(, str)` -* **Syntax:** `ICONTAINS_STRING(expr, str)` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example returns `true` if the `OriginCityName` column from the `flight-carriers` datasource contains the case-insensitive substring `san`. - -```sql -SELECT - "OriginCityName" AS "origin_city", - ICONTAINS_STRING("OriginCityName", 'san') AS "contains_case_insensitive_string" -FROM "flight-carriers" -LIMIT 2 -``` - -Returns the following: - -| `origin_city` | `contains_case_insensitive_string` | -| -- | -- | -| `San Juan, PR` | `true` | -| `Boston, MA` | `false` | - -
- -[Learn more](sql-scalar.md#string-functions) +Finds whether a string is in a given expression, case-insensitive. ## IPV4_MATCH @@ -1419,6 +859,13 @@ Returns true if the IPv6 `address` belongs to the `subnet` literal, else false. Returns an array of field names from `expr` at the specified `path`. +## JSON_MERGE + +**Function type:** [JSON](sql-json-functions.md) + +`JSON_MERGE(expr1, expr2[, expr3 ...])` +Merges two or more JSON `STRING` or `COMPLEX` into one. Preserves the rightmost value when there are key overlaps. Returning always a `COMPLEX` type. + ## JSON_OBJECT **Function type:** [JSON](sql-json-functions.md) @@ -1501,269 +948,75 @@ If you do not supply an `offset`, returns the value evaluated at the row followi ## LEAST -Returns the minimum value from the provided expressions. For information on how Druid interprets the arguments passed into the function, see [Reduction functions](sql-scalar.md#reduction-functions). +`LEAST([expr1, ...])` -* **Syntax:** `LEAST([expr1, ...])` -* **Function type:** Scalar, reduction - -
Example - -The following example returns the minimum value between the strings `apple`, `orange`, and `pear`. Druid interprets these arguments as STRING data type. - -```sql -SELECT LEAST( 'apple', 'orange', 'pear') AS "least" -``` - -Returns the following: - -| `least` | -| -- | -| `apple` | - -
- -[Learn more](sql-scalar.md#reduction-functions) +**Function type:** [Scalar, reduction](sql-scalar.md#reduction-functions) +Returns the minimum value from the provided arguments. ## LEFT -Returns the `N` leftmost characters of an expression, where `N` is an integer value. +`LEFT(expr, [length])` -* **Syntax:** `LEFT(expr, N)` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example returns the `3` leftmost characters of the expression `ABCDEFG`. - -```sql -SELECT - 'ABCDEFG' AS "expression", - LEFT('ABCDEFG', 3) AS "leftmost_characters" -``` - -Returns the following: - -| `expression` | `leftmost_characters` | -| -- | -- | -| `ABCDEFG` | `ABC` | - -
- -[Learn more](sql-scalar.md#string-functions) +Returns the leftmost number of characters from an expression. ## LENGTH -Returns the length of the expression in UTF-16 code units. +`LENGTH(expr)` -* **Syntax:** `LENGTH(expr)` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example returns the character length of the `OriginCityName` column from the `flight-carriers` datasource. - -```sql -SELECT - "OriginCityName" AS "origin_city_name", - LENGTH("OriginCityName") AS "city_name_length" -FROM "flight-carriers" -LIMIT 1 -``` - -Returns the following: - -| `origin_city_name` | `city_name_length` | -| -- | -- | -| `San Juan, PR` | `12` | - -
- -[Learn more](sql-scalar.md#string-functions) +Returns the length of the expression in UTF-16 encoding. ## LN +`LN(expr)` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Calculates the natural logarithm of the numeric expression. -* **Syntax:** `LN()` -* **Function type:** Scalar, numeric - -
Example - -The following example applies the LN function to the `max_temperature` column from the `taxi-trips` datasource. - -```sql -SELECT - "max_temperature" AS "max_temperature", - LN("max_temperature") AS "natural_log_max_temp" -FROM "taxi-trips" -LIMIT 1 -``` - -Returns the following: - -| `max_temperature` | `natural_log_max_temp` | -| -- | -- | -| `76` | `4.330733340286331` | - -
- -[Learn more](sql-scalar.md#numeric-functions) - ## LOG10 -Calculates the base-10 logarithm of the numeric expression. +`LOG10(expr)` -* **Syntax:** `LOG10()` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following example applies the LOG10 function to the `max_temperature` column from the `taxi-trips` datasource. - -```sql -SELECT - "max_temperature" AS "max_temperature", - LOG10("max_temperature") AS "log10_max_temp" -FROM "taxi-trips" -LIMIT 1 -``` -Returns the following: - -| `max_temperature` | `log10_max_temp` | -| -- | -- | -| `76` | `1.8808135922807914` | -
- -[Learn more](sql-scalar.md#numeric-functions) +Calculates the base-10 of the numeric expression. ## LOOKUP -Searches for `expr` in a registered [query-time lookup table](lookups.md) named `lookupName` and returns the mapped value. If `expr` is null or not contained in the lookup, returns `defaultValue` if supplied, otherwise returns null. +`LOOKUP(, [, ])` -* **Syntax:** `LOOKUP(expr, lookupName[, defaultValue])` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example uses a `map` type lookup table named `code_to_name`, which contains the following key-value pairs: - -```json -{ - "SJU": "Luis Munoz Marin International Airport", - "IAD": "Dulles International Airport" -} -``` - -The example uses `code_to_name` to map the `Origin` column from the `flight-carriers` datasource to the corresponding full airport name. Returns `key not found` if no matching key exists in the lookup table. - -```sql -SELECT - "Origin" AS "origin_airport", - LOOKUP("Origin", 'code_to_name','key not found') AS "full_airport_name" -FROM "flight-carriers" -LIMIT 2 -``` - -Returns the following: - -| `origin_airport` | `full_airport_name` | -| -- | -- | -| `SJU` | `Luis Munoz Marin International Airport` | -| `BOS` | `key not found` | - -
- -[Learn more](sql-scalar.md#string-functions) +Looks up the expression in a registered query-time lookup table. ## LOWER +`LOWER(expr)` + +**Function type:** [Scalar, string](sql-scalar.md#string-functions) + Returns the expression in lowercase. -* **Syntax:** `LOWER(expr)` -* **Function type:** Scalar, string - -
Example - -The following example converts the `OriginCityName` column from the `flight-carriers` datasource to lowercase. - -```sql -SELECT - "OriginCityName" AS "origin_city", - LOWER("OriginCityName") AS "lowercase" -FROM "flight-carriers" -LIMIT 1 -``` - -Returns the following: - -| `origin_city` | `lowercase` | -| -- | -- | -`San Juan, PR` | `san juan, pr` | - -
- -[Learn more](sql-scalar.md#string-functions) - - ## LPAD -Returns a string of size `length` from `expr`. When the length of `expr` is less than `length`, left pads `expr` with `chars`, which defaults to the space character. Truncates `expr` to `length` if `length` is shorter than the length of `expr`. +`LPAD(, , [])` -* **Syntax:** `LPAD(expr, length[, chars])` -* **Function type:** Scalar, string - -
Example - -The following example left pads the value of `OriginStateName` from the `flight-carriers` datasource to return a total of 11 characters. - -```sql -SELECT - "OriginStateName" AS "origin_state", - LPAD("OriginStateName", 11, '+') AS "add_left_padding" -FROM "flight-carriers" -LIMIT 3 -``` - -Returns the following: - -| `origin_state` | `add_left_padding` | -| -- | -- | -| `Puerto Rico` | `Puerto Rico` | -| `Massachusetts` | `Massachuset` | -| `Florida` | `++++Florida` | - -
- -[Learn more](sql-scalar.md#string-functions) +**Function type:** [Scalar, string](sql-scalar.md#string-functions) +Returns the leftmost number of characters from an expression, optionally padded with the given characters. ## LTRIM -Trims characters from the leading end of an expression. Defaults `chars` to a space if none is provided. +`LTRIM(, [])` -* **Syntax:** `LTRIM(expr[, chars])` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example trims the `_` characters from the leading end of the string expression. - -```sql -SELECT - '___abc___' AS "original_string", - LTRIM('___abc___', '_') AS "trim_leading_end_of_expression" -``` - -Returns the following: - -| `original_string` | `trim_leading_end_of_expression` | -| -- | -- | -| `___abc___` | `abc___` | - -
- -[Learn more](sql-scalar.md#string-functions) +Trims characters from the leading end of an expression. ## MAX @@ -1791,26 +1044,11 @@ Returns the minimum value of a set of values. ## MOD -Calculates x modulo y, or the remainder of x divided by y. Where x and y are numeric expressions. +`MOD(x, y)` -* **Syntax:** `MOD(x, y)` -* **Function type:** Scalar, numeric +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) -
Example - -The following calculates 78 MOD 10. - -```sql -SELECT MOD(78, 10) as "modulo" -``` -Returns the following: - -| `modulo` | -| -- | -| `8` | -
- -[Learn more](sql-scalar.md#numeric-functions) +Calculates x modulo y, or the remainder of x divided by y. ## MV_APPEND @@ -1966,31 +1204,11 @@ Parses `expr` into a `COMPLEX` object. This operator deserializes JSON val ## PARSE_LONG -Converts a string into a long(BIGINT) with the given radix, or into DECIMAL(base 10) if a radix is not provided. +`PARSE_LONG(, [])` -* **Syntax:**`PARSE_LONG(string[, radix])` -* **Function type:** Scalar, string - -
Example - -The following example converts the string representation of the binary, radix 2, number `1100` into its long (BIGINT) equivalent. - -```sql -SELECT - '1100' AS "binary_as_string", - PARSE_LONG('1110', 2) AS "bigint_value" -``` - -Returns the following: - -| `binary_as_string` | `bigint_value` | -| -- | -- | -| `1100` | `14` | - -
- -[Learn more](sql-scalar.md#string-functions) +**Function type:** [Scalar, string](sql-scalar.md#string-functions) +Converts a string into a BIGINT with the given base or into a DECIMAL data type if the base is not specified. ## PERCENT_RANK @@ -2002,80 +1220,28 @@ Returns the relative rank of the row calculated as a percentage according to the ## POSITION -Returns the one-based index position of a substring within an expression, optionally starting from a given one-based index. If `substring` is not found, returns 0. +`POSITION( IN [FROM ])` -* **Syntax**: `POSITION(substring IN expr [FROM startingIndex])` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example returns the one-based index of the substring `PR` in the `OriginCityName` column from the `flight-carriers` datasource starting from index 5. - -```sql -SELECT - "OriginCityName" AS "origin_city", - POSITION('PR' IN "OriginCityName" FROM 5) AS "index" -FROM "flight-carriers" -LIMIT 2 -``` - -Returns the following: - -| `origin_city` | `index` | -| -- | -- | -| `San Juan, PR` | `11` | -| `Boston, MA` | `0` | - -
- -[Learn more](sql-scalar.md#string-functions) +Returns the one-based index position of a substring within an expression, optionally starting from a given one-based index. ## POWER +`POWER(expr, power)` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Calculates a numerical expression raised to the specified power. -* **Syntax:** `POWER(base, exponent)` -* **Function type:** Scalar, numeric - -
Example - -The following example raises 5 to the power of 2. - -```sql -SELECT POWER(5, 2) AS "power" -``` -Returns the following: - -| `power` | -| -- | -| `25` | -
- -[Learn more](sql-scalar.md#numeric-functions) - ## RADIANS +`RADIANS(expr)` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Converts an angle from degrees to radians. -* **Syntax:** `RADIANS(expr)` -* **Function type:** Scalar, numeric - -
Example - -The following example converts an angle of `180` degrees to radians - -```sql -SELECT RADIANS(180) AS "radians" -``` -Returns the following: - -| `radians` | -| -- | -| `3.141592653589793` | -
- -[Learn more](sql-scalar.md#numeric-functions) - ## RANK `RANK()` @@ -2086,227 +1252,69 @@ Returns the rank with gaps for a row within a window. For example, if two rows t ## REGEXP_EXTRACT -Apply regular expression `pattern` to `expr` and extract the Nth capture group. If `N` is unspecified or zero, returns the first substring that matches the pattern. Returns `null` if there is no matching pattern. +`REGEXP_EXTRACT(, , [])` -* **Syntax:** `REGEXP_EXTRACT(expr, pattern[, N])` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example uses regular expressions to find city names inside the `OriginCityName` column from the `flight-carriers` datasource by matching what comes before the comma. - -```sql -SELECT - "OriginCityName" AS "origin_city", - REGEXP_EXTRACT("OriginCityName", '([^,]+)', 0) AS "pattern_match" -FROM "flight-carriers" -LIMIT 1 -``` - -Returns the following: - -| `origin_city` | `pattern_match` | -| -- | -- | -| `San Juan, PR` | `San Juan`| - -
- -[Learn more](sql-scalar.md#string-functions) +Applies a regular expression to the string expression and returns the _n_th match. ## REGEXP_LIKE -Returns `true` if the regular expression `pattern` finds a match in `expr`. Returns `false` otherwise. +`REGEXP_LIKE(, )` -* **Syntax:** `REGEXP_LIKE(expr, pattern)` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example returns `true` when the `OriginCityName` column from `flight-carriers` has a city name containing a space. - -```sql -SELECT - "OriginCityName" AS "origin_city", - REGEXP_LIKE("OriginCityName", '[A-Za-z]+\s[A-Za-z]+') AS "pattern_found" -FROM "flight-carriers" -LIMIT 2 -``` - -Returns the following: - -| `origin_city` | `pattern_found` | -| -- | -- | -| `San Juan, PR` | `true` | -| `Boston, MA` | `false` | - -
- -[Learn more](sql-scalar.md#string-functions) +Returns true or false signifying whether the regular expression finds a match in the string expression. ## REGEXP_REPLACE -Replaces all occurrences of a regular expression in a string expression with a replacement string. Refer to capture groups in the replacement string using `$group` syntax. For example: `$1` or `$2`. +`REGEXP_REPLACE(, , )` -* **Syntax:** `REGEXP_REPLACE(expr, pattern, replacement)` -* **Function type:** Scalar, string - -
Example - -The following example matches three consecutive words, where each word is its own capture group, and replaces the matched words with the word in the second capture group punctuated with exclamation marks. - -```sql -SELECT - 'foo bar baz' AS "original_string", - REGEXP_REPLACE('foo bar baz', '([A-Za-z]+) ([A-Za-z]+) ([A-Za-z]+)' , '$2!') AS "modified_string" -``` - -Returns the following: - -| `original_string` | `modified_string` | -| -- | -- | -| `foo bar baz` | `bar!` | - -
- -[Learn more](sql-scalar.md#string-functions) +**Function type:** [Scalar, string](sql-scalar.md#string-functions) +Replaces all occurrences of a regular expression in a string expression with a replacement string. The replacement +string may refer to capture groups using `$1`, `$2`, etc. ## REPEAT -Repeats the string expression `N` times, where `N` is an integer. +`REPEAT(, [])` -* **Syntax:** `REPEAT(expr, N)` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example returns the string expression `abc` repeated `3` times. - -```sql -SELECT - 'abc' AS "original_string", - REPEAT('abc', 3) AS "with_repetition" -``` - -Returns the following: - -| `original_string` | `with_repetition` | -| -- | -- | -| `abc` | `abcabcabc` | - -
- -[Learn more](sql-scalar.md#string-functions) +Repeats the string expression an integer number of times. ## REPLACE -Replaces instances of a substring with a replacement string in the given expression. +`REPLACE(expr, pattern, replacement)` -* **Syntax:** `REPLACE(expr, substring, replacement)` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example replaces instances of the substring `abc` with `XYZ`. - -```sql -SELECT - 'abc 123 abc 123' AS "original_string", - REPLACE('abc 123 abc 123', 'abc', 'XYZ') AS "modified_string" -``` - -Returns the following: - -| `original_string` | `modified_string` | -| -- | -- | -| `abc 123 abc 123` | `XYZ 123 XYZ 123` | - -
- -[Learn more](sql-scalar.md#string-functions) +Replaces a pattern with another string in the given expression. ## REVERSE +`REVERSE(expr)` + +**Function type:** [Scalar, string](sql-scalar.md#string-functions) + Reverses the given expression. -* **Syntax:** `REVERSE(expr)` -* **Function type:** Scalar, string - -
Example - -The following example reverses the string expression `abc`. - -```sql -SELECT - 'abc' AS "original_string", - REVERSE('abc') AS "reversal" -``` - -Returns the following: - -| `original_string` | `reversal` | -| -- | -- | -| `abc` | `cba` | - -
- -[Learn more](sql-scalar.md#string-functions) - ## RIGHT -Returns the `N` rightmost characters of an expression, where `N` is an integer value. +`RIGHT(expr, [length])` -* **Syntax:** `RIGHT(expr, N)` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example returns the `3` rightmost characters of the expression `ABCDEFG`. - -```sql -SELECT - 'ABCDEFG' AS "expression", - RIGHT('ABCDEFG', 3) AS "rightmost_characters" -``` - -Returns the following: - -| `expression` | `rightmost_characters` | -| -- | -- | -| `ABCDEFG` | `EFG` | - -
- -[Learn more](sql-scalar.md#string-functions) +Returns the rightmost number of characters from an expression. ## ROUND +`ROUND(expr[, digits])` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Calculates the rounded value for a numerical expression. -* **Syntax:** `ROUND(expr[, digits])` -* **Function type:** Scalar, numeric - -
Example - -The following applies the ROUND function to 0 decimal points on the `pickup_longitude` column from the `taxi-trips` datasource. - -```sql -SELECT - "pickup_longitude" AS "pickup_longitude", - ROUND("pickup_longitude", 0) as "rounded_pickup_longitude" -FROM "taxi-trips" -WHERE "pickup_longitude" IS NOT NULL -LIMIT 1 -``` -Returns the following: - -| `pickup_longitude` | `rounded_pickup_longitude` | -| -- | -- | -| `-73.9377670288086` | `-74` | -
- -[Learn more](sql-scalar.md#numeric-functions) - ## ROW_NUMBER `ROW_NUMBER()` @@ -2317,133 +1325,44 @@ Returns the number of the row within the window starting from 1. ## RPAD -Returns a string of size `length` from `expr`. When the length of `expr` is less than `length`, right pads `expr` with `chars`, which defaults to the space character. Truncates `expr` to `length` if `length` is shorter than the length of `expr`. +`RPAD(, , [])` -* **Syntax:** `RPAD(expr, length[, chars])` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example right pads the value of `OriginStateName` from the `flight-carriers` datasource to return a total of 11 characters. - -```sql -SELECT - "OriginStateName" AS "origin_state", - RPAD("OriginStateName", 11, '+') AS "add_right_padding" -FROM "flight-carriers" -LIMIT 3 -``` - -Returns the following: - -| `origin_state` | `add_right_padding` | -| -- | -- | -| `Puerto Rico` | `Puerto Rico` | -| `Massachusetts` | `Massachuset` | -| `Florida` | `Florida++++` | - -
- -[Learn more](sql-scalar.md#string-functions) +Returns the rightmost number of characters from an expression, optionally padded with the given characters. ## RTRIM -Trims characters from the trailing end of an expression. Defaults `chars` to a space if none is provided. +`RTRIM(, [])` -* **Syntax:** `RTRIM(expr[, chars])` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example trims the `_` characters from the trailing end of the string expression. - -```sql -SELECT - '___abc___' AS "original_string", - RTRIM('___abc___', '_') AS "trim_end" -``` - -Returns the following: - -| `original_string` | `trim_end` | -| -- | -- | -| `___abc___` | `___abc` | - -
- -[Learn more](sql-scalar.md#string-functions) +Trims characters from the trailing end of an expression. ## SAFE_DIVIDE +`SAFE_DIVIDE(x, y)` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Returns `x` divided by `y`, guarded on division by 0. -* **Syntax:** `SAFE_DIVIDE(x, y)` -* **Function type:** Scalar, numeric - -
Example - -The following example calculates divisions of integer `78` by integer `10`. - -```sql -SELECT SAFE_DIVIDE(78, 10) AS "safe_division" -``` - -Returns the following: - -|`safe_division`| -|--| -| `7` | - -
- -[Learn more](sql-scalar.md#numeric-functions) - ## SIN +`SIN(expr)` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Calculates the trigonometric sine of an angle expressed in radians. -* **Syntax:** `SIN(expr)` -* **Function type:** Scalar, numeric - -
Example - -The following example calculates the sine of angle `PI/3` radians. - -```sql -SELECT SIN(PI / 3) AS "sine" -``` -Returns the following: - -| `sine` | -| -- | -| `0.8660254037844386` | -
- -[Learn more](sql-scalar.md#numeric-functions) - ## SQRT +`SQRT(expr)` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Calculates the square root of a numeric expression. -* **Syntax:** `SQRT()` -* **Function type:** Scalar, numeric - -
Example - -The following example calculates the square root of 25. - -```sql -SELECT SQRT(25) AS "square_root" -``` -Returns the following: - -| `square_root` | -| -- | -| `5` | -
- -[Learn more](sql-scalar.md#numeric-functions) - ## STDDEV `STDDEV(expr)` @@ -2487,33 +1406,11 @@ Splits `str1` into an array on the delimiter specified by `str2`, which is a reg ## STRING_FORMAT -Returns a string formatted in the manner of Java's [String.format](https://docs.oracle.com/javase/8/docs/api/java/lang/String.html#format-java.lang.String-java.lang.Object...-). +`STRING_FORMAT(pattern[, args...])` -* **Syntax:** `STRING_FORMAT(pattern[, args...])` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example uses Java String format to pass in `Flight_Number_Reporting_Airline` and `origin_airport` columns, from the `flight-carriers` datasource, as arguments into the string. - -```sql -SELECT - "Flight_Number_Reporting_Airline" AS "flight_number", - "Origin" AS "origin_airport", - STRING_FORMAT('Flight No.%d departing from %s', "Flight_Number_Reporting_Airline", "Origin") AS "departure_announcement" -FROM "flight-carriers" -LIMIT 1 -``` - -Returns the following: - -| `flight_number` | `origin_airport` | `departure_announcement` | -| -- | -- | -- | -| `314` | `SJU` | `Flight No.314 departing from SJU` | - -
- -[Learn more](sql-scalar.md#string-functions) +Returns a string formatted in accordance to Java's String.format method. ## STRING_TO_MV @@ -2525,78 +1422,35 @@ Splits `str1` into an multi-value string on the delimiter specified by `str2`, w ## STRLEN +`STRLEN(expr)` + +**Function type:** [Scalar, string](sql-scalar.md#string-functions) + Alias for [`LENGTH`](#length). -* **Syntax:** `STRLEN(expr)` -* **Function type:** Scalar, string - -[Learn more](sql-scalar.md#string-functions) - ## STRPOS -Returns the one-based index position of a substring within an expression. If `substring` is not found, returns 0. +`STRPOS(, )` -* **Syntax:** `STRPOS(expr, substring)` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example returns the one-based index position of `World`. - -```sql -SELECT - 'Hello World!' AS "original_string", - STRPOS('Hello World!', 'World') AS "index" -``` - -Returns the following: - -| `original_string` | `index` | -| -- | -- | -| `Hello World!` | `7` | - -
- -[Learn more](sql-scalar.md#string-functions) +Returns the one-based index position of a substring within an expression. ## SUBSTR +`SUBSTR(, , [])` + +**Function type:** [Scalar, string](sql-scalar.md#string-functions) + Alias for [`SUBSTRING`](#substring). -* **Syntax:** `SUBSTR(expr, index[, length])` -* **Function type:** Scalar, string - -[Learn more](sql-scalar.md#string-functions) - - ## SUBSTRING -Returns a substring of the expression starting at a given one-based index. If `length` is omitted, extracts characters to the end of the string, otherwise returns a substring of `length` characters. +`SUBSTRING(, , [])` -* **Syntax:** `SUBSTRING(expr, index[, length])` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example extracts a substring from the string expression `abcdefghi` of length `3` starting at index `4` - -```sql -SELECT - 'abcdefghi' AS "original_string", - SUBSTRING('abcdefghi', 4, 3) AS "substring" -``` - -Returns the following: - -| `original_string` | `substring` | -| -- | -- | -| `abcdefghi` | `def` | - -
- - - -[Learn more](sql-scalar.md#string-functions) +Returns a substring of the expression starting at a given one-based index. ## SUM @@ -2608,27 +1462,12 @@ Calculates the sum of a set of values. ## TAN +`TAN(expr)` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Calculates the trigonometric tangent of an angle expressed in radians. -* **Syntax:** `TAN(expr)` -* **Function type:** Scalar, numeric - -
Example - -The following example calculates the tangent of angle `PI/3` radians. - -```sql -SELECT TAN(PI / 3) AS "tangent" -``` -Returns the following: - -| `tangent` | -| -- | -| `1.7320508075688767` | -
- -[Learn more](sql-scalar.md#numeric-functions) - ## TDIGEST_GENERATE_SKETCH `TDIGEST_GENERATE_SKETCH(expr, [compression])` @@ -2647,33 +1486,12 @@ Returns the quantile for the specified fraction from a T-Digest sketch construct ## TEXTCAT +`TEXTCAT(, )` + +**Function type:** [Scalar, string](sql-scalar.md#string-functions) + Concatenates two string expressions. -* **Syntax:** `TEXTCAT(expr, expr)` -* **Function type:** Scalar, string - -
Example - -The following example concatenates the `OriginState` column from the `flight-carriers` datasource to `, USA`. - -```sql -SELECT - "OriginState" AS "origin_state", - TEXTCAT("OriginState", ', USA') AS "concatenate_state_with_USA" -FROM "flight-carriers" -LIMIT 1 -``` - -Returns the following: - -| `origin_state` | `concatenate_state_with_USA` | -| -- | -- | -| `PR` | `PR, USA` | - -
- -[Learn more](sql-scalar.md#string-functions) - ## THETA_SKETCH_ESTIMATE `THETA_SKETCH_ESTIMATE(expr)` @@ -2716,121 +1534,35 @@ Returns a union of Theta sketches. ## TIME_CEIL -Rounds up a timestamp to a given ISO 8601 time period. You can specify `origin` to provide a reference timestamp from which to start rounding. If provided, `timezone` should be a time zone name like `America/Los_Angeles` or an offset like `-08:00`. +`TIME_CEIL(, , [, []])` -* **Syntax:** `TIME_CEIL(timestamp_expr, period[, origin[, timezone]])` -* **Function type:** Scalar, date and time +**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions) -
Example - -The following example rounds up the `__time` column from the `taxi-trips` datasource to the nearest 45th minute in reference to the timestamp `2013-08-01 08:0:00`. - -```sql -SELECT - "__time" AS "original_timestamp", - TIME_CEIL("__time", 'PT45M', TIMESTAMP '2013-08-01 08:0:00') AS "time_ceiling" -FROM "taxi-trips" -LIMIT 2 -``` - -Returns the following: - -| `original_timestamp` | `time_ceiling` | -| -- | -- | -| `2013-08-01T08:14:37.000Z` | `2013-08-01T08:45:00.000Z` | -| `2013-08-01T09:13:00.000Z` | `2013-08-01T09:30:00.000Z` | -
- -[Learn more](sql-scalar.md#date-and-time-functions) +Rounds up a timestamp by a given time period, optionally from some reference time or timezone. ## TIME_EXTRACT -Extracts the value of `unit` from the timestamp and returns it as a number. If provided, `timezone` should be a time zone name like `America/Los_Angeles` or an offset like `-08:00`. +`TIME_EXTRACT(, [, []])` -* **Syntax:** `TIME_EXTRACT(timestamp_expr[, unit[, timezone]])` -* **Function type:** Scalar, date and time +**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions) -
Example - -The following example extracts the hour from the `__time` column in the `taxi-trips` datasource and offsets its timezone by `-04:00` hours. - -```sql -SELECT - "__time" AS "original_timestamp", - TIME_EXTRACT("__time", 'hour', '-04:00') AS "extract_hour" -FROM "taxi-trips" -LIMIT 2 -``` - -Returns the following: - -| `original_timestamp` | `extract_hour` | -| -- | -- | -| `2013-08-01T08:14:37.000Z` | `4` | -| `2013-08-01T09:13:00.000Z` | `5` | - -
- -[Learn more](sql-scalar.md#date-and-time-functions) +Extracts the value of some unit of the timestamp and returns the number. ## TIME_FLOOR -Rounds down a timestamp to a given ISO 8601 time period. You can specify `origin` to provide a reference timestamp from which to start rounding. If provided, `timezone` should be a time zone name like `America/Los_Angeles` or an offset like `-08:00`. +`TIME_FLOOR(, , [, []])` -* **Syntax:** `TIME_FLOOR(timestamp_expr, period[, origin[, timezone]])` -* **Function type:** Scalar, date and time +**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions) -
Example - -The following example rounds down the `__time` column from the `taxi-trips` datasource to the nearest 45th minute in reference to the timestamp `2013-08-01 08:0:00`. - -```sql -SELECT - "__time" AS "original_timestamp", - TIME_FLOOR("__time", 'PT45M', TIMESTAMP '2013-08-01 08:0:00') AS "time_floor" -FROM "taxi-trips" -LIMIT 2 -``` - -Returns the following: - -| `original_timestamp` | `time_floor` | -| -- | -- | -| `2013-08-01T08:14:37.000Z` | `2013-08-01T08:00:00.000Z` | -| `2013-08-01T09:13:00.000Z` | `2013-08-01T08:45:00.000Z` | - -
- -[Learn more](sql-scalar.md#date-and-time-functions) +Rounds down a timestamp by a given time period, optionally from some reference time or timezone. ## TIME_FORMAT -Formats a timestamp as a string in a provided [Joda DateTimeFormat pattern](http://www.joda.org/joda-time/apidocs/org/joda/time/format/DateTimeFormat.html). If no pattern is provided, `pattern` defaults to ISO 8601. If provided, `timezone` should be a time zone name like `America/Los_Angeles` or an offset like `-08:00`. +`TIME_FORMAT(, [, []])` -* **Syntax:** `TIME_FORMAT(timestamp_expr[, pattern[, timezone]])` -* **Function type:** Scalar, date and time +**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions) -
Example - -The following example formats the `__time` column from the `flight-carriers` datasource into a string format and offsets the result's timezone by `-05:00` hours. - -```sql -SELECT - "__time" AS "original_time", -TIME_FORMAT( "__time", 'dd-MM-YYYY hh:mm aa zzz', '-05:00') AS "string" -FROM "taxi-trips" -LIMIT 1 -``` - -Returns the following: - -| `original_time` | `string` | -| -- | -- | -| `2013-08-01T08:14:37.000Z` | `01-08-2013 03:14 AM -05:00` | - -
- -[Learn more](sql-scalar.md#date-and-time-functions) +Formats a timestamp as a string. ## TIME_IN_INTERVAL @@ -2842,62 +1574,19 @@ Returns whether a timestamp is contained within a particular interval, formatted ## TIME_PARSE -Parses a string into a timestamp using a given [Joda DateTimeFormat pattern](http://www.joda.org/joda-time/apidocs/org/joda/time/format/DateTimeFormat.html). If no pattern is provided, `pattern` defaults to ISO 8601. Returns NULL if string cannot be parsed. If provided, `timezone` should be a time zone name like `America/Los_Angeles` or an offset like `-08:00`. +`TIME_PARSE(, [, []])` -* **Syntax:** `TIME_PARSE(string_expr[, pattern[, timezone]])` -* **Function type:** Scalar, date and time +**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions) -
Example - -The following example parses the `FlightDate` STRING column from the `flight-carriers` datasource into a valid timestamp with an offset of `-05:00` hours. - -```sql -SELECT - "FlightDate" AS "original_string", - TIME_PARSE("FlightDate", 'YYYY-MM-dd', '-05:00') AS "timestamp" -FROM "flight-carriers" -LIMIT 1 -``` - -Returns the following: - -| `original_string` | `timestamp` | -| -- | -- | -| `2005-11-01` | `2005-11-01T05:00:00.000Z` | - -
- -[Learn more](sql-scalar.md#date-and-time-functions) +Parses a string into a timestamp. ## TIME_SHIFT -Shifts a timestamp by a given number of time units. The `period` parameter can be any ISO 8601 period. The `step` parameter can be negative. If provided, `timezone` should be a time zone name like `America/Los_Angeles` or an offset like `-08:00`. +`TIME_SHIFT(, , , [])` -* **Syntax:** `TIME_SHIFT(timestamp_expr, period, step[, timezone])` -* **Function type:** Scalar, date and time - -
Example - -The following example shifts the `__time` column from the `taxi-trips` datasource back by 24 hours. - -```sql -SELECT - "__time" AS "original_timestamp", - TIME_SHIFT("__time", 'PT1H', -24) AS "shift_back" -FROM "taxi-trips" -LIMIT 1 -``` - -Returns the following: - -| `original_timestamp` | `shift_back` | -| -- | -- | -| `2013-08-01T08:14:37.000Z` | `2013-07-31T08:14:37.000Z` | - -
- -[Learn more](sql-scalar.md#date-and-time-functions) +**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions) +Shifts a timestamp forwards or backwards by a given number of time units. ## TIMESTAMP_TO_MILLIS @@ -2928,73 +1617,34 @@ Takes the difference between two timestamps, returning the results in the given **Function type:** [JSON](sql-json-functions.md) `TO_JSON_STRING(expr)` + Serializes `expr` into a JSON string. ## TRIM -Trims the leading and/or trailing characters of an expression. Defaults `chars` to a space if none is provided. Defaults to `BOTH` if no directional argument is provided. +`TRIM([BOTH|LEADING|TRAILING] [ FROM] expr)` -* **Syntax:** `TRIM([BOTH|LEADING|TRAILING] [chars FROM] expr)` -* **Function type:** Scalar, string +**Function type:** [Scalar, string](sql-scalar.md#string-functions) -
Example - -The following example trims `_` characters from both ends of the string expression. - -```sql -SELECT - '___abc___' AS "original_string", - TRIM( BOTH '_' FROM '___abc___') AS "trim_expression" -``` - -Returns the following: - -| `original_string` | `trim_expression` | -| -- | -- | -| `___abc___` | `abc` | - -
- -[Learn more](sql-scalar.md#string-functions) +Trims the leading or trailing characters of an expression. ## TRUNC +`TRUNC(expr[, digits])` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Alias for [`TRUNCATE`](#truncate). -* **Syntax:** `TRUNC(expr[, digits])` -* **Function type:** Scalar, numeric - -[Learn more](sql-scalar.md#numeric-functions) - ## TRUNCATE +`TRUNCATE(expr[, digits])` + +**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) + Truncates a numerical expression to a specific number of decimal digits. -* **Syntax:** `TRUNCATE(expr[, digits])` -* **Function type:** Scalar, numeric - -
Example - -The following applies the TRUNCATE function to 1 decimal place on the `pickup_longitude` column from the `taxi-trips` datasource. - -```sql -SELECT - "pickup_longitude" as "pickup_longitude", - TRUNCATE("pickup_longitude", 1) as "truncate_pickup_longitude" -FROM "taxi-trips" -WHERE "pickup_longitude" IS NOT NULL -LIMIT 1 -``` -Returns the following: - -| `pickup_longitude` | `truncate_pickup_longitude` | -| -- | -- | -| `-73.9377670288086` | `-73.9` | -
- - -[Learn more](sql-scalar.md#numeric-functions) ## TRY_PARSE_JSON @@ -3014,33 +1664,12 @@ For more information, see [UNNEST](./sql.md#unnest). ## UPPER +`UPPER(expr)` + +**Function type:** [Scalar, string](sql-scalar.md#string-functions) + Returns the expression in uppercase. -* **Syntax:** `UPPER(expr)` -* **Function type:** Scalar, string - -
Example - -The following example converts the `OriginCityName` column from the `flight-carriers` datasource to uppercase. - -```sql -SELECT - "OriginCityName" AS "origin_city", - UPPER("OriginCityName") AS "uppercase" -FROM "flight-carriers" -LIMIT 1 -``` - -Returns the following: - -| `origin_city` | `uppercase` | -| -- | -- | -`San Juan, PR` | `SAN JUAN, PR` | - -
- -[Learn more](sql-scalar.md#string-functions) - ## VAR_POP `VAR_POP(expr)` @@ -3064,5 +1693,3 @@ Calculates the sample variance of a set of values. **Function type:** [Aggregation](sql-aggregations.md) Alias for [`VAR_SAMP`](#var_samp). - - diff --git a/docs/querying/sql-json-functions.md b/docs/querying/sql-json-functions.md index 199c568c29b..35b4f5e3769 100644 --- a/docs/querying/sql-json-functions.md +++ b/docs/querying/sql-json-functions.md @@ -38,6 +38,7 @@ You can use the following JSON functions to extract, transform, and create `COMP | --- | --- | |`JSON_KEYS(expr, path)`| Returns an array of field names from `expr` at the specified `path`.| |`JSON_OBJECT(KEY expr1 VALUE expr2[, KEY expr3 VALUE expr4, ...])` | Constructs a new `COMPLEX` object. The `KEY` expressions must evaluate to string types. The `VALUE` expressions can be composed of any input type, including other `COMPLEX` values. `JSON_OBJECT` can accept colon-separated key-value pairs. The following syntax is equivalent: `JSON_OBJECT(expr1:expr2[, expr3:expr4, ...])`.| +|`JSON_MERGE(expr1, expr2[, expr3 ...])`| Merges two or more JSON `STRING` or `COMPLEX` into one. Preserves the rightmost value when there are key overlaps. Returning always a `COMPLEX` type.| |`JSON_PATHS(expr)`| Returns an array of all paths which refer to literal values in `expr` in JSONPath format. | |`JSON_QUERY(expr, path)`| Extracts a `COMPLEX` value from `expr`, at the specified `path`. | |`JSON_QUERY_ARRAY(expr, path)`| Extracts an `ARRAY>` value from `expr` at the specified `path`. If value is not an `ARRAY`, it gets translated into a single element `ARRAY` containing the value at `path`. The primary use of this function is to extract arrays of objects to use as inputs to other [array functions](./sql-array-functions.md).| diff --git a/docs/querying/sql-metadata-tables.md b/docs/querying/sql-metadata-tables.md index 4fbbfb2ff4c..b04b3f11e65 100644 --- a/docs/querying/sql-metadata-tables.md +++ b/docs/querying/sql-metadata-tables.md @@ -266,7 +266,7 @@ GROUP BY servers.server; ### TASKS table -The tasks table provides information about active and recently-completed indexing tasks. For more information +The tasks table provides information about active and recently completed tasks. For more information check out the documentation for [ingestion tasks](../ingestion/tasks.md). |Column|Type|Notes| diff --git a/docs/querying/sql-scalar.md b/docs/querying/sql-scalar.md index 28e889b8887..8a8c47160da 100644 --- a/docs/querying/sql-scalar.md +++ b/docs/querying/sql-scalar.md @@ -173,10 +173,10 @@ overhead. |`MILLIS_TO_TIMESTAMP(millis_expr)`|Converts a number of milliseconds since the epoch (1970-01-01 00:00:00 UTC) into a timestamp.| |`TIMESTAMP_TO_MILLIS(timestamp_expr)`|Converts a timestamp into a number of milliseconds since the epoch.| |`EXTRACT(unit FROM timestamp_expr)`|Extracts a time part from `expr`, returning it as a number. Unit can be EPOCH, MICROSECOND, MILLISECOND, SECOND, MINUTE, HOUR, DAY (day of month), DOW (day of week), ISODOW (ISO day of week), DOY (day of year), WEEK (week of year), MONTH, QUARTER, YEAR, ISOYEAR, DECADE, CENTURY or MILLENNIUM. Units must be provided unquoted, like `EXTRACT(HOUR FROM __time)`.| -|`FLOOR(timestamp_expr TO unit)`|Rounds down a timestamp, returning it as a new timestamp. Unit can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.| -|`CEIL(timestamp_expr TO unit)`|Rounds up a timestamp, returning it as a new timestamp. Unit can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.| -|`TIMESTAMPADD(unit, count, timestamp)`|Equivalent to `timestamp + count * INTERVAL '1' UNIT`.| -|`TIMESTAMPDIFF(unit, timestamp1, timestamp2)`|Returns the (signed) number of `unit` between `timestamp1` and `timestamp2`. Unit can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.| +|`FLOOR(timestamp_expr TO unit)`|Rounds down a timestamp, returning it as a new timestamp. The `unit` parameter must be unquoted and can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.| +|`CEIL(timestamp_expr TO unit)`|Rounds up a timestamp, returning it as a new timestamp. The `unit` parameter must be unquoted and can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.| +|`TIMESTAMPADD(unit, count, timestamp)`|Adds a `count` number of time `unit` to timestamp, equivalent to `timestamp + count * unit`. The `unit` parameter must be unquoted and can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.| +|`TIMESTAMPDIFF(unit, timestamp1, timestamp2)`|Returns a signed number of `unit` between `timestamp1` and `timestamp2`. The `unit` parameter must be unquoted and can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.| ## Reduction functions diff --git a/examples/conf/druid/auto/_common/common.jvm.config b/examples/conf/druid/auto/_common/common.jvm.config index fd74cf35897..0c488a2bcea 100644 --- a/examples/conf/druid/auto/_common/common.jvm.config +++ b/examples/conf/druid/auto/_common/common.jvm.config @@ -1,6 +1,5 @@ -server -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/cluster/master/coordinator-overlord/jvm.config b/examples/conf/druid/cluster/master/coordinator-overlord/jvm.config index 5df7d606725..00cfa42c630 100644 --- a/examples/conf/druid/cluster/master/coordinator-overlord/jvm.config +++ b/examples/conf/druid/cluster/master/coordinator-overlord/jvm.config @@ -2,7 +2,6 @@ -Xms15g -Xmx15g -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/cluster/query/router/jvm.config b/examples/conf/druid/cluster/query/router/jvm.config index a844c40b1ea..8fd91200735 100644 --- a/examples/conf/druid/cluster/query/router/jvm.config +++ b/examples/conf/druid/cluster/query/router/jvm.config @@ -1,7 +1,6 @@ -server -Xms1g -Xmx1g --XX:+UseG1GC -XX:MaxDirectMemorySize=128m -XX:+ExitOnOutOfMemoryError -Duser.timezone=UTC diff --git a/examples/conf/druid/single-server/large/broker/jvm.config b/examples/conf/druid/single-server/large/broker/jvm.config index 6c43c24dbb4..f554b705e10 100644 --- a/examples/conf/druid/single-server/large/broker/jvm.config +++ b/examples/conf/druid/single-server/large/broker/jvm.config @@ -3,7 +3,6 @@ -Xmx12g -XX:MaxDirectMemorySize=11g -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/large/coordinator-overlord/jvm.config b/examples/conf/druid/single-server/large/coordinator-overlord/jvm.config index 5df7d606725..00cfa42c630 100644 --- a/examples/conf/druid/single-server/large/coordinator-overlord/jvm.config +++ b/examples/conf/druid/single-server/large/coordinator-overlord/jvm.config @@ -2,7 +2,6 @@ -Xms15g -Xmx15g -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/large/historical/jvm.config b/examples/conf/druid/single-server/large/historical/jvm.config index 16e1f5d7825..8b105559b00 100644 --- a/examples/conf/druid/single-server/large/historical/jvm.config +++ b/examples/conf/druid/single-server/large/historical/jvm.config @@ -3,7 +3,6 @@ -Xmx16g -XX:MaxDirectMemorySize=25g -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/large/middleManager/jvm.config b/examples/conf/druid/single-server/large/middleManager/jvm.config index 82597e0b956..69d307c3120 100644 --- a/examples/conf/druid/single-server/large/middleManager/jvm.config +++ b/examples/conf/druid/single-server/large/middleManager/jvm.config @@ -2,7 +2,6 @@ -Xms256m -Xmx256m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/large/router/jvm.config b/examples/conf/druid/single-server/large/router/jvm.config index a844c40b1ea..8fd91200735 100644 --- a/examples/conf/druid/single-server/large/router/jvm.config +++ b/examples/conf/druid/single-server/large/router/jvm.config @@ -1,7 +1,6 @@ -server -Xms1g -Xmx1g --XX:+UseG1GC -XX:MaxDirectMemorySize=128m -XX:+ExitOnOutOfMemoryError -Duser.timezone=UTC diff --git a/examples/conf/druid/single-server/medium/broker/jvm.config b/examples/conf/druid/single-server/medium/broker/jvm.config index a4bf3d91097..a2cbe790878 100644 --- a/examples/conf/druid/single-server/medium/broker/jvm.config +++ b/examples/conf/druid/single-server/medium/broker/jvm.config @@ -3,7 +3,6 @@ -Xmx8g -XX:MaxDirectMemorySize=5g -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/medium/coordinator-overlord/jvm.config b/examples/conf/druid/single-server/medium/coordinator-overlord/jvm.config index dbddd50ce8d..485137ad3d8 100644 --- a/examples/conf/druid/single-server/medium/coordinator-overlord/jvm.config +++ b/examples/conf/druid/single-server/medium/coordinator-overlord/jvm.config @@ -2,7 +2,6 @@ -Xms9g -Xmx9g -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/medium/historical/jvm.config b/examples/conf/druid/single-server/medium/historical/jvm.config index f31ae92ba9f..891312f3c68 100644 --- a/examples/conf/druid/single-server/medium/historical/jvm.config +++ b/examples/conf/druid/single-server/medium/historical/jvm.config @@ -3,7 +3,6 @@ -Xmx8g -XX:MaxDirectMemorySize=13g -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/medium/middleManager/jvm.config b/examples/conf/druid/single-server/medium/middleManager/jvm.config index 82597e0b956..69d307c3120 100644 --- a/examples/conf/druid/single-server/medium/middleManager/jvm.config +++ b/examples/conf/druid/single-server/medium/middleManager/jvm.config @@ -2,7 +2,6 @@ -Xms256m -Xmx256m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/medium/router/jvm.config b/examples/conf/druid/single-server/medium/router/jvm.config index 6fc9b4cdfef..ebecad4435e 100644 --- a/examples/conf/druid/single-server/medium/router/jvm.config +++ b/examples/conf/druid/single-server/medium/router/jvm.config @@ -1,7 +1,6 @@ -server -Xms512m -Xmx512m --XX:+UseG1GC -XX:MaxDirectMemorySize=128m -XX:+ExitOnOutOfMemoryError -Duser.timezone=UTC diff --git a/examples/conf/druid/single-server/micro-quickstart/broker/jvm.config b/examples/conf/druid/single-server/micro-quickstart/broker/jvm.config index cf72e5d2df9..5d666f6db23 100644 --- a/examples/conf/druid/single-server/micro-quickstart/broker/jvm.config +++ b/examples/conf/druid/single-server/micro-quickstart/broker/jvm.config @@ -3,7 +3,6 @@ -Xmx512m -XX:MaxDirectMemorySize=768m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/jvm.config b/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/jvm.config index 8ab5708c320..aae520d8325 100644 --- a/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/jvm.config +++ b/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/jvm.config @@ -2,7 +2,6 @@ -Xms256m -Xmx256m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/micro-quickstart/historical/jvm.config b/examples/conf/druid/single-server/micro-quickstart/historical/jvm.config index 8747402e85a..79edb06e203 100644 --- a/examples/conf/druid/single-server/micro-quickstart/historical/jvm.config +++ b/examples/conf/druid/single-server/micro-quickstart/historical/jvm.config @@ -3,7 +3,6 @@ -Xmx512m -XX:MaxDirectMemorySize=1280m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/micro-quickstart/middleManager/jvm.config b/examples/conf/druid/single-server/micro-quickstart/middleManager/jvm.config index 30b023cfb8d..8a650fb358c 100644 --- a/examples/conf/druid/single-server/micro-quickstart/middleManager/jvm.config +++ b/examples/conf/druid/single-server/micro-quickstart/middleManager/jvm.config @@ -2,7 +2,6 @@ -Xms64m -Xmx64m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/micro-quickstart/router/jvm.config b/examples/conf/druid/single-server/micro-quickstart/router/jvm.config index e74d6b1375f..b822f460179 100644 --- a/examples/conf/druid/single-server/micro-quickstart/router/jvm.config +++ b/examples/conf/druid/single-server/micro-quickstart/router/jvm.config @@ -1,7 +1,6 @@ -server -Xms128m -Xmx128m --XX:+UseG1GC -XX:MaxDirectMemorySize=128m -XX:+ExitOnOutOfMemoryError -Duser.timezone=UTC diff --git a/examples/conf/druid/single-server/nano-quickstart/broker/jvm.config b/examples/conf/druid/single-server/nano-quickstart/broker/jvm.config index 8551ec59fb0..93d33903541 100644 --- a/examples/conf/druid/single-server/nano-quickstart/broker/jvm.config +++ b/examples/conf/druid/single-server/nano-quickstart/broker/jvm.config @@ -3,7 +3,6 @@ -Xmx512m -XX:MaxDirectMemorySize=400m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/jvm.config b/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/jvm.config index 8ab5708c320..aae520d8325 100644 --- a/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/jvm.config +++ b/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/jvm.config @@ -2,7 +2,6 @@ -Xms256m -Xmx256m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/nano-quickstart/historical/jvm.config b/examples/conf/druid/single-server/nano-quickstart/historical/jvm.config index 8551ec59fb0..93d33903541 100644 --- a/examples/conf/druid/single-server/nano-quickstart/historical/jvm.config +++ b/examples/conf/druid/single-server/nano-quickstart/historical/jvm.config @@ -3,7 +3,6 @@ -Xmx512m -XX:MaxDirectMemorySize=400m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/nano-quickstart/middleManager/jvm.config b/examples/conf/druid/single-server/nano-quickstart/middleManager/jvm.config index 30b023cfb8d..8a650fb358c 100644 --- a/examples/conf/druid/single-server/nano-quickstart/middleManager/jvm.config +++ b/examples/conf/druid/single-server/nano-quickstart/middleManager/jvm.config @@ -2,7 +2,6 @@ -Xms64m -Xmx64m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/nano-quickstart/router/jvm.config b/examples/conf/druid/single-server/nano-quickstart/router/jvm.config index e74d6b1375f..b822f460179 100644 --- a/examples/conf/druid/single-server/nano-quickstart/router/jvm.config +++ b/examples/conf/druid/single-server/nano-quickstart/router/jvm.config @@ -1,7 +1,6 @@ -server -Xms128m -Xmx128m --XX:+UseG1GC -XX:MaxDirectMemorySize=128m -XX:+ExitOnOutOfMemoryError -Duser.timezone=UTC diff --git a/examples/conf/druid/single-server/small/broker/jvm.config b/examples/conf/druid/single-server/small/broker/jvm.config index 1017bce2ab5..1bc85dd7702 100644 --- a/examples/conf/druid/single-server/small/broker/jvm.config +++ b/examples/conf/druid/single-server/small/broker/jvm.config @@ -3,7 +3,6 @@ -Xmx4g -XX:MaxDirectMemorySize=3g -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/small/coordinator-overlord/jvm.config b/examples/conf/druid/single-server/small/coordinator-overlord/jvm.config index 34176680c41..2d9b10d9501 100644 --- a/examples/conf/druid/single-server/small/coordinator-overlord/jvm.config +++ b/examples/conf/druid/single-server/small/coordinator-overlord/jvm.config @@ -2,7 +2,6 @@ -Xms4500m -Xmx4500m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/small/historical/jvm.config b/examples/conf/druid/single-server/small/historical/jvm.config index 21906078471..a567647d543 100644 --- a/examples/conf/druid/single-server/small/historical/jvm.config +++ b/examples/conf/druid/single-server/small/historical/jvm.config @@ -3,7 +3,6 @@ -Xmx4g -XX:MaxDirectMemorySize=8g -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/small/middleManager/jvm.config b/examples/conf/druid/single-server/small/middleManager/jvm.config index a5862186437..c145532ba1e 100644 --- a/examples/conf/druid/single-server/small/middleManager/jvm.config +++ b/examples/conf/druid/single-server/small/middleManager/jvm.config @@ -2,7 +2,6 @@ -Xms128m -Xmx128m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/small/router/jvm.config b/examples/conf/druid/single-server/small/router/jvm.config index ad8a52de508..ebecad4435e 100644 --- a/examples/conf/druid/single-server/small/router/jvm.config +++ b/examples/conf/druid/single-server/small/router/jvm.config @@ -1,10 +1,8 @@ -server -Xms512m -Xmx512m --XX:+UseG1GC -XX:MaxDirectMemorySize=128m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/xlarge/broker/jvm.config b/examples/conf/druid/single-server/xlarge/broker/jvm.config index f83ad0e18b6..b683fd7dd7a 100644 --- a/examples/conf/druid/single-server/xlarge/broker/jvm.config +++ b/examples/conf/druid/single-server/xlarge/broker/jvm.config @@ -3,7 +3,6 @@ -Xmx16g -XX:MaxDirectMemorySize=12g -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/xlarge/coordinator-overlord/jvm.config b/examples/conf/druid/single-server/xlarge/coordinator-overlord/jvm.config index f3ca0fdbb9e..68f9f93c60d 100644 --- a/examples/conf/druid/single-server/xlarge/coordinator-overlord/jvm.config +++ b/examples/conf/druid/single-server/xlarge/coordinator-overlord/jvm.config @@ -2,7 +2,6 @@ -Xms18g -Xmx18g -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/xlarge/historical/jvm.config b/examples/conf/druid/single-server/xlarge/historical/jvm.config index 92553d3c8b4..6d91df0aef0 100644 --- a/examples/conf/druid/single-server/xlarge/historical/jvm.config +++ b/examples/conf/druid/single-server/xlarge/historical/jvm.config @@ -3,7 +3,6 @@ -Xmx24g -XX:MaxDirectMemorySize=44g -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/xlarge/middleManager/jvm.config b/examples/conf/druid/single-server/xlarge/middleManager/jvm.config index 82597e0b956..69d307c3120 100644 --- a/examples/conf/druid/single-server/xlarge/middleManager/jvm.config +++ b/examples/conf/druid/single-server/xlarge/middleManager/jvm.config @@ -2,7 +2,6 @@ -Xms256m -Xmx256m -XX:+ExitOnOutOfMemoryError --XX:+UseG1GC -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Djava.io.tmpdir=var/tmp diff --git a/examples/conf/druid/single-server/xlarge/router/jvm.config b/examples/conf/druid/single-server/xlarge/router/jvm.config index a844c40b1ea..8fd91200735 100644 --- a/examples/conf/druid/single-server/xlarge/router/jvm.config +++ b/examples/conf/druid/single-server/xlarge/router/jvm.config @@ -1,7 +1,6 @@ -server -Xms1g -Xmx1g --XX:+UseG1GC -XX:MaxDirectMemorySize=128m -XX:+ExitOnOutOfMemoryError -Duser.timezone=UTC diff --git a/extensions-contrib/aliyun-oss-extensions/pom.xml b/extensions-contrib/aliyun-oss-extensions/pom.xml index f83b0b5b339..bffaed8c065 100644 --- a/extensions-contrib/aliyun-oss-extensions/pom.xml +++ b/extensions-contrib/aliyun-oss-extensions/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java index 207cddf90b4..12ad9fc8bf9 100644 --- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java +++ b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java @@ -537,7 +537,7 @@ public class OssInputSourceTest extends InitializedNullHandlingTest InputSourceReader reader = inputSource.reader( someSchema, - new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0), + new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null), temporaryFolder.newFolder() ); @@ -584,7 +584,7 @@ public class OssInputSourceTest extends InitializedNullHandlingTest InputSourceReader reader = inputSource.reader( someSchema, - new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0), + new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null), temporaryFolder.newFolder() ); diff --git a/extensions-contrib/ambari-metrics-emitter/pom.xml b/extensions-contrib/ambari-metrics-emitter/pom.xml index 95773a63918..f1ad243e37c 100644 --- a/extensions-contrib/ambari-metrics-emitter/pom.xml +++ b/extensions-contrib/ambari-metrics-emitter/pom.xml @@ -24,7 +24,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/cassandra-storage/pom.xml b/extensions-contrib/cassandra-storage/pom.xml index f502562a816..7b5fc2be183 100644 --- a/extensions-contrib/cassandra-storage/pom.xml +++ b/extensions-contrib/cassandra-storage/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml index 2fca74ef55b..01b006dd5d5 100644 --- a/extensions-contrib/cloudfiles-extensions/pom.xml +++ b/extensions-contrib/cloudfiles-extensions/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/compressed-bigdecimal/pom.xml b/extensions-contrib/compressed-bigdecimal/pom.xml index d6c27d88812..358bc3b26c5 100644 --- a/extensions-contrib/compressed-bigdecimal/pom.xml +++ b/extensions-contrib/compressed-bigdecimal/pom.xml @@ -25,7 +25,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/ddsketch/pom.xml b/extensions-contrib/ddsketch/pom.xml index 7318e953ff7..9c737a75c6a 100644 --- a/extensions-contrib/ddsketch/pom.xml +++ b/extensions-contrib/ddsketch/pom.xml @@ -22,7 +22,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/distinctcount/pom.xml b/extensions-contrib/distinctcount/pom.xml index 977d7550947..fd88a5bca2c 100644 --- a/extensions-contrib/distinctcount/pom.xml +++ b/extensions-contrib/distinctcount/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java index c06b5649681..d3edf0827b3 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java @@ -35,8 +35,8 @@ import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; @@ -104,7 +104,7 @@ public class DistinctCountTimeseriesQueryTest extends InitializedNullHandlingTes final Iterable> results = engine.process( query, - new IncrementalIndexStorageAdapter(index), + new IncrementalIndexCursorFactory(index), new IncrementalIndexTimeBoundaryInspector(index), new DefaultTimeseriesQueryMetrics() ).toList(); diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java index ff5a0c31363..7e3690cc1cb 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java @@ -33,13 +33,13 @@ import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.query.topn.TopNQueryEngine; import org.apache.druid.query.topn.TopNResultValue; -import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; +import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; import org.junit.After; import org.junit.Before; @@ -133,8 +133,7 @@ public class DistinctCountTopNQueryTest extends InitializedNullHandlingTest final Iterable> results = engine.query( query, - new IncrementalIndexStorageAdapter(index), - new IncrementalIndexTimeBoundaryInspector(index), + new IncrementalIndexSegment(index, SegmentId.dummy(QueryRunnerTestHelper.DATA_SOURCE)), null ).toList(); diff --git a/extensions-contrib/dropwizard-emitter/pom.xml b/extensions-contrib/dropwizard-emitter/pom.xml index b53950bf13f..028ac0eb48e 100644 --- a/extensions-contrib/dropwizard-emitter/pom.xml +++ b/extensions-contrib/dropwizard-emitter/pom.xml @@ -24,7 +24,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml index 71a167c315e..6a2b7cc0646 100644 --- a/extensions-contrib/druid-deltalake-extensions/pom.xml +++ b/extensions-contrib/druid-deltalake-extensions/pom.xml @@ -29,7 +29,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java index 01a18e9bc85..c4c2f2668b0 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java +++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java @@ -67,9 +67,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; /** - * Input source to ingest data from a Delta Lake. This input source reads the latest snapshot from a Delta table - * specified by {@code tablePath} parameter. If {@code filter} is specified, it's used at the Kernel level - * for data pruning. The filtering behavior is as follows: + * Input source to ingest data from a Delta Lake. This input source reads the given {@code snapshotVersion} from a Delta + * table specified by {@code tablePath} parameter, or the latest snapshot if it's not specified. + * If {@code filter} is specified, it's used at the Kernel level for data pruning. The filtering behavior is as follows: *
    *
  • When a filter is applied on a partitioned table using the partitioning columns, the filtering is guaranteed.
  • *
  • When a filter is applied on non-partitioned columns, the filtering is best-effort as the Delta @@ -79,7 +79,6 @@ import java.util.stream.Stream; *

    * We leverage the Delta Kernel APIs to interact with a Delta table. The Kernel API abstracts away the * complexities of the Delta protocol itself. - * Note: currently, the Kernel table API only supports reading from the latest snapshot. *

    */ public class DeltaInputSource implements SplittableInputSource @@ -97,11 +96,15 @@ public class DeltaInputSource implements SplittableInputSource @Nullable private final DeltaFilter filter; + @JsonProperty + private final Long snapshotVersion; + @JsonCreator public DeltaInputSource( @JsonProperty("tablePath") final String tablePath, @JsonProperty("deltaSplit") @Nullable final DeltaSplit deltaSplit, - @JsonProperty("filter") @Nullable final DeltaFilter filter + @JsonProperty("filter") @Nullable final DeltaFilter filter, + @JsonProperty("snapshotVersion") @Nullable final Long snapshotVersion ) { if (tablePath == null) { @@ -110,6 +113,7 @@ public class DeltaInputSource implements SplittableInputSource this.tablePath = tablePath; this.deltaSplit = deltaSplit; this.filter = filter; + this.snapshotVersion = snapshotVersion; } @Override @@ -152,15 +156,15 @@ public class DeltaInputSource implements SplittableInputSource } } else { final Table table = Table.forPath(engine, tablePath); - final Snapshot latestSnapshot = getLatestSnapshotForTable(table, engine); + final Snapshot snapshot = getSnapshotForTable(table, engine); - final StructType fullSnapshotSchema = latestSnapshot.getSchema(engine); + final StructType fullSnapshotSchema = snapshot.getSchema(engine); final StructType prunedSchema = pruneSchema( fullSnapshotSchema, inputRowSchema.getColumnsFilter() ); - final ScanBuilder scanBuilder = latestSnapshot.getScanBuilder(engine); + final ScanBuilder scanBuilder = snapshot.getScanBuilder(engine); if (filter != null) { scanBuilder.withFilter(engine, filter.getFilterPredicate(fullSnapshotSchema)); } @@ -206,17 +210,17 @@ public class DeltaInputSource implements SplittableInputSource } final Engine engine = createDeltaEngine(); - final Snapshot latestSnapshot; + final Snapshot snapshot; final Table table = Table.forPath(engine, tablePath); try { - latestSnapshot = getLatestSnapshotForTable(table, engine); + snapshot = getSnapshotForTable(table, engine); } catch (TableNotFoundException e) { throw InvalidInput.exception(e, "tablePath[%s] not found.", tablePath); } - final StructType fullSnapshotSchema = latestSnapshot.getSchema(engine); + final StructType fullSnapshotSchema = snapshot.getSchema(engine); - final ScanBuilder scanBuilder = latestSnapshot.getScanBuilder(engine); + final ScanBuilder scanBuilder = snapshot.getScanBuilder(engine); if (filter != null) { scanBuilder.withFilter(engine, filter.getFilterPredicate(fullSnapshotSchema)); } @@ -254,7 +258,8 @@ public class DeltaInputSource implements SplittableInputSource return new DeltaInputSource( tablePath, split.get(), - filter + filter, + snapshotVersion ); } @@ -333,7 +338,7 @@ public class DeltaInputSource implements SplittableInputSource ); } - private Snapshot getLatestSnapshotForTable(final Table table, final Engine engine) + private Snapshot getSnapshotForTable(final Table table, final Engine engine) { // Setting the LogStore class loader before calling the Delta Kernel snapshot API is required as a workaround with // the 3.2.0 Delta Kernel because the Kernel library cannot instantiate the LogStore class otherwise. Please see @@ -341,7 +346,11 @@ public class DeltaInputSource implements SplittableInputSource final ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); try { Thread.currentThread().setContextClassLoader(LogStore.class.getClassLoader()); - return table.getLatestSnapshot(engine); + if (snapshotVersion != null) { + return table.getSnapshotAsOfVersion(engine, snapshotVersion); + } else { + return table.getLatestSnapshot(engine); + } } finally { Thread.currentThread().setContextClassLoader(currCtxCl); @@ -359,4 +368,10 @@ public class DeltaInputSource implements SplittableInputSource { return filter; } + + @VisibleForTesting + Long getSnapshotVersion() + { + return snapshotVersion; + } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java index 4c1b57c434a..4de745b1927 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java @@ -55,7 +55,8 @@ public class DeltaInputRowTest Object[][] data = new Object[][]{ {NonPartitionedDeltaTable.DELTA_TABLE_PATH, NonPartitionedDeltaTable.FULL_SCHEMA, NonPartitionedDeltaTable.DIMENSIONS, NonPartitionedDeltaTable.EXPECTED_ROWS}, {PartitionedDeltaTable.DELTA_TABLE_PATH, PartitionedDeltaTable.FULL_SCHEMA, PartitionedDeltaTable.DIMENSIONS, PartitionedDeltaTable.EXPECTED_ROWS}, - {ComplexTypesDeltaTable.DELTA_TABLE_PATH, ComplexTypesDeltaTable.FULL_SCHEMA, ComplexTypesDeltaTable.DIMENSIONS, ComplexTypesDeltaTable.EXPECTED_ROWS} + {ComplexTypesDeltaTable.DELTA_TABLE_PATH, ComplexTypesDeltaTable.FULL_SCHEMA, ComplexTypesDeltaTable.DIMENSIONS, ComplexTypesDeltaTable.EXPECTED_ROWS}, + {SnapshotDeltaTable.DELTA_TABLE_PATH, SnapshotDeltaTable.FULL_SCHEMA, SnapshotDeltaTable.DIMENSIONS, SnapshotDeltaTable.LATEST_SNAPSHOT_EXPECTED_ROWS} }; return Arrays.asList(data); } @@ -124,7 +125,7 @@ public class DeltaInputRowTest @ParameterizedTest(name = "{index}:with context {0}") public void testReadNonExistentTable() { - final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null, null, null); MatcherAssert.assertThat( Assert.assertThrows( diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java index b6b223d9e0a..d8eb083cf57 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceSerdeTest.java @@ -139,4 +139,18 @@ public class DeltaInputSourceSerdeTest exception.getCause().getMessage() ); } + + @Test + public void testDeltaInputSourceWithSnapshotVersion() throws JsonProcessingException + { + final String payload = "{\n" + + " \"type\": \"delta\",\n" + + " \"tablePath\": \"foo/bar\",\n" + + " \"snapshotVersion\": 56\n" + + " }"; + + final DeltaInputSource deltaInputSource = OBJECT_MAPPER.readValue(payload, DeltaInputSource.class); + Assert.assertEquals("foo/bar", deltaInputSource.getTablePath()); + Assert.assertEquals((Long) 56L, deltaInputSource.getSnapshotVersion()); + } } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java index e6bcf9f5fc8..cbbcaefb3ce 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputSourceTest.java @@ -19,6 +19,7 @@ package org.apache.druid.delta.input; +import io.delta.kernel.exceptions.KernelException; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; @@ -68,27 +69,62 @@ public class DeltaInputSourceTest { NonPartitionedDeltaTable.DELTA_TABLE_PATH, NonPartitionedDeltaTable.FULL_SCHEMA, + null, NonPartitionedDeltaTable.EXPECTED_ROWS }, { NonPartitionedDeltaTable.DELTA_TABLE_PATH, NonPartitionedDeltaTable.SCHEMA_1, + null, NonPartitionedDeltaTable.EXPECTED_ROWS }, { NonPartitionedDeltaTable.DELTA_TABLE_PATH, NonPartitionedDeltaTable.SCHEMA_2, + null, NonPartitionedDeltaTable.EXPECTED_ROWS }, { PartitionedDeltaTable.DELTA_TABLE_PATH, PartitionedDeltaTable.FULL_SCHEMA, + null, PartitionedDeltaTable.EXPECTED_ROWS }, { ComplexTypesDeltaTable.DELTA_TABLE_PATH, ComplexTypesDeltaTable.FULL_SCHEMA, + null, ComplexTypesDeltaTable.EXPECTED_ROWS + }, + { + SnapshotDeltaTable.DELTA_TABLE_PATH, + SnapshotDeltaTable.FULL_SCHEMA, + 0L, + SnapshotDeltaTable.V0_SNAPSHOT_EXPECTED_ROWS + }, + { + SnapshotDeltaTable.DELTA_TABLE_PATH, + SnapshotDeltaTable.FULL_SCHEMA, + 1L, + SnapshotDeltaTable.V1_SNAPSHOT_EXPECTED_ROWS + }, + { + SnapshotDeltaTable.DELTA_TABLE_PATH, + SnapshotDeltaTable.FULL_SCHEMA, + 2L, + SnapshotDeltaTable.V2_SNAPSHOT_EXPECTED_ROWS + }, + { + SnapshotDeltaTable.DELTA_TABLE_PATH, + SnapshotDeltaTable.FULL_SCHEMA, + 3L, + SnapshotDeltaTable.LATEST_SNAPSHOT_EXPECTED_ROWS + }, + { + SnapshotDeltaTable.DELTA_TABLE_PATH, + SnapshotDeltaTable.FULL_SCHEMA, + null, + SnapshotDeltaTable.LATEST_SNAPSHOT_EXPECTED_ROWS } }; } @@ -98,12 +134,14 @@ public class DeltaInputSourceTest @Parameterized.Parameter(1) public InputRowSchema schema; @Parameterized.Parameter(2) + public Long snapshotVersion; + @Parameterized.Parameter(3) public List> expectedRows; @Test public void testSampleDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, snapshotVersion); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); List actualSampledRows = sampleAllRows(inputSourceReader); @@ -137,7 +175,7 @@ public class DeltaInputSourceTest @Test public void testReadDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, null, snapshotVersion); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); final List actualReadRows = readAllRows(inputSourceReader); validateRows(expectedRows, actualReadRows, schema); @@ -269,7 +307,7 @@ public class DeltaInputSourceTest @Test public void testSampleDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, filter); + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, filter, null); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); List actualSampledRows = sampleAllRows(inputSourceReader); @@ -311,7 +349,7 @@ public class DeltaInputSourceTest @Test public void testReadDeltaTable() throws IOException { - final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, filter); + final DeltaInputSource deltaInputSource = new DeltaInputSource(deltaTablePath, null, filter, null); final InputSourceReader inputSourceReader = deltaInputSource.reader(schema, null, null); final List actualReadRows = readAllRows(inputSourceReader); validateRows(expectedRows, actualReadRows, schema); @@ -326,7 +364,7 @@ public class DeltaInputSourceTest MatcherAssert.assertThat( Assert.assertThrows( DruidException.class, - () -> new DeltaInputSource(null, null, null) + () -> new DeltaInputSource(null, null, null, null) ), DruidExceptionMatcher.invalidInput().expectMessageIs( "tablePath cannot be null." @@ -337,7 +375,7 @@ public class DeltaInputSourceTest @Test public void testSplitNonExistentTable() { - final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null, null, null); MatcherAssert.assertThat( Assert.assertThrows( @@ -353,7 +391,7 @@ public class DeltaInputSourceTest @Test public void testReadNonExistentTable() { - final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null, null); + final DeltaInputSource deltaInputSource = new DeltaInputSource("non-existent-table", null, null, null); MatcherAssert.assertThat( Assert.assertThrows( @@ -365,6 +403,22 @@ public class DeltaInputSourceTest ) ); } + + @Test + public void testReadNonExistentSnapshot() + { + final DeltaInputSource deltaInputSource = new DeltaInputSource( + SnapshotDeltaTable.DELTA_TABLE_PATH, + null, + null, + 100L + ); + + Assert.assertThrows( + KernelException.class, + () -> deltaInputSource.reader(null, null, null) + ); + } } private static List sampleAllRows(InputSourceReader reader) throws IOException diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java index 7ac3eec09ef..5a0eff0fd16 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java @@ -37,7 +37,9 @@ public class RowSerdeTest { Object[][] data = new Object[][]{ {NonPartitionedDeltaTable.DELTA_TABLE_PATH}, - {PartitionedDeltaTable.DELTA_TABLE_PATH} + {PartitionedDeltaTable.DELTA_TABLE_PATH}, + {ComplexTypesDeltaTable.DELTA_TABLE_PATH}, + {SnapshotDeltaTable.DELTA_TABLE_PATH} }; return Arrays.asList(data); } diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java new file mode 100644 index 00000000000..781fa087a15 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/SnapshotDeltaTable.java @@ -0,0 +1,129 @@ +/* + * 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.delta.input; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.ColumnsFilter; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.segment.AutoTypeColumnSchema; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Refer to extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md to generate the + * sample complex types Delta Lake table used in the unit tests. + * + */ +public class SnapshotDeltaTable +{ + /** + * The Delta table path used by unit tests. + */ + public static final String DELTA_TABLE_PATH = "src/test/resources/snapshot-table"; + + /** + * The list of dimensions in the Delta table {@link #DELTA_TABLE_PATH}. + */ + public static final List DIMENSIONS = ImmutableList.of("id", "map_info"); + + public static final List> V0_SNAPSHOT_EXPECTED_ROWS = new ArrayList<>( + ImmutableList.of( + ImmutableMap.of( + "id", 0L, + "map_info", ImmutableMap.of("snapshotVersion", 0) + ), + ImmutableMap.of( + "id", 1L, + "map_info", ImmutableMap.of("snapshotVersion", 0) + ), + ImmutableMap.of( + "id", 2L, + "map_info", ImmutableMap.of("snapshotVersion", 0) + ) + ) + ); + + public static final List> V1_SNAPSHOT_EXPECTED_ROWS = new ArrayList<>( + ImmutableList.of( + ImmutableMap.of( + "id", 0L, + "map_info", ImmutableMap.of("snapshotVersion", 0) + ), + ImmutableMap.of( + "id", 2L, + "map_info", ImmutableMap.of("snapshotVersion", 0) + ) + ) + ); + + public static final List> V2_SNAPSHOT_EXPECTED_ROWS = new ArrayList<>( + ImmutableList.of( + ImmutableMap.of( + "id", 2L, + "map_info", ImmutableMap.of("snapshotVersion", 2) + ), + ImmutableMap.of( + "id", 0L, + "map_info", ImmutableMap.of("snapshotVersion", 0) + ) + ) + ); + + public static final List> LATEST_SNAPSHOT_EXPECTED_ROWS = new ArrayList<>( + ImmutableList.of( + ImmutableMap.of( + "id", 1L, + "map_info", ImmutableMap.of("snapshotVersion", 3) + ), + ImmutableMap.of( + "id", 4L, + "map_info", ImmutableMap.of("snapshotVersion", 3) + ), + ImmutableMap.of( + "id", 2L, + "map_info", ImmutableMap.of("snapshotVersion", 2) + ), + ImmutableMap.of( + "id", 0L, + "map_info", ImmutableMap.of("snapshotVersion", 0) + ) + ) + ); + + /** + * The Druid schema used for ingestion of {@link #DELTA_TABLE_PATH}. + */ + public static final InputRowSchema FULL_SCHEMA = new InputRowSchema( + new TimestampSpec("na", "posix", DateTimes.of("2024-01-01")), + new DimensionsSpec( + ImmutableList.of( + new AutoTypeColumnSchema("id", null), + new AutoTypeColumnSchema("map_info", null) + ) + ), + ColumnsFilter.all() + ); +} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md index f45b33ab62c..41aec3e3016 100644 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/README.md @@ -44,18 +44,20 @@ Delta table to `resources/employee-delta-table`. You can override the defaults b ```shell python3 create_delta_table.py -h -usage: create_delta_table.py [-h] --save_path SAVE_PATH [--save_mode {append,overwrite}] [--partitioned_by {date,name}] [--num_records NUM_RECORDS] +usage: create_delta_table.py [-h] [--delta_table_type {TableType.SIMPLE,TableType.COMPLEX,TableType.SNAPSHOTS}] --save_path SAVE_PATH [--save_mode {append,overwrite}] [--partitioned_by {date,name,id}] [--num_records NUM_RECORDS] Script to write a Delta Lake table. options: -h, --help show this help message and exit + --delta_table_type {TableType.SIMPLE,TableType.COMPLEX,TableType.SNAPSHOTS} + Choose a Delta table type to generate. (default: TableType.SIMPLE) --save_path SAVE_PATH Save path for Delta table (default: None) --save_mode {append,overwrite} Specify write mode (append/overwrite) (default: append) - --partitioned_by {date,name} - Partitioned by columns (default: None) + --partitioned_by {date,name,id} + Column to partition the Delta table (default: None) --num_records NUM_RECORDS Specify number of Delta records to write (default: 5) ``` @@ -88,10 +90,21 @@ The resulting Delta table is checked in to the repo. The expectated rows to be u ### Complex types table `complex-types-table`: The test data in `resources/complex-types-table` contains 5 Delta records generated with 1 snapshot. -The table was generated by running the following commands: +The table was generated by running the following command: ```shell -python3 create_delta_table.py --save_path=complex-types-table --num_records=5 --gen_complex_types=True +python3 create_delta_table.py --save_path=complex-types-table --delta_table_type=complex ``` The resulting Delta table is checked in to the repo. The expectated rows to be used in tests are updated in `ComplexTypesDeltaTable.java` accordingly. + +### Snapshots table `snapshot-table`: + +The test data in `resources/snapshot-table` contains 4 Delta snapshots with delete, update and removal of records across +snapshots. The table was generated by running the following command: +```shell +python3 create_delta_table.py --save_path=snapshot-table --partitioned_by=id --delta_table_type=snapshots --num_records=3 +``` + +The resulting Delta table is checked in to the repo. The expectated rows to be used in tests are updated in +`SnapshotDeltaTable.java` accordingly. diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py index a116513b01d..ffca055e73c 100755 --- a/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/create_delta_table.py @@ -16,11 +16,20 @@ # limitations under the License. import argparse +from enum import Enum + from delta import * import pyspark from pyspark.sql.types import MapType, StructType, StructField, ShortType, StringType, TimestampType, LongType, IntegerType, DoubleType, FloatType, DateType, BooleanType, ArrayType +from pyspark.sql.functions import expr from datetime import datetime, timedelta import random +from delta.tables import DeltaTable + +class TableType(Enum): + SIMPLE = "simple" + COMPLEX = "complex" + SNAPSHOTS = "snapshots" def config_spark_with_delta_lake(): @@ -40,15 +49,12 @@ def config_spark_with_delta_lake(): def create_dataset_with_complex_types(num_records): """ Create a mock dataset with records containing complex types like arrays, structs and maps. - Parameters: - num_records (int): Number of records to generate. - Returns: - Tuple: A tuple containing a list of records and the corresponding schema. - List of Records: Each record is a tuple representing a row of data. - StructType: The schema defining the structure of the records. - Example: ```python data, schema = create_dataset_with_complex_types(10) @@ -86,6 +92,59 @@ def create_dataset_with_complex_types(num_records): return data, schema +def create_snapshots_table(num_records): + """ + Create a mock dataset for snapshots. + Parameters: + - num_records (int): Number of records to generate. + Returns: + - Tuple: A tuple containing a list of records and the corresponding schema pertaining to a single snapshot. + Example: + ```python + data, schema = create_snapshots_table(5) + ``` + """ + schema = StructType([ + StructField("id", LongType(), False), + StructField("map_info", MapType(StringType(), IntegerType())) + ]) + + data = [] + + for idx in range(num_records): + record = ( + idx, + {"snapshotVersion": 0} + ) + data.append(record) + return data, schema + + +def update_table(spark, schema, delta_table_path): + """ + Update table at the specified delta path with updates: deletion, partial upsert, and insertion. + Each update generates a distinct snapshot for the Delta table. + """ + delta_table = DeltaTable.forPath(spark, delta_table_path) + + # Snapshot 1: remove record with id = 2; result : (id=0, id=2) + delta_table.delete(condition="id=1") + + # Snapshot 2: do a partial update of snapshotInfo map for id = 2 ; result : (id=2, id=0) + delta_table.update( + condition="id=2", + set={"map_info": expr("map('snapshotVersion', 2)")} + ) + + # Snapshot 3: New records to be appended; result : (id=1, id=4, id=2, id=0) + append_data = [ + (1, {"snapshotVersion": 3}), + (4, {"snapshotVersion": 3}) + ] + append_df = spark.createDataFrame(append_data, schema) + append_df.write.format("delta").mode("append").save(delta_table_path) + + def create_dataset(num_records): """ Generate a mock employee dataset with different datatypes for testing purposes. @@ -141,19 +200,18 @@ def main(): parser = argparse.ArgumentParser(description="Script to write a Delta Lake table.", formatter_class=argparse.ArgumentDefaultsHelpFormatter) - parser.add_argument("--gen_complex_types", type=bool, default=False, help="Generate a Delta table with records" - " containing complex types like structs," - " maps and arrays.") + parser.add_argument('--delta_table_type', type=lambda t: TableType[t.upper()], choices=TableType, + default=TableType.SIMPLE, help='Choose a Delta table type to generate.') parser.add_argument('--save_path', default=None, required=True, help="Save path for Delta table") parser.add_argument('--save_mode', choices=('append', 'overwrite'), default="append", help="Specify write mode (append/overwrite)") - parser.add_argument('--partitioned_by', choices=("date", "name"), default=None, + parser.add_argument('--partitioned_by', choices=("date", "name", "id"), default=None, help="Column to partition the Delta table") parser.add_argument('--num_records', type=int, default=5, help="Specify number of Delta records to write") args = parser.parse_args() - is_gen_complex_types = args.gen_complex_types + delta_table_type = args.delta_table_type save_mode = args.save_mode save_path = args.save_path num_records = args.num_records @@ -161,21 +219,29 @@ def main(): spark = config_spark_with_delta_lake() - if is_gen_complex_types: - data, schema = create_dataset_with_complex_types(num_records=num_records) - else: + if delta_table_type == TableType.SIMPLE: data, schema = create_dataset(num_records=num_records) + elif delta_table_type == TableType.COMPLEX: + data, schema = create_dataset_with_complex_types(num_records=num_records) + elif delta_table_type == TableType.SNAPSHOTS: + data, schema = create_snapshots_table(num_records) + else: + args.print_help() + raise Exception("Unknown value specified for --delta_table_type") df = spark.createDataFrame(data, schema=schema) if not partitioned_by: df.write.format("delta").mode(save_mode).save(save_path) else: - df.write.format("delta").partitionBy("name").mode(save_mode).save(save_path) + df.write.format("delta").partitionBy(partitioned_by).mode(save_mode).save(save_path) df.show() print(f"Generated Delta table records partitioned by {partitioned_by} in {save_path} in {save_mode} mode" - f" with {num_records} records.") + f" with {num_records} records with {delta_table_type}.") + + if delta_table_type == TableType.SNAPSHOTS: + update_table(spark, schema, save_path) if __name__ == "__main__": diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000000.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000000.json.crc new file mode 100644 index 00000000000..8d01e0c37e7 Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000000.json.crc differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000001.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000001.json.crc new file mode 100644 index 00000000000..1215445f9f2 Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000001.json.crc differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000002.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000002.json.crc new file mode 100644 index 00000000000..012c2ad2f3c Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000002.json.crc differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000003.json.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000003.json.crc new file mode 100644 index 00000000000..03c7cb26637 Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/.00000000000000000003.json.crc differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000000.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..bd652b26a77 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000000.json @@ -0,0 +1,6 @@ +{"commitInfo":{"timestamp":1725465348581,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[\"id\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"3","numOutputRows":"3","numOutputBytes":"2607"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0","txnId":"d52bcd81-2310-417a-acb2-e206a4882383"}} +{"metaData":{"id":"5a4682fa-c3d8-4f49-8825-b8540e20ce93","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"map_info\",\"type\":{\"type\":\"map\",\"keyType\":\"string\",\"valueType\":\"integer\",\"valueContainsNull\":true},\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["id"],"configuration":{},"createdTime":1725465346226}} +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"add":{"path":"id=0/part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet","partitionValues":{"id":"0"},"size":869,"modificationTime":1725465348507,"dataChange":true,"stats":"{\"numRecords\":1,\"nullCount\":{\"map_info\":0}}"}} +{"add":{"path":"id=1/part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet","partitionValues":{"id":"1"},"size":869,"modificationTime":1725465348507,"dataChange":true,"stats":"{\"numRecords\":1,\"nullCount\":{\"map_info\":0}}"}} +{"add":{"path":"id=2/part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet","partitionValues":{"id":"2"},"size":869,"modificationTime":1725465348506,"dataChange":true,"stats":"{\"numRecords\":1,\"nullCount\":{\"map_info\":0}}"}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000001.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000001.json new file mode 100644 index 00000000000..bdde476c742 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000001.json @@ -0,0 +1,2 @@ +{"commitInfo":{"timestamp":1725465352088,"operation":"DELETE","operationParameters":{"predicate":"[\"(id#852L = 1)\"]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"1","numRemovedBytes":"869","numCopiedRows":"0","numDeletionVectorsAdded":"0","numDeletionVectorsRemoved":"0","numAddedChangeFiles":"0","executionTimeMs":"426","numDeletionVectorsUpdated":"0","numDeletedRows":"1","scanTimeMs":"421","numAddedFiles":"0","numAddedBytes":"0","rewriteTimeMs":"0"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0","txnId":"5af91bc8-feb5-40e2-b7d0-76acd1038ba7"}} +{"remove":{"path":"id=1/part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet","deletionTimestamp":1725465351650,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"id":"1"},"size":869,"stats":"{\"numRecords\":1}"}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000002.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000002.json new file mode 100644 index 00000000000..9e2ee0442b2 --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000002.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1725465353405,"operation":"UPDATE","operationParameters":{"predicate":"[\"(id#852L = 2)\"]"},"readVersion":1,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numRemovedFiles":"1","numRemovedBytes":"869","numCopiedRows":"0","numDeletionVectorsAdded":"0","numDeletionVectorsRemoved":"0","numAddedChangeFiles":"0","executionTimeMs":"448","numDeletionVectorsUpdated":"0","scanTimeMs":"215","numAddedFiles":"1","numUpdatedRows":"1","numAddedBytes":"869","rewriteTimeMs":"232"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0","txnId":"24e06ceb-96fe-44b4-bcf8-9c06604ec5f9"}} +{"add":{"path":"id=2/part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet","partitionValues":{"id":"2"},"size":869,"modificationTime":1725465353400,"dataChange":true,"stats":"{\"numRecords\":1,\"nullCount\":{\"map_info\":0}}"}} +{"remove":{"path":"id=2/part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet","deletionTimestamp":1725465353403,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"id":"2"},"size":869}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000003.json b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000003.json new file mode 100644 index 00000000000..077f7ca11ea --- /dev/null +++ b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/_delta_log/00000000000000000003.json @@ -0,0 +1,3 @@ +{"commitInfo":{"timestamp":1725465354608,"operation":"WRITE","operationParameters":{"mode":"Append","partitionBy":"[]"},"readVersion":2,"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"2","numOutputRows":"2","numOutputBytes":"1738"},"engineInfo":"Apache-Spark/3.5.0 Delta-Lake/3.1.0","txnId":"ab44598c-07be-41df-bb30-e5c2f050cfe4"}} +{"add":{"path":"id=1/part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet","partitionValues":{"id":"1"},"size":869,"modificationTime":1725465354584,"dataChange":true,"stats":"{\"numRecords\":1,\"nullCount\":{\"map_info\":0}}"}} +{"add":{"path":"id=4/part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet","partitionValues":{"id":"4"},"size":869,"modificationTime":1725465354593,"dataChange":true,"stats":"{\"numRecords\":1,\"nullCount\":{\"map_info\":0}}"}} diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/.part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/.part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet.crc new file mode 100644 index 00000000000..c1b3cedd811 Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/.part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet.crc differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet new file mode 100644 index 00000000000..8a0655de859 Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=0/part-00003-8610110f-f5a0-4856-a5a8-516e5b35ef44.c000.snappy.parquet differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet.crc new file mode 100644 index 00000000000..b19268a21f0 Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet.crc differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet.crc new file mode 100644 index 00000000000..c1b3cedd811 Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/.part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet.crc differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet new file mode 100644 index 00000000000..3ae3011478e Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00004-a614b691-2e73-4603-92a8-92ec51e4bdb9.c000.snappy.parquet differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet new file mode 100644 index 00000000000..8a0655de859 Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=1/part-00006-120df0a3-1c7a-4a2e-81aa-7bc8140b0f09.c000.snappy.parquet differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet.crc new file mode 100644 index 00000000000..dce55f335fc Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet.crc differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet.crc new file mode 100644 index 00000000000..c1b3cedd811 Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/.part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet.crc differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet new file mode 100644 index 00000000000..03888f094fe Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00000-bffba8b9-3388-4c46-b2ca-db8d7288d345.c000.snappy.parquet differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet new file mode 100644 index 00000000000..8a0655de859 Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=2/part-00009-246861b8-01b0-446c-b4f1-ab0c2e762044.c000.snappy.parquet differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/.part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet.crc b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/.part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet.crc new file mode 100644 index 00000000000..b19268a21f0 Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/.part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet.crc differ diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet new file mode 100644 index 00000000000..3ae3011478e Binary files /dev/null and b/extensions-contrib/druid-deltalake-extensions/src/test/resources/snapshot-table/id=4/part-00009-d3e45f07-7843-4b3d-8d6c-dd90dd3db251.c000.snappy.parquet differ diff --git a/extensions-contrib/druid-iceberg-extensions/pom.xml b/extensions-contrib/druid-iceberg-extensions/pom.xml index e3c996461e7..e8644b0aa9b 100644 --- a/extensions-contrib/druid-iceberg-extensions/pom.xml +++ b/extensions-contrib/druid-iceberg-extensions/pom.xml @@ -29,7 +29,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/gce-extensions/pom.xml b/extensions-contrib/gce-extensions/pom.xml index e71965e5dd6..ec805fe742d 100644 --- a/extensions-contrib/gce-extensions/pom.xml +++ b/extensions-contrib/gce-extensions/pom.xml @@ -21,7 +21,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/graphite-emitter/pom.xml b/extensions-contrib/graphite-emitter/pom.xml index 534e4947b33..b915e22bb20 100644 --- a/extensions-contrib/graphite-emitter/pom.xml +++ b/extensions-contrib/graphite-emitter/pom.xml @@ -24,7 +24,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/grpc-query/README.md b/extensions-contrib/grpc-query/README.md new file mode 100644 index 00000000000..22bdb976147 --- /dev/null +++ b/extensions-contrib/grpc-query/README.md @@ -0,0 +1,312 @@ + + +# gRPC Query Extension for Druid + +This extension provides a gRPC API for SQL and Native queries. + +Druid uses REST as its RPC protocol. Druid has a large variety of REST operations +including query, ingest jobs, monitoring, configuration and many more. Although +REST is a universally supported RPC format, it is not the only one in use. This +extension allows gRPC-based clients to issue SQL queries. + +Druid is optimized for high-concurrency, low-complexity queries that return a +small result set (a few thousand rows at most). The small-query focus allows +Druid to offer a simple, stateless request/response REST API. This gRPC API +follows that Druid pattern: it is optimized for simple queries and follows +Druid's request/response model. APIs such as JDBC can handle larger results +because they are stateful: a client can request pages of results using multiple +API calls. This API does not support paging: the entire result set is returned +in the response, resulting in an API which is fast for small queries, and not +suitable for larger result sets. + +## Use Cases + +The gRPC query extension can be used in two ways, depending on the selected +result format. + +### CSV or JSON Response Format + +The simplest way to use the gRPC extension is to send a query request that +uses CSV or JSON as the return format. The client simply pulls the results +from the response and does something useful with them. For the CSV format, +headers can be created from the column metadata in the response message. + +### Protobuf Response Format + +Some applications want to use Protobuf as the result format. In this case, +the extension encodes Protobuf-encoded rows as the binary payload of the query +response. This works for an application which uses a fixed set of queries, each +of which is carefully designed to power one application, say a dashboard. The +(simplified) message flow is: + +```text ++-----------+ query -> +-------+ +| Dashboard | -- gRPC --> | Druid | ++-----------+ <- data +-------+ +``` + +In practice, there may be multiple proxy layers: one on the application side, and +the Router on the Druid side. + +The dashboard displays a fixed set of reports and charts. Each of those sends a +well-defined query specified as part of the application. The returned data is thus +both well-known and fixed for each query. The set of queries is fixed by the contents +of the dashboard. That is, this is not an ad-hoc query use case. + +Because the queries are locked down, and are part of the application, the set of valid +result sets is also well known and locked down. Given this well-controlled use case, it +is possible to use a pre-defined Protobuf message to represent the results of each distinct +query. (Protobuf is a compiled format: the solution works only because the set of messages +are well known. It would not work for the ad-hoc case in which each query has a different +result set schema.) + +To be very clear: the application has a fixed set of queries to be sent to Druid via gRPC. +For each query, there is a fixed Protobuf response format defined by the application. +No other queries, aside from this well-known set, will be sent to the gRPC endpoint using +the Protobuf response format. If the set of queries is not well-defined, use the CSV +or JSON response format instead. + +## Installation + +The gRPC query extension is a "contrib" extension and is not installed by default when +you install Druid. Instead, you must install it manually. + +In development, you can build Druid with all the "contrib" extensions. When building +Druid, include the `-P bundle-contrib-exts` in addition to the `-P dist` option: + +```bash +mvn package -Pdist,bundle-contrib-exts ... +``` + +In production, follow the [Druid documentation](https://druid.apache.org/docs/latest/development/extensions.html). + +To enable the extension, add the following to the load list in +`_commmon/common.runtime.properties`: + +```text +druid.extensions.loadList=[..., "grpc-query"] +``` + +Adding the extension to the load list automatically enables the extension, +but only in the Broker. + +If you use the Protobuf response format, bundle up your Protobuf classes +into a jar file, and place that jar file in the +`$DRUID_HOME/extensions/grpc-query` directory. The Protobuf classes will +appear on the class path and will be available from the `grpc-query` +extension. + +### Configuration + +Enable and configure the extension in `broker/runtime.properties`: + +```text +druid.grpcQuery.port=50051 +``` + +The default port is 50051 (preliminary). + +If you use the Protobuf response format, bundle up your Protobuf classes +into a jar file, and place that jar file in the +`$DRUID_HOME/extensions/grpc-query` directory. The Protobuf classes will +appear on the class path and will be available from the `grpc-query` +extension. + +## Usage + +See the `src/main/proto/query.proto` file in the `grpc-query` project for the request and +response message formats. The request message format closely follows the REST JSON message +format. The response is optimized for gRPC: it contains an error (if the request fails), +or the result schema and result data as a binary payload. You can query the gRPC endpoint +with any gRPC client. + +Although both Druid SQL and Druid itself support a `float` data type, that type is not +usable in a Protobuf response object. Internally Druid converts all `float` values to +`double`. As a result, the Protobuf reponse object supports only the `double` type. +An attempt to use `float` will lead to a runtime error when processing the query. +Use the `double` type instead. + +Sample request, + +``` +QueryRequest.newBuilder() + .setQuery("SELECT * FROM foo") + .setResultFormat(QueryResultFormat.CSV) + .setQueryType(QueryOuterClass.QueryType.SQL) + .build(); +``` + +When using Protobuf response format, bundle up your Protobuf classes +into a jar file, and place that jar file in the +`$DRUID_HOME/extensions/grpc-query` directory. +Specify the response Protobuf message name in the request. + +``` +QueryRequest.newBuilder() + .setQuery("SELECT dim1, dim2, dim3, cnt, m1, m2, unique_dim1, __time AS "date" FROM foo") + .setQueryType(QueryOuterClass.QueryType.SQL) + .setProtobufMessageName(QueryResult.class.getName()) + .setResultFormat(QueryResultFormat.PROTOBUF_INLINE) + .build(); + +Response message + +message QueryResult { + string dim1 = 1; + string dim2 = 2; + string dim3 = 3; + int64 cnt = 4; + float m1 = 5; + double m2 = 6; + bytes unique_dim1 = 7; + google.protobuf.Timestamp date = 8; +} +``` + +## Security + +The extension supports both "anonymous" and basic authorization. Anonymous is the mode +for an out-of-the-box Druid: no authorization needed. The extension does not yet support +other security extensions: each needs its own specific integration. + +Clients that use basic authentication must include a set of credentials. See +`BasicCredentials` for a typical implementation and `BasicAuthTest` for how to +configure the credentials in the client. + +## Implementation Notes + +This project contains several components: + +* Guice module and associated server initialization code. +* Netty-based gRPC server. +* A "driver" that performs the actual query and generates the results. + +## Debugging + +Debugging of the gRPC extension requires extra care due to the nuances of loading +classes from an extension. + +### Running in a Server + +Druid extensions are designed to run in the Druid server. The gRPC extension is +loaded only in the Druid broker using the contiguration described above. If something +fails during startup, the Broker will crash. Consult the Broker logs to determine +what went wrong. Startup failures are typically due to required jars not being installed +as part of the extension. Check the `pom.xml` file to track down what's missing. + +Failures can also occur when running a query. Such failures will result in a failure +response and should result in a log entry in the Broker log file. Use the log entry +to sort out what went wrong. + +You can also attach a debugger to the running process. You'll have to enable the debugger +in the server by adding the required parameters to the Broker's `jvm.config` file. + +### Debugging using Unit Tests + +To debug the functionality of the extension, your best bet is to debug in the context +of a unit test. Druid provides a special test-only SQL stack with a few pre-defined +datasources. See the various `CalciteQueryTest` classes to see what these are. You can +also query Druid's various system tables. See `GrpcQueryTest` for a simple "starter" +unit test that configures the server and uses an in-process client to send requests. + +Most unit testing can be done without the gRPC server, by calling the `QueryDriver` +class directly. That is, if the goal is work with the code that takes a request, runs +a query, and produces a response, then the driver is the key and the server is just a +bit of extra copmlexity. See the `DriverTest` class for an example unit test. + +### Debugging in a Server in an IDE + +We would like to be able to debug the gRPC extension, within the Broker, in an IDE. +As it turns out, doing so breaks Druid's class loader mechanisms in ways that are both +hard to understand and hard to work around. When run in a server, Java creates an instance +of `GrpcQueryModule` using the extension's class loader. Java then uses that same class +loader to load other classes in the extension, including those here and those in the +shaded gRPC jar file. + +However, when run in an IDE, if this project is on the class path, then the `GrpcQueryModule` +class will be loaded from the "App" class loader. This works fine: it causes the other +classes of this module to also be loaded from the class path. However, once execution +calls into gRPC, Java will use the App class loader, not the extension class loader, and +will fail to find some of the classes, resulting in Java exceptions. Worse, in some cases, +Java may load the same class from both class loaders. To Java, these are not the same +classes, and you will get mysterious errors as a result. + +For now, the lesson is: don't try to debug the extension in the Broker in the IDE. Use +one of the above options instead. + +For reference (and in case we figure out a solution to the class loader conflict), +the way to debug the Broker in an IDE is the following: + +* Build your branch. Use the `-P bundle-contrib-exts` flag in place of `-P dist`, as described + above. +* Create an install from the distribution produced above. +* Use the `single-server/micro-quickstart` config for debugging. +* Configure the installation using the steps above. +* Modify the Supervisor config for your config to comment out the line that launches + the broker. Use the hash (`#`) character to comment out the line. +* In your IDE, define a launch configuration for the Broker. + * The launch command is `server broker` + * Add the following JVM arguments: + +```text +--add-exports java.base/jdk.internal.perf=ALL-UNNAMED +--add-exports jdk.management/com.sun.management.internal=ALL-UNNAMED +``` + + * Define `grpc-query` as a project dependency. (This is for Eclipse; IntelliJ may differ.) + * Configure the class path to include the common and Broker properties files. +* Launch the micro-quickstart cluster. +* Launch the Broker in your IDE. + +### gRPC Logging + +Debugging of the gRPC stack is difficult since the shaded jar loses source attachments. + +Logging helps. gRPC logging is not enabled via Druid's logging system. Intead, [create +the following `logging.properties` file](https://stackoverflow.com/questions/50243717/grpc-logger-level): + +```text +handlers=java.util.logging.ConsoleHandler +io.grpc.level=FINE +java.util.logging.ConsoleHandler.level=FINE +java.util.logging.ConsoleHandler.formatter=java.util.logging.SimpleFormatter +``` + +Then, pass the following on the command line: + +```text +-Djava.util.logging.config.file=logging.properties +``` + +Adjust the path to the file depending on where you put the file. + +## Acknowledgements + +This is not the first project to have created a gRPC API for Druid. Others include: + +* [[Proposal] define a RPC protocol for querying data, support apache Arrow as data + exchange interface](https://github.com/apache/druid/issues/3891) +* [gRPC Druid extension PoC](https://github.com/ndolgov/gruid) +* [Druid gRPC-json server extension](https://github.com/apache/druid/pull/6798) + +Full credit goes to those who have gone this way before. + +Note that the class loader solution used by the two code bases above turned out +to not be needed. See the notes above about the class loader issues. diff --git a/extensions-contrib/grpc-query/pom.xml b/extensions-contrib/grpc-query/pom.xml new file mode 100644 index 00000000000..101e2f34b74 --- /dev/null +++ b/extensions-contrib/grpc-query/pom.xml @@ -0,0 +1,375 @@ + + + + + + 4.0.0 + org.apache.druid.extensions.contrib + grpc-query + grpc-query + grpc-query + + + org.apache.druid + druid + 32.0.0-SNAPSHOT + ../../pom.xml + + + + + + io.grpc + grpc-bom + 1.59.0 + pom + import + + + + + + + org.apache.druid + druid-server + ${project.parent.version} + provided + + + org.apache.druid + druid-processing + ${project.parent.version} + provided + + + org.apache.druid + druid-sql + ${project.parent.version} + provided + + + com.fasterxml.jackson.module + jackson-module-guice + provided + + + com.google.inject + guice + provided + + + com.fasterxml.jackson.core + jackson-databind + provided + + + com.fasterxml.jackson.core + jackson-core + provided + + + com.google.inject.extensions + guice-multibindings + provided + + + com.google.guava + guava + ${guava.version} + provided + + + com.google.code.findbugs + jsr305 + provided + + + + io.netty + netty-buffer + provided + + + io.netty + netty-codec-http + provided + + + io.netty + netty-common + provided + + + io.netty + netty-handler + provided + + + io.netty + netty-resolver + provided + + + io.netty + netty-transport + provided + + + io.netty + netty-codec-http2 + + + javax.ws.rs + jsr311-api + provided + + + io.grpc + grpc-api + + + io.grpc + grpc-protobuf + + + io.grpc + grpc-stub + + + io.grpc + grpc-netty + + + io.grpc + grpc-core + + + com.google.protobuf + protobuf-java + ${protobuf.version} + + + jakarta.validation + jakarta.validation-api + provided + + + org.apache.calcite.avatica + avatica-core + provided + + + jakarta.inject + jakarta.inject-api + provided + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + joda-time + joda-time + provided + + + org.apache.calcite + calcite-core + provided + + + javax.inject + javax.inject + 1 + provided + + + + + org.apache.druid + druid-sql + ${project.parent.version} + test-jar + test + + + junit + junit + test + + + org.junit.jupiter + junit-jupiter-api + test + + + org.apache.druid + druid-server + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-processing + ${project.parent.version} + test-jar + test + + + org.easymock + easymock + test + + + org.apache.druid.extensions + druid-basic-security + ${project.parent.version} + test + + + org.reflections + reflections + test + + + + + + + kr.motd.maven + os-maven-plugin + 1.5.0.Final + + + + + + org.xolstice.maven.plugins + protobuf-maven-plugin + 0.6.1 + + com.google.protobuf:protoc:3.21.7:exe:${os.detected.classifier} + grpc-java + io.grpc:protoc-gen-grpc-java:1.52.0:exe:${os.detected.classifier} + + + + + compile + compile-custom + test-compile + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-test-source + generate-sources + + add-source + + + + target/generated-test-sources/protobuf/java + target/generated-sources/protobuf/grpc-java + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + package + + test-jar + + + tests + + + + + proto-jar + package + + + test-jar + + + test-proto + + **/proto/* + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + io.netty:netty-codec-http2 + io.grpc:grpc-core:jar + io.grpc:grpc-netty:jar + + + + + + diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/client/GrpcResponseHandler.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/client/GrpcResponseHandler.java new file mode 100644 index 00000000000..37ac2cf2e85 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/client/GrpcResponseHandler.java @@ -0,0 +1,96 @@ +/* + * 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.grpc.client; + +import com.google.protobuf.AbstractMessageLite; +import com.google.protobuf.ByteString; +import com.google.protobuf.MessageLite; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; + +public class GrpcResponseHandler +{ + private final T message; + + private GrpcResponseHandler(final Class clazz) + { + this.message = get(clazz); + } + + public static GrpcResponseHandler of(Class clazz) + { + return new GrpcResponseHandler<>(clazz); + } + + public List get(ByteString byteString) + { + return get(new ByteArrayInputStream(byteString.toByteArray())); + } + + @SuppressWarnings("unchecked") + public List get(InputStream inputStream) + { + try { + final List data = new ArrayList<>(); + while (true) { + try { + final MessageLite messageLite = + message + .getDefaultInstanceForType() + .getParserForType() + .parseDelimitedFrom(inputStream); + if (messageLite == null) { + break; + } + data.add((T) messageLite); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + return data; + } + finally { + try { + inputStream.close(); + } + catch (IOException e) { + // ignore + } + } + } + + @SuppressWarnings("unchecked") + private T get(Class clazz) + { + try { + final Method method = clazz.getMethod("getDefaultInstance", new Class[0]); + return (T) method.invoke(null); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.java new file mode 100644 index 00000000000..6621a92ed95 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/guice/GrpcQueryModule.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.grpc.guice; + +import com.google.inject.Binder; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.grpc.server.GrpcEndpointInitializer; +import org.apache.druid.grpc.server.GrpcQueryConfig; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.annotations.LoadScope; +import org.apache.druid.initialization.DruidModule; + +@LoadScope(roles = NodeRole.BROKER_JSON_NAME) +public class GrpcQueryModule implements DruidModule +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, GrpcQueryConfig.CONFIG_BASE, GrpcQueryConfig.class); + LifecycleModule.register(binder, GrpcEndpointInitializer.class); + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/AnonymousAuthServerInterceptor.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/AnonymousAuthServerInterceptor.java new file mode 100644 index 00000000000..3059c603d47 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/AnonymousAuthServerInterceptor.java @@ -0,0 +1,68 @@ +/* + * 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.grpc.server; + +import com.google.common.collect.ImmutableMap; +import io.grpc.Context; +import io.grpc.Contexts; +import io.grpc.Metadata; +import io.grpc.ServerCall; +import io.grpc.ServerCall.Listener; +import io.grpc.ServerCallHandler; +import io.grpc.ServerInterceptor; +import org.apache.druid.server.security.Authenticator; + +import javax.inject.Inject; + +/** + * "Authorizes" an anonymous request, which just means adding an "allow all" + * authorization result in the context. Use this form for either of Druid's + * "allow all" authorizers. + * + * @see {@link BasicAuthServerInterceptor} for details + */ +public class AnonymousAuthServerInterceptor implements ServerInterceptor +{ + private final Authenticator authenticator; + + @Inject + public AnonymousAuthServerInterceptor(Authenticator authenticator) + { + this.authenticator = authenticator; + } + + @Override + public Listener interceptCall( + ServerCall call, + Metadata headers, + ServerCallHandler next + ) + { + return Contexts.interceptCall( + Context.current().withValue( + QueryServer.AUTH_KEY, + authenticator.authenticateJDBCContext(ImmutableMap.of()) + ), + call, + headers, + next + ); + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/BasicAuthServerInterceptor.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/BasicAuthServerInterceptor.java new file mode 100644 index 00000000000..15a4926e209 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/BasicAuthServerInterceptor.java @@ -0,0 +1,149 @@ +/* + * 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.grpc.server; + +import com.google.common.collect.ImmutableMap; +import io.grpc.Context; +import io.grpc.Contexts; +import io.grpc.Metadata; +import io.grpc.ServerCall; +import io.grpc.ServerCall.Listener; +import io.grpc.ServerCallHandler; +import io.grpc.ServerInterceptor; +import io.grpc.Status; +import io.grpc.StatusRuntimeException; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.Authenticator; + +import javax.inject.Inject; + +/** + * Authorizes a Basic Auth user name and password and sets the resulting + * {@link AuthenticationResult} on the call context. + *

    + * Implements the gRPC {@link ServerInterceptor} to wrap the actual RPC + * call with a step which pulls the "Authorization" header from the request, + * decodes the user name and password, looks up the user using the + * BasicHTTPAuthenticator#authenticateJDBCContext(java.util.Map) + * method, and attaches the resulting {@link AuthenticationResult} to the call + * {@link Context}. The gRPC service will later retrieve the auth result to pass + * into the Driver for use in validating query resources. + *

    + * Note that gRPC documentation in this area is sparse. Examples are hard to + * find. gRPC provides exactly one (obscure) way to do things, as represented + * here. + *

    + * Auth failures can occur in many ways: missing or badly formed header, invalid + * user name or password, etc. In each case, the code throws a + * {@link StatusRuntimeException} with {@link Status#PERMISSION_DENIED}. No hint + * of the problem is provided to the user. + *

    + * This pattern can be replicated for other supported Druid authorizers. + */ +public class BasicAuthServerInterceptor implements ServerInterceptor +{ + public static final String AUTHORIZATION_HEADER = "Authorization"; + private static final String BASIC_PREFIX = "Basic "; + private static final Metadata.Key AUTHORIZATION_KEY = + Metadata.Key.of(AUTHORIZATION_HEADER, Metadata.ASCII_STRING_MARSHALLER); + private static final Logger LOG = new Logger(BasicAuthServerInterceptor.class); + + // Want BasicHTTPAuthenticator, but it is not visible here. + private final Authenticator authenticator; + + @Inject + public BasicAuthServerInterceptor(Authenticator authenticator) + { + this.authenticator = authenticator; + } + + @Override + public Listener interceptCall( + ServerCall call, + Metadata headers, + ServerCallHandler next + ) + { + // Use a gRPC method to wrap the actual call in a new context + // that includes the auth result. + return Contexts.interceptCall( + Context.current().withValue( + QueryServer.AUTH_KEY, + authenticate(headers.get(AUTHORIZATION_KEY)) + ), + call, + headers, + next + ); + } + + // See BasicHTTPAuthenticator.Filter + public AuthenticationResult authenticate(String encodedUserSecret) + { + if (encodedUserSecret == null) { + throw new StatusRuntimeException(Status.PERMISSION_DENIED); + } + + if (!encodedUserSecret.startsWith(BASIC_PREFIX)) { + throw new StatusRuntimeException(Status.PERMISSION_DENIED); + } + encodedUserSecret = encodedUserSecret.substring(BASIC_PREFIX.length()); + + // At this point, encodedUserSecret is not null, indicating that the request intends to perform + // Basic HTTP authentication. + // Copy of BasicAuthUtils.decodeUserSecret() which is not visible here. + String decodedUserSecret; + try { + decodedUserSecret = StringUtils.fromUtf8(StringUtils.decodeBase64String(encodedUserSecret)); + } + catch (IllegalArgumentException iae) { + LOG.info("Malformed user secret."); + throw new StatusRuntimeException(Status.PERMISSION_DENIED); + } + + String[] splits = decodedUserSecret.split(":"); + if (splits.length != 2) { + // The decoded user secret is not of the right format + throw new StatusRuntimeException(Status.PERMISSION_DENIED); + } + + final String user = splits[0]; + final String password = splits[1]; + + // Fail fast for any authentication error. If the authentication result is null we also fail + // as this indicates a non-existent user. + try { + AuthenticationResult authenticationResult = authenticator.authenticateJDBCContext( + ImmutableMap.of("user", user, "password", password) + ); + if (authenticationResult == null) { + throw new StatusRuntimeException(Status.PERMISSION_DENIED); + } + return authenticationResult; + } + // Want BasicSecurityAuthenticationException, but it is not visible here. + catch (IllegalArgumentException ex) { + LOG.info("Exception authenticating user [%s] - [%s]", user, ex.getMessage()); + throw new StatusRuntimeException(Status.PERMISSION_DENIED); + } + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java new file mode 100644 index 00000000000..1cb3884884c --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcEndpointInitializer.java @@ -0,0 +1,116 @@ +/* + * 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.grpc.server; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.guice.ManageLifecycleServer; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.guice.annotations.NativeQuery; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.security.AuthenticatorMapper; +import org.apache.druid.sql.SqlStatementFactory; + +import javax.inject.Inject; + +import java.io.IOException; + +/** + * Initializes the gRPC endpoint (server). This version uses a Netty-based server + * separate from Druid's primary Jetty-based server. We may want to consider a + * + * recent addition to the gRPC examples to run gRPC as a servlet. However, trying + * that turned out to incur many issues, including the fact that there was no way + * to pass the AuthenticationResult down through the many layers of gRPC into the + * query code. So, we use the gRPC server instead. + *

    + * An instance of this class is created by Guice and managed via Druid's + * lifecycle manager. + */ +@ManageLifecycleServer +public class GrpcEndpointInitializer +{ + private static final Logger log = new Logger(GrpcEndpointInitializer.class); + + private final GrpcQueryConfig config; + private final QueryDriver driver; + private final AuthenticatorMapper authMapper; + + private QueryServer server; + + @Inject + public GrpcEndpointInitializer( + GrpcQueryConfig config, + final @Json ObjectMapper jsonMapper, + final @NativeQuery SqlStatementFactory sqlStatementFactory, + final QueryLifecycleFactory queryLifecycleFactory, + final AuthenticatorMapper authMapper + ) + { + this.config = config; + this.authMapper = authMapper; + this.driver = new QueryDriver(jsonMapper, sqlStatementFactory, queryLifecycleFactory); + } + + @LifecycleStart + public void start() + { + server = new QueryServer(config, driver, authMapper); + try { + server.start(); + } + catch (IOException e) { + // Indicates an error when gRPC tried to start the server + // (such the port is already in use.) + log.error(e, "Fatal error: gRPC query server startup failed"); + + // This exception will bring down the Broker as there is not much we can + // do if we can't start the gRPC endpoint. + throw new ISE(e, "Fatal error: grpc query server startup failed"); + } + catch (Throwable t) { + // Catch-all for other errors. The most likely error is that some class was not found + // (that is, class loader issues in an IDE, or a jar missing in the extension). + log.error(t, "Fatal error: gRPC query server startup failed"); + + // This exception will bring down the Broker as there is not much we can + // do if we can't start the gRPC endpoint. + throw t; + } + } + + @LifecycleStop + public void stop() + { + if (server != null) { + try { + server.blockUntilShutdown(); + } + catch (InterruptedException e) { + // Just warn. We're shutting down anyway, so no need to throw an exception. + log.warn(e, "gRPC query server shutdown failed"); + } + server = null; + } + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcQueryConfig.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcQueryConfig.java new file mode 100644 index 00000000000..a9cdde23970 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/GrpcQueryConfig.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.grpc.server; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.validation.constraints.Max; + +/** + * Grpc configs for the extension. + */ +public class GrpcQueryConfig +{ + public static final String CONFIG_BASE = "druid.grpcQuery"; + + @JsonProperty + @Max(0xffff) + private int port = 50051; + + public GrpcQueryConfig() + { + } + + public GrpcQueryConfig(int port) + { + this.port = port; + } + + /** + * @return the port to accept gRPC client connections on + */ + public int getPort() + { + return port; + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/HealthService.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/HealthService.java new file mode 100644 index 00000000000..d40b0b68bd2 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/HealthService.java @@ -0,0 +1,170 @@ +/* + * 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.grpc.server; + +import com.google.common.util.concurrent.MoreExecutors; +import io.grpc.Context; +import io.grpc.Status; +import io.grpc.stub.StreamObserver; +import org.apache.druid.grpc.proto.HealthGrpc; +import org.apache.druid.grpc.proto.HealthOuterClass.HealthCheckRequest; +import org.apache.druid.grpc.proto.HealthOuterClass.HealthCheckResponse; +import org.apache.druid.grpc.proto.HealthOuterClass.HealthCheckResponse.ServingStatus; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; + +/** + * Implementation of grpc health service. Provides {@code check(HealthCheckRequest, StreamObserver(HealthCheckResponse))} + * method to get health of a specific service or the overall server health. + *

    + * A client can call the {@code watch(HealthCheckRequest, StreamObserver(HealthCheckResponse))} method + * to perform a streaming health-check. + * The server will immediately send back a message indicating the current serving status. + * It will then subsequently send a new message whenever the service's serving status changes. + */ +class HealthService extends HealthGrpc.HealthImplBase +{ + private final ConcurrentMap serviceStatusMap; + private final ConcurrentMap cancellationContexts; + private final ConcurrentMap statusChangeLatchMap; + + public HealthService() + { + this.serviceStatusMap = new ConcurrentHashMap<>(); + this.cancellationContexts = new ConcurrentHashMap<>(); + this.statusChangeLatchMap = new ConcurrentHashMap<>(); + } + + @Override + public void check( + HealthCheckRequest request, + StreamObserver responseObserver + ) + { + String serviceName = request.getService(); + ServingStatus status = getServiceStatus(serviceName); + HealthCheckResponse response = buildHealthCheckResponse(status); + responseObserver.onNext(response); + responseObserver.onCompleted(); + } + + @Override + public void watch( + HealthCheckRequest request, + StreamObserver responseObserver + ) + { + String serviceName = request.getService(); + + Context.CancellableContext existingContext = cancellationContexts.get(serviceName); + if (existingContext != null) { + // Another request is already watching the same service + responseObserver.onError(Status.ALREADY_EXISTS.withDescription( + "Another watch request is already in progress for the same service").asRuntimeException()); + return; + } + + Context.CancellableContext cancellableContext = Context.current().withCancellation(); + cancellationContexts.put(serviceName, cancellableContext); + + // Attach a cancellation listener to the context + cancellableContext.addListener((context) -> { + // If the context is cancelled, remove the observer from the map + cancellationContexts.remove(serviceName); + }, MoreExecutors.directExecutor()); + + + // Send an initial response with the current serving status + ServingStatus servingStatus = getServiceStatus(serviceName); + HealthCheckResponse initialResponse = buildHealthCheckResponse(servingStatus); + responseObserver.onNext(initialResponse); + + // Continuously listen for service status changes + while (!cancellableContext.isCancelled()) { + // Wait for the service status to change + // Update the serving status and send a new response + servingStatus = waitForServiceStatusChange(serviceName); + HealthCheckResponse updatedResponse = buildHealthCheckResponse(servingStatus); + responseObserver.onNext(updatedResponse); + } + + cancellationContexts.remove(serviceName); + responseObserver.onCompleted(); + } + + private HealthCheckResponse buildHealthCheckResponse(ServingStatus status) + { + return HealthCheckResponse + .newBuilder() + .setStatus(status) + .build(); + } + + // Method to register a new service with its initial serving status + public void registerService(String serviceName, ServingStatus servingStatus) + { + setServiceStatus(serviceName, servingStatus); + } + + // Method to unregister a service + public void unregisterService(String serviceName) + { + setServiceStatus(serviceName, ServingStatus.NOT_SERVING); + } + + private void setServiceStatus(String serviceName, ServingStatus newStatus) + { + ServingStatus currentStatus = getServiceStatus(serviceName); + if (currentStatus != newStatus) { + serviceStatusMap.put(serviceName, newStatus); + + // Notify the waiting threads + CountDownLatch statusChangeLatch = statusChangeLatchMap.get(serviceName); + if (statusChangeLatch != null) { + statusChangeLatch.countDown(); + } + } + } + + public ServingStatus getServiceStatus(String serviceName) + { + return serviceStatusMap.getOrDefault(serviceName, ServingStatus.UNKNOWN); + } + + public ServingStatus waitForServiceStatusChange(String serviceName) + { + CountDownLatch statusChangeLatch = new CountDownLatch(1); + statusChangeLatchMap.put(serviceName, statusChangeLatch); + + // Wait for the status change or until the thread is interrupted + try { + statusChangeLatch.await(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + + statusChangeLatchMap.remove(serviceName); + + return getServiceStatus(serviceName); + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ProtobufTransformer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ProtobufTransformer.java new file mode 100644 index 00000000000..795ea2b40af --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ProtobufTransformer.java @@ -0,0 +1,167 @@ +/* + * 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.grpc.server; + +import com.google.protobuf.ByteString; +import com.google.protobuf.Timestamp; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.SqlRowTransformer; +import org.apache.druid.sql.calcite.planner.Calcites; +import org.apache.druid.sql.calcite.table.RowSignatures; +import org.joda.time.DateTime; +import org.joda.time.DateTimeUtils; +import org.joda.time.DateTimeZone; + +import javax.annotation.Nullable; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectOutputStream; +import java.util.Optional; +import java.util.TimeZone; + +/** + * Transforms query result for protobuf format + */ +public class ProtobufTransformer +{ + + /** + * Transform a sql query result into protobuf result format. + * For complex or missing column type the object is converted into ByteString. + * date and time column types is converted into proto timestamp. + * Remaining column types are not converted. + * + * @param rowTransformer row signature for sql query result + * @param row result row + * @param i index in the result row + * @return transformed query result in protobuf result format + */ + @Nullable + public static Object transform(SqlRowTransformer rowTransformer, Object[] row, int i) + { + if (row[i] == null) { + return null; + } + final RelDataType rowType = rowTransformer.getRowType(); + final SqlTypeName sqlTypeName = rowType.getFieldList().get(i).getType().getSqlTypeName(); + final RowSignature signature = RowSignatures.fromRelDataType(rowType.getFieldNames(), rowType); + final Optional columnType = signature.getColumnType(i); + + if (sqlTypeName == SqlTypeName.TIMESTAMP + || sqlTypeName == SqlTypeName.DATE) { + if (sqlTypeName == SqlTypeName.TIMESTAMP) { + return convertEpochToProtoTimestamp((long) row[i]); + } + return convertDateToProtoTimestamp((int) row[i]); + } + + if (!columnType.isPresent()) { + return convertComplexType(row[i]); + } + + final ColumnType druidType = columnType.get(); + + if (druidType == ColumnType.STRING) { + return row[i]; + } else if (druidType == ColumnType.LONG) { + return row[i]; + } else if (druidType == ColumnType.FLOAT) { + return row[i]; + } else if (druidType == ColumnType.DOUBLE) { + return row[i]; + } else { + return convertComplexType(row[i]); + } + } + + /** + * Transform a native query result into protobuf result format. + * For complex or missing column type the object is converted into ByteString. + * date and time column types are converted into proto timestamp. + * Remaining column types are not converted. + * + * @param rowSignature type signature for a query result row + * @param row result row + * @param i index in the result + * @param convertToTimestamp if the result should be converted to proto timestamp + * @return transformed query result in protobuf result format + */ + @Nullable + public static Object transform(RowSignature rowSignature, Object[] row, int i, boolean convertToTimestamp) + { + if (row[i] == null) { + return null; + } + + final Optional columnType = rowSignature.getColumnType(i); + + if (convertToTimestamp) { + return convertEpochToProtoTimestamp((long) row[i]); + } + + if (!columnType.isPresent()) { + return convertComplexType(row[i]); + } + + final ColumnType druidType = columnType.get(); + + if (druidType == ColumnType.STRING) { + return row[i]; + } else if (druidType == ColumnType.LONG) { + return row[i]; + } else if (druidType == ColumnType.FLOAT) { + return row[i]; + } else if (druidType == ColumnType.DOUBLE) { + return row[i]; + } else { + return convertComplexType(row[i]); + } + } + + public static Timestamp convertEpochToProtoTimestamp(long value) + { + DateTime dateTime = Calcites.calciteTimestampToJoda(value, DateTimeZone.forTimeZone(TimeZone.getTimeZone("UTC"))); + long seconds = DateTimeUtils.getInstantMillis(dateTime) / 1000; + return Timestamp.newBuilder().setSeconds(seconds).build(); + } + + public static Timestamp convertDateToProtoTimestamp(int value) + { + DateTime dateTime = Calcites.calciteDateToJoda(value, DateTimeZone.forTimeZone(TimeZone.getTimeZone("UTC"))); + long seconds = DateTimeUtils.getInstantMillis(dateTime) / 1000; + return Timestamp.newBuilder().setSeconds(seconds).build(); + } + + private static ByteString convertComplexType(Object value) + { + try (ByteArrayOutputStream bos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(bos)) { + oos.writeObject(value); + oos.flush(); + return ByteString.copyFrom(bos.toByteArray()); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ProtobufWriter.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ProtobufWriter.java new file mode 100644 index 00000000000..bf88c33d08a --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/ProtobufWriter.java @@ -0,0 +1,146 @@ +/* + * 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.grpc.server; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.GeneratedMessageV3; +import com.google.protobuf.Message; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.http.ResultFormat; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.io.OutputStream; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.HashMap; +import java.util.Map; + +/** + * Implementation of {@code ResultFormat.Writer} for protobuf message. + */ +public class ProtobufWriter implements ResultFormat.Writer +{ + private final OutputStream outputStream; + private final GeneratedMessageV3 message; + private Message.Builder rowBuilder; + private final Map methods = new HashMap<>(); + + + public ProtobufWriter(OutputStream outputStream, Class clazz) + { + this.outputStream = outputStream; + this.message = get(clazz); + } + + private GeneratedMessageV3 get(Class clazz) + { + try { + final Method method = clazz.getMethod("getDefaultInstance", new Class[0]); + return clazz.cast(method.invoke(null)); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void writeResponseStart() + { + } + + @Override + public void writeHeader(RelDataType rowType, boolean includeTypes, boolean includeSqlTypes) + { + } + + @Override + public void writeHeaderFromRowSignature(RowSignature rowSignature, boolean b) + { + + } + + @Override + public void writeRowStart() + { + rowBuilder = message.getDefaultInstanceForType().newBuilderForType(); + } + + @Override + public void writeRowField(String name, @Nullable Object value) + { + if (value == null) { + return; + } + final Descriptors.FieldDescriptor fieldDescriptor = + message.getDescriptorForType().findFieldByName(name); + // we should throw an exception if fieldDescriptor is null + // this means the .proto fields don't match returned column names + if (fieldDescriptor == null) { + throw new QueryDriver.RequestError( + "Field [%s] not found in Protobuf [%s]", + name, + message.getClass() + ); + } + final Method method = methods.computeIfAbsent("setField", k -> { + try { + return rowBuilder + .getClass() + .getMethod( + "setField", new Class[]{Descriptors.FieldDescriptor.class, Object.class}); + } + catch (NoSuchMethodException e) { + throw new RuntimeException(e); + } + }); + try { + method.invoke(rowBuilder, fieldDescriptor, value); + } + catch (IllegalAccessException | InvocationTargetException e) { + throw new QueryDriver.RequestError( + "Could not write value [%s] to field [%s]", + value, + name + ); + } + } + + @Override + public void writeRowEnd() throws IOException + { + Message rowMessage = rowBuilder.build(); + rowMessage.writeDelimitedTo(outputStream); + } + + @Override + public void writeResponseEnd() + { + } + + @Override + public void close() throws IOException + { + outputStream.flush(); + outputStream.close(); + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java new file mode 100644 index 00000000000..096a1439a4f --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryDriver.java @@ -0,0 +1,720 @@ +/* + * 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.grpc.server; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; +import com.google.protobuf.ByteString; +import com.google.protobuf.GeneratedMessageV3; +import org.apache.calcite.avatica.SqlType; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.druid.grpc.proto.QueryOuterClass; +import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; +import org.apache.druid.grpc.proto.QueryOuterClass.DruidType; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryParameter; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.guava.Accumulator; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryToolChest; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.server.QueryLifecycle; +import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.sql.DirectStatement; +import org.apache.druid.sql.DirectStatement.ResultSet; +import org.apache.druid.sql.SqlPlanningException; +import org.apache.druid.sql.SqlQueryPlus; +import org.apache.druid.sql.SqlRowTransformer; +import org.apache.druid.sql.SqlStatementFactory; +import org.apache.druid.sql.calcite.table.RowSignatures; +import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.sql.http.SqlParameter; +import org.joda.time.format.ISODateTimeFormat; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; + + +/** + * "Driver" for the gRPC query endpoint. Handles translating the gRPC {@link QueryRequest} + * into Druid's internal formats, running the query, and translating the results into a + * gRPC {@link QueryResponse}. Allows for easier unit testing as we separate the machinery + * of running a query, given the request, from the gRPC server machinery. + */ +public class QueryDriver +{ + private static final Logger log = new Logger(QueryDriver.class); + + private static final String TIME_FIELD_KEY = "timeFieldKey"; + + /** + * Internal runtime exception to report request errors. + */ + protected static class RequestError extends RE + { + public RequestError(String msg, Object... args) + { + super(msg, args); + } + } + + private final ObjectMapper jsonMapper; + private final SqlStatementFactory sqlStatementFactory; + private final QueryLifecycleFactory queryLifecycleFactory; + + public QueryDriver( + final ObjectMapper jsonMapper, + final SqlStatementFactory sqlStatementFactory, + final QueryLifecycleFactory queryLifecycleFactory + ) + { + this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "jsonMapper"); + this.sqlStatementFactory = Preconditions.checkNotNull(sqlStatementFactory, "sqlStatementFactory"); + this.queryLifecycleFactory = queryLifecycleFactory; + } + + /** + * First-cut synchronous query handler. Druid prefers to stream results, in + * part to avoid overly-short network timeouts. However, for now, we simply run + * the query within this call and prepare the Protobuf response. Async handling + * can come later. + */ + public QueryResponse submitQuery(QueryRequest request, AuthenticationResult authResult) + { + if (request.getQueryType() == QueryOuterClass.QueryType.NATIVE) { + return runNativeQuery(request, authResult); + } else { + return runSqlQuery(request, authResult); + } + } + + private QueryResponse runNativeQuery(QueryRequest request, AuthenticationResult authResult) + { + Query query; + try { + query = jsonMapper.readValue(request.getQuery(), Query.class); + } + catch (JsonProcessingException e) { + return QueryResponse.newBuilder() + .setQueryId("") + .setStatus(QueryStatus.REQUEST_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + if (Strings.isNullOrEmpty(query.getId())) { + query = query.withId(UUID.randomUUID().toString()); + } + + final QueryLifecycle queryLifecycle = queryLifecycleFactory.factorize(); + + final org.apache.druid.server.QueryResponse queryResponse; + final String currThreadName = Thread.currentThread().getName(); + try { + queryLifecycle.initialize(query); + Access authorizationResult = queryLifecycle.authorize(authResult); + if (!authorizationResult.isAllowed()) { + throw new ForbiddenException(Access.DEFAULT_ERROR_MESSAGE); + } + queryResponse = queryLifecycle.execute(); + + QueryToolChest queryToolChest = queryLifecycle.getToolChest(); + + Sequence sequence = queryToolChest.resultsAsArrays(query, queryResponse.getResults()); + RowSignature rowSignature = queryToolChest.resultArraySignature(query); + + Thread.currentThread().setName(StringUtils.format("grpc-native[%s]", query.getId())); + final ByteString results = encodeNativeResults(request, sequence, rowSignature); + return QueryResponse.newBuilder() + .setQueryId(query.getId()) + .setStatus(QueryStatus.OK) + .setData(results) + .clearErrorMessage() + .addAllColumns(encodeNativeColumns(rowSignature, request.getSkipColumnsList())) + .build(); + } + catch (IOException | RuntimeException e) { + return QueryResponse.newBuilder() + .setQueryId(query.getId()) + .setStatus(QueryStatus.RUNTIME_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + finally { + Thread.currentThread().setName(currThreadName); + } + } + + private QueryResponse runSqlQuery(QueryRequest request, AuthenticationResult authResult) + { + final SqlQueryPlus queryPlus; + try { + queryPlus = translateQuery(request, authResult); + } + catch (RuntimeException e) { + return QueryResponse.newBuilder() + .setQueryId("") + .setStatus(QueryStatus.REQUEST_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + final DirectStatement stmt = sqlStatementFactory.directStatement(queryPlus); + final String currThreadName = Thread.currentThread().getName(); + try { + Thread.currentThread().setName(StringUtils.format("grpc-sql[%s]", stmt.sqlQueryId())); + final ResultSet thePlan = stmt.plan(); + final SqlRowTransformer rowTransformer = thePlan.createRowTransformer(); + final ByteString results = encodeSqlResults(request, thePlan.run().getResults(), rowTransformer); + stmt.reporter().succeeded(results.size()); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.OK) + .setData(results) + .clearErrorMessage() + .addAllColumns(encodeSqlColumns(rowTransformer)) + .build(); + } + catch (ForbiddenException e) { + stmt.reporter().failed(e); + stmt.close(); + throw e; + } + catch (RequestError e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.REQUEST_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + catch (SqlPlanningException e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.INVALID_SQL) + .setErrorMessage(e.getMessage()) + .build(); + } + catch (IOException | RuntimeException e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.RUNTIME_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + // There is a claim that Calcite sometimes throws a java.lang.AssertionError, but we do not have a test that can + // reproduce it checked into the code (the best we have is something that uses mocks to throw an Error, which is + // dubious at best). We keep this just in case, but it might be best to remove it and see where the + // AssertionErrors are coming from and do something to ensure that they don't actually make it out of Calcite + catch (AssertionError e) { + stmt.reporter().failed(e); + stmt.close(); + return QueryResponse.newBuilder() + .setQueryId(stmt.sqlQueryId()) + .setStatus(QueryStatus.RUNTIME_ERROR) + .setErrorMessage(e.getMessage()) + .build(); + } + finally { + Thread.currentThread().setName(currThreadName); + } + } + + /** + * Convert the rRPC query format to the internal {@link SqlQueryPlus} format. + */ + private SqlQueryPlus translateQuery(QueryRequest request, AuthenticationResult authResult) + { + return SqlQueryPlus.builder() + .sql(request.getQuery()) + .context(translateContext(request)) + .sqlParameters(translateParameters(request)) + .auth(authResult) + .build(); + } + + /** + * Translate the query context from the gRPC format to the internal format. When + * read from REST/JSON, the JSON translator will convert the type of each value + * into a number, Boolean, etc. gRPC has no similar feature. Rather than clutter up + * the gRPC request with typed context values, we rely on the existing code that can + * translate string values to the desired type on the fly. Thus, we build up a + * {@code Map}. + */ + private Map translateContext(QueryRequest request) + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + if (request.getContextCount() > 0) { + for (Map.Entry entry : request.getContextMap().entrySet()) { + builder.put(entry.getKey(), entry.getValue()); + } + } + return builder.build(); + } + + /** + * Convert the gRPC parameter format to the internal Druid {@link SqlParameter} + * format. That format is then again translated by the {@link SqlQueryPlus} class. + */ + private List translateParameters(QueryRequest request) + { + if (request.getParametersCount() == 0) { + return null; + } + List params = new ArrayList<>(); + for (QueryParameter value : request.getParametersList()) { + params.add(translateParameter(value)); + } + return params; + } + + private SqlParameter translateParameter(QueryParameter value) + { + switch (value.getValueCase()) { + case DOUBLEVALUE: + return new SqlParameter(SqlType.DOUBLE, value.getDoubleValue()); + case LONGVALUE: + return new SqlParameter(SqlType.BIGINT, value.getLongValue()); + case STRINGVALUE: + return new SqlParameter(SqlType.VARCHAR, value.getStringValue()); + case NULLVALUE: + case VALUE_NOT_SET: + return null; + case ARRAYVALUE: + default: + throw new RequestError("Invalid parameter type: " + value.getValueCase().name()); + } + } + + /** + * Translate the column schema from the Druid internal form to the gRPC + * {@link ColumnSchema} form. Note that since the gRPC response returns the + * schema, none of the data formats include a header. This makes the data format + * simpler and cleaner. + */ + private Iterable encodeSqlColumns(SqlRowTransformer rowTransformer) + { + RelDataType rowType = rowTransformer.getRowType(); + final RowSignature signature = RowSignatures.fromRelDataType(rowType.getFieldNames(), rowType); + List cols = new ArrayList<>(); + for (int i = 0; i < rowType.getFieldCount(); i++) { + ColumnSchema col = ColumnSchema.newBuilder() + .setName(signature.getColumnName(i)) + .setSqlType(rowType.getFieldList().get(i).getType().getSqlTypeName().getName()) + .setDruidType(convertDruidType(signature.getColumnType(i))) + .build(); + cols.add(col); + } + return cols; + } + + private Iterable encodeNativeColumns(RowSignature rowSignature, List skipColumns) + { + List cols = new ArrayList<>(); + for (int i = 0; i < rowSignature.getColumnNames().size(); i++) { + if (skipColumns.contains(rowSignature.getColumnName(i))) { + continue; + } + ColumnSchema col = ColumnSchema.newBuilder() + .setName(rowSignature.getColumnName(i)) + .setDruidType(convertDruidType(rowSignature.getColumnType(i))) + .build(); + cols.add(col); + } + return cols; + } + + /** + * Convert from Druid's internal format of the Druid data type to the gRPC form. + */ + private DruidType convertDruidType(Optional colType) + { + if (!colType.isPresent()) { + return DruidType.UNKNOWN_TYPE; + } + ColumnType druidType = colType.get(); + if (druidType == ColumnType.STRING) { + return DruidType.STRING; + } + if (druidType == ColumnType.STRING_ARRAY) { + return DruidType.STRING_ARRAY; + } + if (druidType == ColumnType.LONG) { + return DruidType.LONG; + } + if (druidType == ColumnType.LONG_ARRAY) { + return DruidType.LONG_ARRAY; + } + if (druidType == ColumnType.FLOAT) { + return DruidType.FLOAT; + } + if (druidType == ColumnType.FLOAT_ARRAY) { + return DruidType.FLOAT_ARRAY; + } + if (druidType == ColumnType.DOUBLE) { + return DruidType.DOUBLE; + } + if (druidType == ColumnType.DOUBLE_ARRAY) { + return DruidType.DOUBLE_ARRAY; + } + if (druidType == ColumnType.UNKNOWN_COMPLEX) { + return DruidType.COMPLEX; + } + return DruidType.UNKNOWN_TYPE; + } + + /** + * Generic mechanism to write query results to one of the supported gRPC formats. + */ + public interface GrpcResultWriter + { + void start() throws IOException; + + void writeRow(Object[] row) throws IOException; + + void close() throws IOException; + } + + /** + * Writer for the SQL result formats. Reuses the SQL format writer implementations. + * Note: gRPC does not use the headers: schema information is available in the + * rRPC response. + */ + public static class GrpcSqlResultFormatWriter implements GrpcResultWriter + { + protected final ResultFormat.Writer formatWriter; + protected final SqlRowTransformer rowTransformer; + + public GrpcSqlResultFormatWriter( + final ResultFormat.Writer formatWriter, + final SqlRowTransformer rowTransformer + ) + { + this.formatWriter = formatWriter; + this.rowTransformer = rowTransformer; + } + + @Override + public void start() throws IOException + { + formatWriter.writeResponseStart(); + } + + @Override + public void writeRow(Object[] row) throws IOException + { + formatWriter.writeRowStart(); + for (int i = 0; i < rowTransformer.getFieldList().size(); i++) { + final Object value; + if (formatWriter instanceof ProtobufWriter) { + value = ProtobufTransformer.transform(rowTransformer, row, i); + } else { + value = rowTransformer.transform(row, i); + } + formatWriter.writeRowField(rowTransformer.getFieldList().get(i), value); + } + formatWriter.writeRowEnd(); + } + + @Override + public void close() throws IOException + { + formatWriter.writeResponseEnd(); + formatWriter.close(); + } + } + + public static class GrpcNativeResultFormatWriter implements GrpcResultWriter + { + protected final ResultFormat.Writer formatWriter; + protected final RowSignature rowSignature; + private final String timeFieldName; + private final List timeColumns; + private final List skipColumns; + + public GrpcNativeResultFormatWriter( + final ResultFormat.Writer formatWriter, + final RowSignature rowSignature, + final String timeFieldName, + final List timeColumns, + final List skipColumns + ) + { + this.formatWriter = formatWriter; + this.rowSignature = rowSignature; + this.timeFieldName = timeFieldName; + this.timeColumns = timeColumns; + this.skipColumns = skipColumns; + } + + @Override + public void start() throws IOException + { + formatWriter.writeResponseStart(); + } + + @Override + public void writeRow(Object[] row) throws IOException + { + formatWriter.writeRowStart(); + + for (int i = 0; i < rowSignature.getColumnNames().size(); i++) { + + final String columnName = rowSignature.getColumnName(i); + if (skipColumns.contains(columnName)) { + log.debug("Skipping column [%s] from the result.", columnName); + continue; + } + + boolean isDruidTimeColumn = columnName.equals(ColumnHolder.TIME_COLUMN_NAME); + boolean convertTime = timeColumns.contains(rowSignature.getColumnName(i)); + + final Object value; + if (formatWriter instanceof ProtobufWriter) { + value = ProtobufTransformer.transform(rowSignature, row, i, convertTime); + } else { + if (convertTime) { + value = ISODateTimeFormat.dateTime().print(((long) row[i])); + } else { + value = row[i]; + } + } + final String outputColumnName; + if (isDruidTimeColumn) { + outputColumnName = timeFieldName; + } else { + outputColumnName = rowSignature.getColumnName(i); + } + formatWriter.writeRowField(outputColumnName, value); + } + formatWriter.writeRowEnd(); + } + + + @Override + public void close() throws IOException + { + formatWriter.writeResponseEnd(); + formatWriter.close(); + } + } + + /** + * Internal runtime exception to pass {@link IOException}s though the + * {@link Sequence} {@link Accumulator} protocol. + */ + private static class ResponseError extends RuntimeException + { + public ResponseError(IOException e) + { + super(e); + } + } + + /** + * Druid query results use a complex {@link Sequence} mechanism. This class uses an + * {@link Accumulator} to walk the results and present each to the associated + * results writer. This is a very rough analogy of the {@code SqlResourceQueryResultPusher} + * in the REST {@code SqlResource} class. + */ + public static class GrpcResultsAccumulator implements Accumulator + { + private final GrpcResultWriter writer; + + public GrpcResultsAccumulator(final GrpcResultWriter writer) + { + this.writer = writer; + } + + public void push(Sequence results) throws IOException + { + writer.start(); + try { + results.accumulate(null, this); + } + catch (ResponseError e) { + throw (IOException) e.getCause(); + } + writer.close(); + } + + @Override + public Void accumulate(Void accumulated, Object[] in) + { + try { + writer.writeRow(in); + } + catch (IOException e) { + throw new ResponseError(e); + } + return null; + } + } + + /** + * Convert the query results to a set of bytes to be attached to the query response. + *

    + * This version is pretty basic: the results are materialized as a byte array. That's + * fine for small result sets, but should be rethought for larger result sets. + */ + private ByteString encodeSqlResults( + final QueryRequest request, + final Sequence result, + final SqlRowTransformer rowTransformer + ) throws IOException + { + // Accumulate the results as a byte array. + ByteArrayOutputStream out = new ByteArrayOutputStream(); + GrpcResultWriter writer; + + // For the SQL-supported formats, use the SQL-provided writers. + switch (request.getResultFormat()) { + case CSV: + writer = new GrpcSqlResultFormatWriter( + ResultFormat.CSV.createFormatter(out, jsonMapper), + rowTransformer + ); + break; + case JSON_ARRAY: + writer = new GrpcSqlResultFormatWriter( + ResultFormat.ARRAY.createFormatter(out, jsonMapper), + rowTransformer + ); + break; + case JSON_ARRAY_LINES: + writer = new GrpcSqlResultFormatWriter( + ResultFormat.ARRAYLINES.createFormatter(out, jsonMapper), + rowTransformer + ); + break; + case PROTOBUF_INLINE: + writer = new GrpcSqlResultFormatWriter( + new ProtobufWriter(out, getProtobufClass(request)), + rowTransformer + ); + break; + default: + throw new RequestError("Unsupported query result format: " + request.getResultFormat().name()); + } + GrpcResultsAccumulator accumulator = new GrpcResultsAccumulator(writer); + accumulator.push(result); + return ByteString.copyFrom(out.toByteArray()); + } + + private ByteString encodeNativeResults( + final QueryRequest request, + final Sequence result, + final RowSignature rowSignature + ) throws IOException + { + // Accumulate the results as a byte array. + ByteArrayOutputStream out = new ByteArrayOutputStream(); + GrpcResultWriter writer; + final String timeFieldName = request.getContextMap().getOrDefault(TIME_FIELD_KEY, "time"); + final List skipColumns = request.getSkipColumnsList(); + final List timeColumns = request.getTimeColumnsList(); + + switch (request.getResultFormat()) { + case CSV: + writer = new GrpcNativeResultFormatWriter( + ResultFormat.CSV.createFormatter(out, jsonMapper), + rowSignature, + timeFieldName, + timeColumns, + skipColumns + ); + break; + case JSON_ARRAY: + writer = new GrpcNativeResultFormatWriter( + ResultFormat.ARRAY.createFormatter(out, jsonMapper), + rowSignature, + timeFieldName, + timeColumns, + skipColumns + ); + break; + case JSON_ARRAY_LINES: + writer = new GrpcNativeResultFormatWriter( + ResultFormat.ARRAYLINES.createFormatter(out, jsonMapper), + rowSignature, + timeFieldName, + timeColumns, + skipColumns + ); + break; + case PROTOBUF_INLINE: + writer = new GrpcNativeResultFormatWriter( + new ProtobufWriter(out, getProtobufClass(request)), + rowSignature, + timeFieldName, + timeColumns, + skipColumns + ); + break; + default: + throw new RequestError("Unsupported query result format: " + request.getResultFormat()); + } + GrpcResultsAccumulator accumulator = new GrpcResultsAccumulator(writer); + accumulator.push(result); + return ByteString.copyFrom(out.toByteArray()); + } + + @SuppressWarnings("unchecked") + private Class getProtobufClass(final QueryRequest request) + { + try { + return (Class) Class.forName(request.getProtobufMessageName()); + } + catch (ClassNotFoundException e) { + throw new RequestError( + "The Protobuf class [%s] is not known. Is your protobuf jar on the class path?", + request.getProtobufMessageName() + ); + } + catch (ClassCastException e) { + throw new RequestError( + "The class [%s] is not a Protobuf", + request.getProtobufMessageName() + ); + } + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java new file mode 100644 index 00000000000..569ed69ef3e --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryServer.java @@ -0,0 +1,143 @@ +/* + * 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.grpc.server; + +import io.grpc.Context; +import io.grpc.Grpc; +import io.grpc.InsecureServerCredentials; +import io.grpc.Server; +import io.grpc.ServerInterceptor; +import io.grpc.ServerInterceptors; +import org.apache.druid.grpc.proto.HealthOuterClass.HealthCheckResponse.ServingStatus; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.security.AllowAllAuthenticator; +import org.apache.druid.server.security.AnonymousAuthenticator; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.Authenticator; +import org.apache.druid.server.security.AuthenticatorMapper; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + + +/** + * Basic gRPC server adapted from the gRPC examples. Delegates to the + * {@link QueryDriver} class to do the actual work of running the query. + *

    + * This class is preliminary. It is good enough for unit tests, but a bit more work + * is needed to integrate this class into the Druid server. + *

    + * Also, how will authorization be handled in the gRPC path? + */ +public class QueryServer +{ + public static final Context.Key AUTH_KEY = Context.key("druid-auth"); + private static final Logger log = new Logger(QueryServer.class); + + private final AuthenticatorMapper authMapper; + private final int port; + private final QueryDriver driver; + private Server server; + + private final HealthService healthService; + + public QueryServer( + GrpcQueryConfig config, + QueryDriver driver, + AuthenticatorMapper authMapper + ) + { + this.port = config.getPort(); + this.driver = driver; + this.authMapper = authMapper; + this.healthService = new HealthService(); + } + + public void start() throws IOException + { + server = Grpc.newServerBuilderForPort(port, InsecureServerCredentials.create()) + .addService(ServerInterceptors.intercept(new QueryService(driver), makeSecurityInterceptor())) + .addService(healthService) + .build() + .start(); + + healthService.registerService(QueryService.class.getSimpleName(), ServingStatus.SERVING); + healthService.registerService("", ServingStatus.SERVING); + + log.info("Grpc Server started, listening on " + port); + } + + + /** + * Map from a Druid authenticator to a gRPC server interceptor. This is a bit of a hack. + * Authenticators don't know about gRPC: we have to explicitly do the mapping. This means + * that auth extensions occur independently of gRPC and are not supported. Longer term, + * we need a way for the extension itself to do the required mapping. + */ + private ServerInterceptor makeSecurityInterceptor() + { + // First look for a Basic authenticator + for (Authenticator authenticator : authMapper.getAuthenticatorChain()) { + // Want authenticator instanceof BasicHTTPAuthenticator, but + // BasicHTTPAuthenticator is not visible here. + if ("BasicHTTPAuthenticator".equals(authenticator.getClass().getSimpleName())) { + log.info("Using Basic authentication"); + return new BasicAuthServerInterceptor(authenticator); + } + } + + // Otherwise, look for an Anonymous authenticator + for (Authenticator authenticator : authMapper.getAuthenticatorChain()) { + if (authenticator instanceof AnonymousAuthenticator || authenticator instanceof AllowAllAuthenticator) { + log.info("Using Anonymous authentication"); + return new AnonymousAuthServerInterceptor(authenticator); + } + } + + // gRPC does not support other forms of authenticators yet. + String msg = "The gRPC query server requires either a Basic or Anonymous authorizer: it does not work with others yet."; + log.error(msg); + throw new UOE(msg); + } + + public void stop() throws InterruptedException + { + if (server != null) { + log.info("Server stopping"); + healthService.unregisterService(QueryService.class.getSimpleName()); + healthService.unregisterService(""); + server.shutdown().awaitTermination(30, TimeUnit.SECONDS); + } + } + + /** + * Await termination on the main thread since the grpc library uses daemon threads. + */ + public void blockUntilShutdown() throws InterruptedException + { + if (server != null) { + log.info("Grpc Server stopping"); + healthService.unregisterService(QueryService.class.getSimpleName()); + healthService.unregisterService(""); + server.awaitTermination(); + } + } +} diff --git a/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.java new file mode 100644 index 00000000000..ddd99c456e3 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/java/org/apache/druid/grpc/server/QueryService.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.grpc.server; + +import io.grpc.Status; +import io.grpc.StatusRuntimeException; +import io.grpc.stub.StreamObserver; +import org.apache.druid.grpc.proto.QueryGrpc; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.server.security.ForbiddenException; + +/** + * Implementation of the gRPC Query service. Provides a single method + * to run a query using the "driver" that holds the actual Druid SQL + * logic. + */ +class QueryService extends QueryGrpc.QueryImplBase +{ + private final QueryDriver driver; + + public QueryService(QueryDriver driver) + { + this.driver = driver; + } + + @Override + public void submitQuery(QueryRequest request, StreamObserver responseObserver) + { + try { + QueryResponse reply = driver.submitQuery(request, QueryServer.AUTH_KEY.get()); + responseObserver.onNext(reply); + responseObserver.onCompleted(); + } + catch (ForbiddenException e) { + // This block mimics the Servlet pattern of throwing ForbiddenException for + // all access denied cases rather than handling permissions in each message + // handler. + responseObserver.onError(new StatusRuntimeException(Status.PERMISSION_DENIED)); + } + } +} diff --git a/extensions-contrib/grpc-query/src/main/proto/health.proto b/extensions-contrib/grpc-query/src/main/proto/health.proto new file mode 100644 index 00000000000..304a2e8bd21 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/proto/health.proto @@ -0,0 +1,44 @@ +// 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. + +syntax = "proto3"; + +option java_package = "org.apache.druid.grpc.proto"; + +package druidGrpc; + +message HealthCheckRequest { + string service = 1; +} + +message HealthCheckResponse { + enum ServingStatus { + UNKNOWN = 0; + SERVING = 1; + NOT_SERVING = 2; + SERVICE_UNKNOWN = 3; // Used only by the Watch method. + } + ServingStatus status = 1; +} + +service Health { + // returns health of the server + rpc Check(HealthCheckRequest) returns (HealthCheckResponse); + + // streaming health check api. The server will immediately send back a message indicating + // the current serving status. It will then subsequently send a new message whenever the + // service's serving status changes. + rpc Watch(HealthCheckRequest) returns (stream HealthCheckResponse); +} diff --git a/extensions-contrib/grpc-query/src/main/proto/query.proto b/extensions-contrib/grpc-query/src/main/proto/query.proto new file mode 100644 index 00000000000..bd14e222ac7 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/proto/query.proto @@ -0,0 +1,144 @@ +// 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. + +syntax = "proto3"; + +option java_package = "org.apache.druid.grpc.proto"; + +import "google/protobuf/timestamp.proto"; + +package druidGrpc; + +service Query { + rpc SubmitQuery (QueryRequest) returns (QueryResponse) {} +} + +enum QueryType { + SQL = 0; + NATIVE = 1; +} + +// Defines the format of the query results. Must be one of the +// non-unknown formats. +enum QueryResultFormat { + UNKNOWN_FORMAT = 0; // Default value. An old server will see this value + // (and fail the request) if a new client passes a new format. + CSV = 1; + // JSON_OBJECT = 2; -- Not yet + JSON_ARRAY = 3; + // JSON_OBJECT_LINES = 4; -- Not yet + JSON_ARRAY_LINES = 5; + + // The protobuf formats also require that the protobufMessageName be set in + // the query request. + PROTOBUF_INLINE = 6; + PROTOBUF_RESPONSE = 7; +} + +// Value for a query parameter. The value is essentially a variant of the +// supported parameter types. The type chosen here must match (or be converable +// to) the type of the corresponding expression in the SQL statement. +message QueryParameter { + oneof value { + bool nullValue = 1; + string stringValue = 2; + sint64 longValue = 3; + double doubleValue = 4; + StringArray arrayValue = 5; + } +} + +// Query pararameter value for string array properties. At present, string +// arrays are primarily used by the MSQ engine, which is not yet available +// via the gRPC API. +message StringArray { + repeated string value = 1; +} + +message QueryRequest { + string query = 1; + QueryResultFormat resultFormat = 2; + map context = 3; + // Query parameters. If your query is SELECT * FROM foo WHERE x = ? AND y > ? + // Then you would include two parameters in the order in which the question + // marks lexically appear in the query. + repeated QueryParameter parameters = 4; + // The name of the Protobuf message to encode the response if the + // resultFormat is one of the PROTOBUF formats. + optional string protobufMessageName = 5; + + // used only for native query + // columns to skip writing in the result, for example, it can used to skip writing + // time field in the result for timeseries query + repeated string skipColumns = 6; + + // used only for native query + // columns which should be converted to Timestamp + repeated string timeColumns = 7; + + QueryType queryType = 8; +} + +// Unauthorized errors return as a StatusRuntimeException with +// getStatus().getCode() == Status.Code.PERMISSION_DENIED +enum QueryStatus { + UNKNOWN_STATUS = 0; // Default value which means "unknown failure". Older clients + // will see this value if a future version adds a new failure + // type. + OK = 1; + REQUEST_ERROR = 3; + INVALID_SQL = 4; + RUNTIME_ERROR = 5; +} + +enum DruidType { + UNKNOWN_TYPE = 0; // Default value for unknown, or for future new values + // as seen by old clients. + STRING = 1; + LONG = 2; + DOUBLE = 3; + FLOAT = 4; + STRING_ARRAY = 5; + LONG_ARRAY = 6; + DOUBLE_ARRAY = 7; + FLOAT_ARRAY = 8; + COMPLEX = 9; +} + +message ColumnSchema { + string name = 1; + string sqlType = 2; + DruidType druidType = 3; +} + +message QueryResponse { + string queryId = 1; + QueryStatus status = 2; + + // Error message if the query fails. Not set if the query succeeds. + optional string errorMessage = 3; + + // The schema of the returned results. This schema is redundant for the + // JSON and Protobuf formats. It can be used to generate column heads, + // and understand types, for the CSV result format. + repeated ColumnSchema columns = 4; + + // The query response, encoded using the requested response format. + // Note that the entire response is placed into a single messages. As + // a result, this RPC is intended ONLY for queries that return small + // result sets. It will perform poorly (and consume excess memory) if + // used for large result sets. + optional bytes data = 5; +} diff --git a/extensions-contrib/grpc-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/grpc-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule new file mode 100644 index 00000000000..e35e9a02699 --- /dev/null +++ b/extensions-contrib/grpc-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -0,0 +1,16 @@ +# 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. + +org.apache.druid.grpc.guice.GrpcQueryModule diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/BasicAuthTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/BasicAuthTest.java new file mode 100644 index 00000000000..d7d01843234 --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/BasicAuthTest.java @@ -0,0 +1,216 @@ +/* + * 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.grpc; + +import com.google.common.collect.ImmutableMap; +import io.grpc.Status; +import io.grpc.StatusRuntimeException; +import org.apache.druid.grpc.proto.QueryOuterClass; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.apache.druid.grpc.server.GrpcQueryConfig; +import org.apache.druid.grpc.server.QueryDriver; +import org.apache.druid.grpc.server.QueryServer; +import org.apache.druid.metadata.DefaultPasswordProvider; +import org.apache.druid.security.basic.authentication.BasicHTTPAuthenticator; +import org.apache.druid.security.basic.authentication.validator.CredentialsValidator; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthenticatorMapper; +import org.apache.druid.sql.calcite.BaseCalciteQueryTest; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.calcite.util.SqlTestFramework; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +/** + * Simple test that runs the gRPC server, on top of a test SQL stack. + * Uses a simple client to send a query to the server. This is a basic + * sanity check of the gRPC stack. Uses allow-all security, which + * does a sanity check of the auth chain. + */ +public class BasicAuthTest extends BaseCalciteQueryTest +{ + private static QueryServer server; + + @BeforeEach + public void setup() throws IOException + { + SqlTestFramework sqlTestFramework = queryFramework(); + SqlTestFramework.PlannerFixture plannerFixture = sqlTestFramework.plannerFixture( + BaseCalciteQueryTest.PLANNER_CONFIG_DEFAULT, + new AuthConfig() + ); + QueryDriver driver = new QueryDriver( + sqlTestFramework.queryJsonMapper(), + plannerFixture.statementFactory(), + sqlTestFramework.queryLifecycleFactory() + ); + + CredentialsValidator validator = new CredentialsValidator() + { + @Override + public AuthenticationResult validateCredentials(String authenticatorName, String authorizerName, + String username, char[] password) + { + if (CalciteTests.TEST_SUPERUSER_NAME.equals(username)) { + if (!"secret".equals(new String(password))) { + return null; + } + return CalciteTests.SUPER_USER_AUTH_RESULT; + } + if ("regular".equals(username)) { + if (!"pwd".equals(new String(password))) { + return null; + } + return CalciteTests.REGULAR_USER_AUTH_RESULT; + } + return null; + } + }; + BasicHTTPAuthenticator basicAuth = new BasicHTTPAuthenticator( + null, + "test", + "test", + new DefaultPasswordProvider("druid"), + new DefaultPasswordProvider("druid"), + null, + null, + null, + false, + validator + ); + AuthenticatorMapper authMapper = new AuthenticatorMapper( + ImmutableMap.of( + "test", + basicAuth + ) + ); + GrpcQueryConfig config = new GrpcQueryConfig(50051); + server = new QueryServer(config, driver, authMapper); + try { + server.start(); + } + catch (IOException e) { + e.printStackTrace(); + throw e; + } + catch (RuntimeException e) { + e.printStackTrace(); + throw e; + } + } + + @AfterEach + public void tearDown() throws InterruptedException + { + if (server != null) { + server.stop(); + server.blockUntilShutdown(); + } + } + + @Test + public void testMissingAuth() + { + QueryRequest request = QueryRequest.newBuilder() + .setQuery("SELECT * FROM foo") + .setResultFormat(QueryResultFormat.CSV) + .setQueryType(QueryOuterClass.QueryType.SQL) + .build(); + + try (TestClient client = new TestClient(TestClient.DEFAULT_HOST)) { + StatusRuntimeException e = assertThrows(StatusRuntimeException.class, () -> client.getQueryClient().submitQuery(request)); + assertEquals(Status.PERMISSION_DENIED, e.getStatus()); + } + } + + @Test + public void testInvalidUser() + { + QueryRequest request = QueryRequest.newBuilder() + .setQuery("SELECT * FROM foo") + .setResultFormat(QueryResultFormat.CSV) + .setQueryType(QueryOuterClass.QueryType.SQL) + .build(); + + try (TestClient client = new TestClient(TestClient.DEFAULT_HOST, "invalid", "pwd")) { + StatusRuntimeException e = assertThrows(StatusRuntimeException.class, () -> client.getQueryClient().submitQuery(request)); + assertEquals(Status.PERMISSION_DENIED, e.getStatus()); + } + } + + @Test + public void testInvalidPassword() + { + QueryRequest request = QueryRequest.newBuilder() + .setQuery("SELECT * FROM foo") + .setResultFormat(QueryResultFormat.CSV) + .setQueryType(QueryOuterClass.QueryType.SQL) + .build(); + + try (TestClient client = new TestClient(TestClient.DEFAULT_HOST, CalciteTests.TEST_SUPERUSER_NAME, "invalid")) { + StatusRuntimeException e = assertThrows(StatusRuntimeException.class, () -> client.getQueryClient().submitQuery(request)); + assertEquals(Status.PERMISSION_DENIED, e.getStatus()); + } + } + + @Test + public void testValidUser() + { + QueryRequest request = QueryRequest.newBuilder() + .setQuery("SELECT * FROM foo") + .setResultFormat(QueryResultFormat.CSV) + .setQueryType(QueryOuterClass.QueryType.SQL) + .build(); + + try (TestClient client = new TestClient(TestClient.DEFAULT_HOST, CalciteTests.TEST_SUPERUSER_NAME, "secret")) { + QueryResponse response = client.getQueryClient().submitQuery(request); + assertEquals(QueryStatus.OK, response.getStatus()); + } + try (TestClient client = new TestClient(TestClient.DEFAULT_HOST, "regular", "pwd")) { + QueryResponse response = client.getQueryClient().submitQuery(request); + assertEquals(QueryStatus.OK, response.getStatus()); + } + } + + @Test + public void testUnauthorized() + { + QueryRequest request = QueryRequest.newBuilder() + .setQuery("SELECT * FROM forbiddenDatasource") + .setResultFormat(QueryResultFormat.CSV) + .setQueryType(QueryOuterClass.QueryType.SQL) + .build(); + + try (TestClient client = new TestClient(TestClient.DEFAULT_HOST, "regular", "pwd")) { + StatusRuntimeException e = assertThrows(StatusRuntimeException.class, () -> client.getQueryClient().submitQuery(request)); + assertEquals(Status.PERMISSION_DENIED, e.getStatus()); + } + } +} diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/BasicCredentials.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/BasicCredentials.java new file mode 100644 index 00000000000..66f0501acbb --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/BasicCredentials.java @@ -0,0 +1,71 @@ +/* + * 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.grpc; + +import io.grpc.CallCredentials; +import io.grpc.Metadata; + +import java.nio.charset.StandardCharsets; +import java.util.Base64; +import java.util.concurrent.Executor; + +/** + * Applies Basic Auth credentials to an outgoing request. Use this + * class to set the Basic Auth user name and password on a gRPC client: + *

    
    + * QueryBlockingStub client = QueryGrpc.newBlockingStub(channel)
    + *     .withCallCredentials( new BasicCredentials(user, password));
    + * 
    + */ +public class BasicCredentials extends CallCredentials +{ + public static final String AUTHORIZATION_HEADER = "Authorization"; + private static final Metadata.Key AUTH_KEY = Metadata.Key.of(AUTHORIZATION_HEADER, Metadata.ASCII_STRING_MARSHALLER); + + public final String user; + public final String password; + + public BasicCredentials(String user, String password) + { + this.user = user; + this.password = password; + } + + @Override + public void applyRequestMetadata(RequestInfo requestInfo, Executor exec, MetadataApplier applier) + { + Metadata metadata = new Metadata(); + metadata.put(AUTH_KEY, getBasicAuthenticationHeader(user, password)); + applier.apply(metadata); + } + + // Source: https://www.baeldung.com/java-httpclient-basic-auth#authenticate-using-http-headers + private static String getBasicAuthenticationHeader(String username, String password) + { + String valueToEncode = username + ":" + password; + return "Basic " + Base64.getEncoder().encodeToString(valueToEncode.getBytes(StandardCharsets.UTF_8)); + } + + @Override + public void thisUsesUnstableApi() + { + // We've been warned. + } +} diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java new file mode 100644 index 00000000000..0c7a6506b51 --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/DriverTest.java @@ -0,0 +1,221 @@ +/* + * 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.grpc; + +import org.apache.druid.grpc.proto.QueryOuterClass; +import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; +import org.apache.druid.grpc.proto.QueryOuterClass.DruidType; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.apache.druid.grpc.server.QueryDriver; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.sql.calcite.BaseCalciteQueryTest; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.calcite.util.SqlTestFramework; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class DriverTest extends BaseCalciteQueryTest +{ + private QueryDriver driver; + + @BeforeEach + public void setup() + { + SqlTestFramework sqlTestFramework = queryFramework(); + SqlTestFramework.PlannerFixture plannerFixture = sqlTestFramework.plannerFixture( + BaseCalciteQueryTest.PLANNER_CONFIG_DEFAULT, + new AuthConfig() + ); + driver = new QueryDriver( + sqlTestFramework.queryJsonMapper(), + plannerFixture.statementFactory(), + sqlTestFramework.queryLifecycleFactory() + ); + } + + @Test + public void testBasics_sql() + { + String sql = "SELECT __time, dim2 FROM foo"; + QueryRequest request = QueryRequest.newBuilder() + .setQuery(sql) + .setResultFormat(QueryResultFormat.CSV) + .setQueryType(QueryOuterClass.QueryType.SQL) + .build(); + QueryResponse response = driver.submitQuery(request, CalciteTests.REGULAR_USER_AUTH_RESULT); + + assertEquals(QueryStatus.OK, response.getStatus()); + assertFalse(response.hasErrorMessage()); + assertTrue(response.getQueryId().length() > 5); + List columns = response.getColumnsList(); + assertEquals(2, columns.size()); + ColumnSchema col = columns.get(0); + assertEquals("__time", col.getName()); + assertEquals("TIMESTAMP", col.getSqlType()); + assertEquals(DruidType.LONG, col.getDruidType()); + col = columns.get(1); + assertEquals("dim2", col.getName()); + assertEquals("VARCHAR", col.getSqlType()); + assertEquals(DruidType.STRING, col.getDruidType()); + + List expectedResults = Arrays.asList( + "2000-01-01T00:00:00.000Z,a", + "2000-01-02T00:00:00.000Z,", + "2000-01-03T00:00:00.000Z,", + "2001-01-01T00:00:00.000Z,a", + "2001-01-02T00:00:00.000Z,abc", + "2001-01-03T00:00:00.000Z," + ); + String results = response.getData().toStringUtf8(); + assertEquals(expectedResults, Arrays.asList(results.split("\n"))); + } + + @Test + public void testBasics_native_groupby() + { + String query = "{\n" + + " \"queryType\": \"groupBy\",\n" + + " \"dataSource\": \"wikipedia\",\n" + + " \"dimensions\": [\n" + + " {\n" + + " \"type\": \"default\",\n" + + " \"dimension\": \"countryName\",\n" + + " \"outputType\": \"STRING\"\n" + + " }\n" + + " ],\n" + + " \"aggregations\": [\n" + + " {\n" + + " \"type\": \"longSum\",\n" + + " \"name\": \"sum\",\n" + + " \"fieldName\": \"added\"\n" + + " }\n" + + " ],\n" + + " \"filter\": {\n" + + " \"type\": \"in\",\n" + + " \"dimension\": \"countryName\",\n" + + " \"values\": [\n" + + " \"Albania\",\n" + + " \"Angola\"\n" + + " ]\n" + + " }," + + " \"intervals\": [\n" + + " \"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"\n" + + " ],\n" + + " \"granularity\": \"day\"\n" + + "}"; + + QueryRequest request = QueryRequest.newBuilder() + .setQuery(query) + .setResultFormat(QueryResultFormat.CSV) + .addTimeColumns("__time") + .setQueryType(QueryOuterClass.QueryType.NATIVE) + .build(); + QueryResponse response = driver.submitQuery(request, CalciteTests.REGULAR_USER_AUTH_RESULT); + + assertEquals(QueryStatus.OK, response.getStatus()); + assertFalse(response.hasErrorMessage()); + assertTrue(response.getQueryId().length() > 5); + List columns = response.getColumnsList(); + assertEquals(3, columns.size()); + ColumnSchema col = columns.get(0); + assertEquals("__time", col.getName()); + assertEquals("", col.getSqlType()); + assertEquals(DruidType.LONG, col.getDruidType()); + col = columns.get(1); + assertEquals("countryName", col.getName()); + assertEquals("", col.getSqlType()); + assertEquals(DruidType.STRING, col.getDruidType()); + + col = columns.get(2); + assertEquals("sum", col.getName()); + assertEquals("", col.getSqlType()); + assertEquals(DruidType.LONG, col.getDruidType()); + + List expectedResults = Arrays.asList( + "2015-09-12T00:00:00.000Z,Albania,80", + "2015-09-12T00:00:00.000Z,Angola,784" + ); + String results = response.getData().toStringUtf8(); + assertEquals(expectedResults, Arrays.asList(results.split("\n"))); + } + + @Test + public void testBasics_native_timeseries() + { + String query = "{\n" + + " \"queryType\": \"timeseries\",\n" + + " \"dataSource\": \"foo\",\n" + + " \"granularity\": \"day\",\n" + + " \"descending\": \"true\",\n" + + " \"aggregations\": [\n" + + " {\n" + + " \"type\": \"longSum\",\n" + + " \"name\": \"timeseries\",\n" + + " \"fieldName\": \"m2\"\n" + + " }\n" + + " ],\n" + + " \"intervals\": [\n" + + " \"2000-01-01T00:00:00.000/2000-01-04T00:00:00.000\"\n" + + " ]\n" + + "}"; + + QueryRequest request = QueryRequest.newBuilder() + .setQuery(query) + .setResultFormat(QueryResultFormat.CSV) + .addTimeColumns("__time") + .setQueryType(QueryOuterClass.QueryType.NATIVE) + .build(); + QueryResponse response = driver.submitQuery(request, CalciteTests.REGULAR_USER_AUTH_RESULT); + + assertEquals(QueryStatus.OK, response.getStatus()); + + assertEquals(QueryStatus.OK, response.getStatus()); + assertFalse(response.hasErrorMessage()); + assertTrue(response.getQueryId().length() > 5); + List columns = response.getColumnsList(); + assertEquals(2, columns.size()); + ColumnSchema col = columns.get(0); + assertEquals("__time", col.getName()); + assertEquals("", col.getSqlType()); + assertEquals(DruidType.LONG, col.getDruidType()); + col = columns.get(1); + assertEquals("timeseries", col.getName()); + assertEquals("", col.getSqlType()); + assertEquals(DruidType.LONG, col.getDruidType()); + + List expectedResults = Arrays.asList( + "2000-01-03T00:00:00.000Z,3", + "2000-01-02T00:00:00.000Z,2", + "2000-01-01T00:00:00.000Z,1" + ); + String results = response.getData().toStringUtf8(); + assertEquals(expectedResults, Arrays.asList(results.split("\n"))); + } +} diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java new file mode 100644 index 00000000000..8b0129b0e42 --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTest.java @@ -0,0 +1,433 @@ +/* + * 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.grpc; + +import com.google.common.collect.ImmutableMap; +import io.grpc.StatusRuntimeException; +import org.apache.druid.grpc.client.GrpcResponseHandler; +import org.apache.druid.grpc.proto.HealthOuterClass.HealthCheckRequest; +import org.apache.druid.grpc.proto.HealthOuterClass.HealthCheckResponse; +import org.apache.druid.grpc.proto.QueryOuterClass; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.apache.druid.grpc.proto.TestResults; +import org.apache.druid.grpc.proto.TestResults.QueryResult; +import org.apache.druid.grpc.server.GrpcQueryConfig; +import org.apache.druid.grpc.server.QueryDriver; +import org.apache.druid.grpc.server.QueryServer; +import org.apache.druid.server.security.AllowAllAuthenticator; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthenticatorMapper; +import org.apache.druid.sql.calcite.BaseCalciteQueryTest; +import org.apache.druid.sql.calcite.util.SqlTestFramework; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.Executors; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +/** + * Simple test that runs the gRPC server, on top of a test SQL stack. + * Uses a simple client to send a query to the server. This is a basic + * sanity check of the gRPC stack. Uses allow-all security, which + * does a sanity check of the auth chain. + */ +public class GrpcQueryTest extends BaseCalciteQueryTest +{ + private static QueryServer server; + private static TestClient client; + + @BeforeEach + public void setup() throws IOException + { + SqlTestFramework sqlTestFramework = queryFramework(); + SqlTestFramework.PlannerFixture plannerFixture = sqlTestFramework.plannerFixture( + BaseCalciteQueryTest.PLANNER_CONFIG_DEFAULT, + new AuthConfig() + ); + + QueryDriver driver = new QueryDriver( + sqlTestFramework.queryJsonMapper(), + plannerFixture.statementFactory(), + sqlTestFramework.queryLifecycleFactory() + ); + AuthenticatorMapper authMapper = new AuthenticatorMapper( + ImmutableMap.of( + "test", + new AllowAllAuthenticator() + ) + ); + GrpcQueryConfig config = new GrpcQueryConfig(50051); + server = new QueryServer(config, driver, authMapper); + try { + server.start(); + } + catch (IOException | RuntimeException e) { + e.printStackTrace(); + throw e; + } + client = new TestClient(TestClient.DEFAULT_HOST); + } + + @AfterEach + public void tearDown() throws InterruptedException + { + if (client != null) { + client.close(); + } + if (server != null) { + server.stop(); + server.blockUntilShutdown(); + } + } + + /** + * Do a very basic query. + */ + @Test + public void testBasics_sql() + { + QueryRequest request = QueryRequest.newBuilder() + .setQuery("SELECT * FROM foo") + .setResultFormat(QueryResultFormat.CSV) + .setQueryType(QueryOuterClass.QueryType.SQL) + .build(); + + QueryResponse response = client.getQueryClient().submitQuery(request); + assertEquals(QueryStatus.OK, response.getStatus()); + } + + /** + * Do a very basic query that outputs protobuf. + */ + @Test + public void testGrpcBasics_sql() + { + QueryRequest request = QueryRequest.newBuilder() + .setQuery("SELECT dim1, dim2, dim3, cnt, m1, m2, unique_dim1, __time AS \"date\" FROM foo") + .setQueryType(QueryOuterClass.QueryType.SQL) + .setProtobufMessageName(QueryResult.class.getName()) + .setResultFormat(QueryResultFormat.PROTOBUF_INLINE) + .build(); + + QueryResponse response = client.getQueryClient().submitQuery(request); + GrpcResponseHandler handler = GrpcResponseHandler.of(QueryResult.class); + List queryResults = handler.get(response.getData()); + assertEquals(QueryStatus.OK, response.getStatus()); + assertEquals(6, queryResults.size()); + } + + @Test + public void testGrpcBasics_native_timeseries() + { + String query = "{\n" + + " \"queryType\": \"timeseries\",\n" + + " \"dataSource\": \"foo\",\n" + + " \"granularity\": \"day\",\n" + + " \"descending\": \"true\",\n" + + " \"aggregations\": [\n" + + " {\n" + + " \"type\": \"longSum\",\n" + + " \"name\": \"timeseries\",\n" + + " \"fieldName\": \"m2\"\n" + + " }\n" + + " ],\n" + + " \"intervals\": [\n" + + " \"2000-01-01T00:00:00.000/2000-01-05T00:00:00.000\"\n" + + " ]\n" + + "}"; + + QueryRequest request = QueryRequest.newBuilder() + .setQuery(query) + .setQueryType(QueryOuterClass.QueryType.NATIVE) + .addTimeColumns("__time") + .setProtobufMessageName(TestResults.NativeQueryResultTimeSeries.class.getName()) + .setResultFormat(QueryResultFormat.PROTOBUF_INLINE) + .build(); + + QueryResponse response = client.getQueryClient().submitQuery(request); + GrpcResponseHandler handler = GrpcResponseHandler.of(TestResults.NativeQueryResultTimeSeries.class); + List queryResults = handler.get(response.getData()); + assertEquals(QueryStatus.OK, response.getStatus()); + assertEquals(4, queryResults.size()); + + QueryRequest requestSkipTime = QueryRequest.newBuilder() + .setQuery(query) + .setQueryType(QueryOuterClass.QueryType.NATIVE) + .addSkipColumns("__time") + .setProtobufMessageName(TestResults.NativeQueryResultTimeSeriesSkipTime.class.getName()) + .setResultFormat(QueryResultFormat.PROTOBUF_INLINE) + .build(); + + QueryResponse responseSkipTime = client.getQueryClient().submitQuery(requestSkipTime); + GrpcResponseHandler handlerSkipTime = GrpcResponseHandler.of(TestResults.NativeQueryResultTimeSeriesSkipTime.class); + List queryResultsSkipTime = handlerSkipTime.get(responseSkipTime.getData()); + assertEquals(QueryStatus.OK, response.getStatus()); + assertEquals(4, queryResultsSkipTime.size()); + } + + @Test + public void testGrpcBasics_native_groupby_day_granularity() + { + String query = "{\n" + + " \"queryType\": \"groupBy\",\n" + + " \"dataSource\": \"wikipedia\",\n" + + " \"dimensions\": [\n" + + " {\n" + + " \"type\": \"default\",\n" + + " \"dimension\": \"countryName\",\n" + + " \"outputType\": \"STRING\"\n" + + " }\n" + + " ],\n" + + " \"aggregations\": [\n" + + " {\n" + + " \"type\": \"longSum\",\n" + + " \"name\": \"aggregate\",\n" + + " \"fieldName\": \"added\"\n" + + " }\n" + + " ],\n" + + " \"filter\": {\n" + + " \"type\": \"in\",\n" + + " \"dimension\": \"countryName\",\n" + + " \"values\": [\n" + + " \"Albania\",\n" + + " \"Angola\"\n" + + " ]\n" + + " }," + + " \"intervals\": [\n" + + " \"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"\n" + + " ],\n" + + " \"granularity\": \"day\"\n" + + "}"; + + QueryRequest request = QueryRequest.newBuilder() + .setQuery(query) + .setQueryType(QueryOuterClass.QueryType.NATIVE) + .putContext("timeFieldKey", "date") + .addTimeColumns("__time") + .setProtobufMessageName(TestResults.NativeQueryResultGroupby.class.getName()) + .setResultFormat(QueryResultFormat.PROTOBUF_INLINE) + .build(); + + QueryResponse response = client.getQueryClient().submitQuery(request); + GrpcResponseHandler handler = GrpcResponseHandler.of(TestResults.NativeQueryResultGroupby.class); + List queryResults = handler.get(response.getData()); + assertEquals(QueryStatus.OK, response.getStatus()); + assertEquals(2, queryResults.size()); + + } + + @Test + public void testGrpcBasics_native_groupby() + { + String query = "{\n" + + " \"queryType\": \"groupBy\",\n" + + " \"dataSource\": \"wikipedia\",\n" + + " \"dimensions\": [\n" + + " {\n" + + " \"type\": \"default\",\n" + + " \"dimension\": \"countryName\",\n" + + " \"outputType\": \"STRING\"\n" + + " }\n" + + " ],\n" + + " \"aggregations\": [\n" + + " {\n" + + " \"type\": \"longSum\",\n" + + " \"name\": \"aggregate\",\n" + + " \"fieldName\": \"added\"\n" + + " }\n" + + " ],\n" + + " \"filter\": {\n" + + " \"type\": \"in\",\n" + + " \"dimension\": \"countryName\",\n" + + " \"values\": [\n" + + " \"Albania\",\n" + + " \"Angola\"\n" + + " ]\n" + + " }," + + " \"intervals\": [\n" + + " \"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"\n" + + " ],\n" + + " \"granularity\": \"all\"\n" + + "}"; + + QueryRequest request = QueryRequest.newBuilder() + .setQuery(query) + .setQueryType(QueryOuterClass.QueryType.NATIVE) + .setProtobufMessageName(TestResults.NativeQueryResultGroupbyWithoutTime.class.getName()) + .setResultFormat(QueryResultFormat.PROTOBUF_INLINE) + .build(); + + QueryResponse response = client.getQueryClient().submitQuery(request); + GrpcResponseHandler handler = GrpcResponseHandler.of(TestResults.NativeQueryResultGroupbyWithoutTime.class); + List queryResults = handler.get(response.getData()); + assertEquals(QueryStatus.OK, response.getStatus()); + assertEquals(2, queryResults.size()); + } + + @Test + public void testGrpcBasics_native_groupby_timeaggregate() + { + String query = "{\n" + + " \"queryType\": \"groupBy\",\n" + + " \"dataSource\": \"wikipedia\",\n" + + " \"dimensions\": [\n" + + " {\n" + + " \"type\": \"default\",\n" + + " \"dimension\": \"countryName\",\n" + + " \"outputType\": \"STRING\"\n" + + " },\n" + + " {\n" + + " \"type\":\"default\",\n" + + " \"dimension\": \"__time\",\n" + + " \"outputName\": \"timeCol\",\n" + + " \"outputType\": \"LONG\"\n" + + " }" + + " ],\n" + + " \"aggregations\": [\n" + + " {\n" + + " \"type\": \"longSum\",\n" + + " \"name\": \"aggregate\",\n" + + " \"fieldName\": \"added\"\n" + + " }\n" + + " ],\n" + + " \"filter\": {\n" + + " \"type\": \"in\",\n" + + " \"dimension\": \"countryName\",\n" + + " \"values\": [\n" + + " \"Albania\",\n" + + " \"Angola\"\n" + + " ]\n" + + " }," + + " \"intervals\": [\n" + + " \"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"\n" + + " ],\n" + + " \"granularity\": \"day\"\n" + + "}"; + + QueryRequest request = QueryRequest.newBuilder() + .setQuery(query) + .setQueryType(QueryOuterClass.QueryType.NATIVE) + .addSkipColumns("__time") + .addTimeColumns("timeCol") + .setProtobufMessageName(TestResults.NativeQueryResultGroupbyTimeRenamed.class.getName()) + .setResultFormat(QueryResultFormat.PROTOBUF_INLINE) + .build(); + + QueryResponse response = client.getQueryClient().submitQuery(request); + GrpcResponseHandler handler = GrpcResponseHandler.of(TestResults.NativeQueryResultGroupbyTimeRenamed.class); + List queryResults = handler.get(response.getData()); + assertEquals(QueryStatus.OK, response.getStatus()); + assertEquals(6, queryResults.size()); + } + + @Test + public void testGrpcEmptyResponse_sql() + { + QueryRequest request = QueryRequest.newBuilder() + .setQuery("SELECT dim1, dim2, dim3, cnt, m1, m2, unique_dim1, __time AS \"date\" FROM foo where cnt = 100000") + .setProtobufMessageName(QueryResult.class.getName()) + .setResultFormat(QueryResultFormat.PROTOBUF_INLINE) + .setQueryType(QueryOuterClass.QueryType.SQL) + .build(); + + QueryResponse response = client.getQueryClient().submitQuery(request); + GrpcResponseHandler handler = GrpcResponseHandler.of(QueryResult.class); + List queryResults = handler.get(response.getData()); + assertEquals(QueryStatus.OK, response.getStatus()); + assertEquals(0, queryResults.size()); + } + + @Test + public void test_health_check() + { + HealthCheckRequest healthCheckRequest = HealthCheckRequest + .newBuilder() + .setService("QueryService") + .build(); + + HealthCheckResponse healthCheckResponse = client + .getHealthCheckClient() + .check(healthCheckRequest); + + assertEquals(HealthCheckResponse.ServingStatus.SERVING, healthCheckResponse.getStatus()); + + healthCheckRequest = HealthCheckRequest + .newBuilder() + .setService("") + .build(); + + healthCheckResponse = client + .getHealthCheckClient() + .check(healthCheckRequest); + + assertEquals(HealthCheckResponse.ServingStatus.SERVING, healthCheckResponse.getStatus()); + } + + @Test + public void test_health_watch() + { + HealthCheckRequest healthCheckRequest = HealthCheckRequest + .newBuilder() + .setService("QueryService") + .build(); + + Iterator streamingHealthCheckResponse = client + .getHealthCheckClient() + .watch(healthCheckRequest); + + assertTrue(streamingHealthCheckResponse.hasNext()); + assertEquals(HealthCheckResponse.ServingStatus.SERVING, streamingHealthCheckResponse.next().getStatus()); + + Executors.newSingleThreadExecutor().submit(() -> { + Iterator secondRequest = client + .getHealthCheckClient() + .watch(healthCheckRequest); + + assertThrows(StatusRuntimeException.class, secondRequest::hasNext); + }); + + // stop the service from another thread + Executors.newSingleThreadExecutor().submit(() -> { + try { + Thread.sleep(10_000); + server.stop(); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + + // hasNext call would block until the status has changed + // as soon as the server is stopped the status would change to NOT_SERVING + if (streamingHealthCheckResponse.hasNext()) { + assertEquals(HealthCheckResponse.ServingStatus.NOT_SERVING, streamingHealthCheckResponse.next().getStatus()); + } + } +} diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTestClient.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTestClient.java new file mode 100644 index 00000000000..aecded9f4b6 --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/GrpcQueryTestClient.java @@ -0,0 +1,68 @@ +/* + * 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.grpc; + +import org.apache.druid.grpc.proto.QueryOuterClass.ColumnSchema; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryRequest; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResultFormat; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryStatus; +import org.apache.druid.java.util.common.StringUtils; + +import java.nio.charset.StandardCharsets; + +/** + * Super-simple command-line (or IDE launched) client which makes a + * single query request and prints + * the response. Useful because Druid provides no other rRPC client + * to use to test the rRPC endpoint. Pass the desired query as the + * one and only command line parameter. Does not (yet) support the + * query context or query parameters. + */ +public class GrpcQueryTestClient +{ + public static void main(String[] args) + { + if (args.length != 1) { + System.err.println("Usage: sql-query"); + System.exit(1); + } + TestClient client = new TestClient(TestClient.DEFAULT_HOST); + QueryRequest request = QueryRequest.newBuilder() + .setQuery(args[0]) + .setResultFormat(QueryResultFormat.CSV) + .build(); + QueryResponse response = client.getQueryClient().submitQuery(request); + + + if (response.getStatus() != QueryStatus.OK) { + System.err.println("Failed: " + response.getStatus().name()); + System.err.println(response.getErrorMessage()); + System.exit(1); + } + System.out.println("Columns:"); + for (ColumnSchema col : response.getColumnsList()) { + System.out.println(StringUtils.format("%s %s (%s)", col.getName(), col.getSqlType(), col.getDruidType().name())); + } + System.out.println("Data:"); + System.out.println(response.getData().toString(StandardCharsets.UTF_8)); + client.close(); + } +} diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestClient.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestClient.java new file mode 100644 index 00000000000..96b9cf104ed --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestClient.java @@ -0,0 +1,100 @@ +/* + * 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.grpc; + +import io.grpc.CallCredentials; +import io.grpc.Grpc; +import io.grpc.InsecureChannelCredentials; +import io.grpc.ManagedChannel; +import org.apache.druid.grpc.proto.HealthGrpc; +import org.apache.druid.grpc.proto.QueryGrpc; +import org.apache.druid.grpc.proto.QueryGrpc.QueryBlockingStub; + +import java.util.concurrent.TimeUnit; + +/** + * Super-simple test client that connects to a gRPC query endpoint + * and allows submitting a rRPC query request and returns the response. + * The server can be in the same or another process. + */ +public class TestClient implements AutoCloseable +{ + public static final String DEFAULT_HOST = "localhost:50051"; + private final ManagedChannel channel; + private QueryBlockingStub queryClient; + private HealthGrpc.HealthBlockingStub healthCheckClient; + + public TestClient(String target) + { + // Access a service running on the local machine on port 50051 + this(target, null); + } + + public TestClient(String target, String user, String password) + { + this(target, new BasicCredentials(user, password)); + } + + public TestClient(String target, CallCredentials callCreds) + { + // Create a communication channel to the server, known as a Channel. Channels are thread-safe + // and reusable. It is common to create channels at the beginning of your application and reuse + // them until the application shuts down. + // + // For the example we use plaintext insecure credentials to avoid needing TLS certificates. To + // use TLS, use TlsChannelCredentials instead. + channel = Grpc.newChannelBuilder(target, InsecureChannelCredentials.create()) + .build(); + queryClient = QueryGrpc.newBlockingStub(channel); + healthCheckClient = HealthGrpc.newBlockingStub(channel); + if (callCreds != null) { + queryClient = queryClient.withCallCredentials(callCreds); + } + } + + public QueryBlockingStub getQueryClient() + { + return queryClient; + } + + public HealthGrpc.HealthBlockingStub getHealthCheckClient() + { + return healthCheckClient; + } + + public QueryBlockingStub client() + { + return queryClient; + } + + @Override + public void close() + { + // ManagedChannels use resources like threads and TCP connections. To prevent leaking these + // resources the channel should be shut down when it will no longer be used. If it may be used + // again leave it running. + try { + channel.shutdownNow().awaitTermination(5, TimeUnit.SECONDS); + } + catch (InterruptedException e) { + // Ignore + } + } +} diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestServer.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestServer.java new file mode 100644 index 00000000000..b25d2e2ea53 --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/TestServer.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.grpc; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.grpc.server.GrpcEndpointInitializer; +import org.apache.druid.grpc.server.GrpcQueryConfig; +import org.apache.druid.server.security.AllowAllAuthenticator; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthenticatorMapper; +import org.apache.druid.sql.calcite.BaseCalciteQueryTest; +import org.apache.druid.sql.calcite.util.SqlTestFramework; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +/** + * Super-simple test server that uses the CalciteTests setup. + */ +public class TestServer extends BaseCalciteQueryTest +{ + private GrpcEndpointInitializer serverInit; + + @Test + @Disabled + public void run() + { + SqlTestFramework sqlTestFramework = queryFramework(); + SqlTestFramework.PlannerFixture plannerFixture = sqlTestFramework.plannerFixture( + BaseCalciteQueryTest.PLANNER_CONFIG_DEFAULT, + new AuthConfig() + ); + GrpcQueryConfig config = new GrpcQueryConfig(50051); + AuthenticatorMapper authMapper = new AuthenticatorMapper( + ImmutableMap.of( + "test", + new AllowAllAuthenticator() + ) + ); + serverInit = new GrpcEndpointInitializer( + config, + sqlTestFramework.queryJsonMapper(), + plannerFixture.statementFactory(), + null, + authMapper + ); + serverInit.start(); + Runtime.getRuntime().addShutdownHook(new Thread() + { + @Override + public void run() + { + // Use stderr here since the logger may have been reset by its JVM shutdown hook. + System.err.println("*** shutting down gRPC server since JVM is shutting down"); + serverInit.stop(); + } + }); + } +} diff --git a/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/client/GrpcResponseHandlerTest.java b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/client/GrpcResponseHandlerTest.java new file mode 100644 index 00000000000..1f3ba79814c --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/java/org/apache/druid/grpc/client/GrpcResponseHandlerTest.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.grpc.client; + +import com.google.protobuf.ByteString; +import org.apache.druid.grpc.proto.QueryOuterClass.QueryResponse; +import org.apache.druid.grpc.proto.TestResults.QueryResult; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class GrpcResponseHandlerTest +{ + private static List EXPECTED_RESULTS = Arrays.asList( + QueryResult.newBuilder().setDim1("test").setCnt(100).build(), + QueryResult.newBuilder().setDim2("test2").setCnt(100).setM2(200.10).build() + ); + + @Test + public void testEmptyResponse() + { + GrpcResponseHandler handler = GrpcResponseHandler.of(QueryResult.class); + List queryResults = handler.get(ByteString.EMPTY); + assertTrue(queryResults.isEmpty()); + } + + @Test + public void testNonEmptyResponse() + { + GrpcResponseHandler handler = GrpcResponseHandler.of(QueryResult.class); + QueryResponse queryResponse = getQueryResponse(); + List queryResults = handler.get(queryResponse.getData()); + assertEquals(2, queryResults.size()); + assertEquals(EXPECTED_RESULTS, queryResults); + } + + private static QueryResponse getQueryResponse() + { + try { + final ByteArrayOutputStream out = new ByteArrayOutputStream(); + for (QueryResult queryResult : EXPECTED_RESULTS) { + queryResult.writeDelimitedTo(out); + } + return QueryResponse.newBuilder() + .setData(ByteString.copyFrom(out.toByteArray())) + .build(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/extensions-contrib/grpc-query/src/test/proto/all_types.proto b/extensions-contrib/grpc-query/src/test/proto/all_types.proto new file mode 100644 index 00000000000..e5438b06c2e --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/proto/all_types.proto @@ -0,0 +1,31 @@ +// 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. + +syntax = "proto3"; + +import "google/protobuf/timestamp.proto"; + +package org.apache.druid.grpc.results; +option java_package = "org.apache.druid.grpc.proto"; + +message AllTypesQueryResult { + google.protobuf.Timestamp time_value = 1; + string string_value = 2; + int64 long_value = 3; + // Use of a 'float' type will cause a runtime error. SQL 'FLOAT' + // types are actually stored as 'double' internally. + double float_value = 4; + double double_value = 5; +} diff --git a/extensions-contrib/grpc-query/src/test/proto/test_results.proto b/extensions-contrib/grpc-query/src/test/proto/test_results.proto new file mode 100644 index 00000000000..5f3ed16198c --- /dev/null +++ b/extensions-contrib/grpc-query/src/test/proto/test_results.proto @@ -0,0 +1,58 @@ +// 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. + +syntax = "proto3"; + +import "google/protobuf/timestamp.proto"; + +package org.apache.druid.grpc.results; +option java_package = "org.apache.druid.grpc.proto"; + +message QueryResult { + string dim1 = 1; + string dim2 = 2; + string dim3 = 3; + int64 cnt = 4; + float m1 = 5; + double m2 = 6; + bytes unique_dim1 = 7; + google.protobuf.Timestamp date = 8; +} + +message NativeQueryResultTimeSeries { + int64 timeseries = 1; + google.protobuf.Timestamp time = 2; +} + +message NativeQueryResultTimeSeriesSkipTime { + int64 timeseries = 1; +} + +message NativeQueryResultGroupby { + int64 aggregate = 1; + string countryName = 2; + google.protobuf.Timestamp date = 3; +} + +message NativeQueryResultGroupbyTimeRenamed { + int64 aggregate = 1; + string countryName = 2; + google.protobuf.Timestamp timeCol = 3; +} + +message NativeQueryResultGroupbyWithoutTime { + int64 aggregate = 1; + string countryName = 2; +} diff --git a/extensions-contrib/influx-extensions/pom.xml b/extensions-contrib/influx-extensions/pom.xml index a63aa1d8396..829d4e50d4e 100644 --- a/extensions-contrib/influx-extensions/pom.xml +++ b/extensions-contrib/influx-extensions/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/influxdb-emitter/pom.xml b/extensions-contrib/influxdb-emitter/pom.xml index e51a6d3509c..802ce4b8896 100644 --- a/extensions-contrib/influxdb-emitter/pom.xml +++ b/extensions-contrib/influxdb-emitter/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/kafka-emitter/pom.xml b/extensions-contrib/kafka-emitter/pom.xml index 3918c6974f4..8f94fe954bd 100644 --- a/extensions-contrib/kafka-emitter/pom.xml +++ b/extensions-contrib/kafka-emitter/pom.xml @@ -24,7 +24,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/kubernetes-overlord-extensions/pom.xml b/extensions-contrib/kubernetes-overlord-extensions/pom.xml index be40224054d..90c29362e64 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/pom.xml +++ b/extensions-contrib/kubernetes-overlord-extensions/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java index 568f8ed5a11..644a7f109b2 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java @@ -37,6 +37,7 @@ public class DruidK8sConstants public static final String TASK_JSON_ENV = "TASK_JSON"; public static final String TASK_DIR_ENV = "TASK_DIR"; public static final String TASK_ID_ENV = "TASK_ID"; + public static final String LOAD_BROADCAST_DATASOURCE_MODE_ENV = "LOAD_BROADCAST_DATASOURCE_MODE"; public static final String LOAD_BROADCAST_SEGMENTS_ENV = "LOAD_BROADCAST_SEGMENTS"; public static final String JAVA_OPTS = "JAVA_OPTS"; public static final String DRUID_HOST_ENV = "druid_host"; diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java index c15698803d9..cc689f925f4 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapter.java @@ -444,12 +444,16 @@ public abstract class K8sTaskAdapter implements TaskAdapter } // If the task type is queryable, we need to load broadcast segments on the peon, used for - // join queries + // join queries. This is replaced by --loadBroadcastDatasourceMode option, but is preserved here + // for backwards compatibility and can be removed in a future release. if (task.supportsQueries()) { command.add("--loadBroadcastSegments"); command.add("true"); } + command.add("--loadBroadcastDatasourceMode"); + command.add(task.getBroadcastDatasourceLoadingSpec().getMode().toString()); + command.add("--taskId"); command.add(task.getId()); log.info( diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index e8aaf1bbab3..321fe3fcb3e 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -280,6 +280,10 @@ public class PodTemplateTaskAdapter implements TaskAdapter .withName(DruidK8sConstants.TASK_ID_ENV) .withValue(task.getId()) .build(), + new EnvVarBuilder() + .withName(DruidK8sConstants.LOAD_BROADCAST_DATASOURCE_MODE_ENV) + .withValue(task.getBroadcastDatasourceLoadingSpec().getMode().toString()) + .build(), new EnvVarBuilder() .withName(DruidK8sConstants.LOAD_BROADCAST_SEGMENTS_ENV) .withValue(Boolean.toString(task.supportsQueries())) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java index b3fda99b222..161e8056952 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java @@ -34,7 +34,6 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.DataSchema; @@ -66,18 +65,19 @@ public class K8sTestUtils null, null, new IndexTask.IndexIngestionSpec( - new DataSchema( - "foo", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P2D")) - ), - null - ), + DataSchema.builder() + .withDataSource("foo") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P2D")) + ) + ) + .build(), new IndexTask.IndexIOConfig( new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index ac2aaa70558..b25f23a25dd 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -46,6 +46,7 @@ import org.apache.druid.k8s.overlord.execution.KubernetesTaskRunnerDynamicConfig import org.apache.druid.k8s.overlord.execution.Selector; import org.apache.druid.k8s.overlord.execution.SelectorBasedPodTemplateSelectStrategy; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.coordination.BroadcastDatasourceLoadingSpec; import org.apache.druid.tasklogs.TaskLogs; import org.easymock.EasyMock; import org.junit.Assert; @@ -537,6 +538,7 @@ public class PodTemplateTaskAdapterTest EasyMock.expect(task.getId()).andReturn("id").anyTimes(); EasyMock.expect(task.getGroupId()).andReturn("groupid").anyTimes(); EasyMock.expect(task.getDataSource()).andReturn("datasource").anyTimes(); + EasyMock.expect(task.getBroadcastDatasourceLoadingSpec()).andReturn(BroadcastDatasourceLoadingSpec.ALL).anyTimes(); EasyMock.replay(task); Job actual = adapter.fromTask(task); @@ -550,7 +552,46 @@ public class PodTemplateTaskAdapterTest } @Test - public void test_fromTask_withIndexKafkaPodTemplateInRuntimeProperites() throws IOException + public void test_fromTask_withBroadcastDatasourceLoadingModeAll() throws IOException + { + Path templatePath = Files.createFile(tempDir.resolve("noop.yaml")); + mapper.writeValue(templatePath.toFile(), podTemplateSpec); + + Properties props = new Properties(); + props.setProperty("druid.indexer.runner.k8s.podTemplate.base", templatePath.toString()); + props.setProperty("druid.indexer.runner.k8s.podTemplate.queryable", templatePath.toString()); + + PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( + taskRunnerConfig, + taskConfig, + node, + mapper, + props, + taskLogs, + dynamicConfigRef + ); + + Task task = EasyMock.mock(Task.class); + EasyMock.expect(task.supportsQueries()).andReturn(true); + EasyMock.expect(task.getType()).andReturn("queryable").anyTimes(); + EasyMock.expect(task.getId()).andReturn("id").anyTimes(); + EasyMock.expect(task.getGroupId()).andReturn("groupid").anyTimes(); + EasyMock.expect(task.getDataSource()).andReturn("datasource").anyTimes(); + EasyMock.expect(task.getBroadcastDatasourceLoadingSpec()).andReturn(BroadcastDatasourceLoadingSpec.ALL).anyTimes(); + + EasyMock.replay(task); + Job actual = adapter.fromTask(task); + EasyMock.verify(task); + + Assertions.assertEquals(BroadcastDatasourceLoadingSpec.Mode.ALL.toString(), actual.getSpec().getTemplate() + .getSpec().getContainers() + .get(0).getEnv().stream() + .filter(env -> env.getName().equals(DruidK8sConstants.LOAD_BROADCAST_DATASOURCE_MODE_ENV)) + .collect(Collectors.toList()).get(0).getValue()); + } + + @Test + public void test_fromTask_withIndexKafkaPodTemplateInRuntimeProperties() throws IOException { Path baseTemplatePath = Files.createFile(tempDir.resolve("base.yaml")); mapper.writeValue(baseTemplatePath.toFile(), podTemplateSpec); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml index ddae7c0567f..ac539c5da15 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml @@ -45,6 +45,8 @@ spec: value: "/tmp" - name: "TASK_ID" value: "id" + - name: "LOAD_BROADCAST_DATASOURCE_MODE" + value: "ALL" - name: "LOAD_BROADCAST_SEGMENTS" value: "false" - name: "TASK_JSON" diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBase.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBase.yaml index 532c3dd53e8..f7c2ff958bb 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBase.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBase.yaml @@ -45,6 +45,8 @@ spec: value: "/tmp" - name: "TASK_ID" value: "id" + - name: "LOAD_BROADCAST_DATASOURCE_MODE" + value: "ALL" - name: "LOAD_BROADCAST_SEGMENTS" value: "false" - name: "TASK_JSON" diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml index d6c316dcdde..3a3af1528b5 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml @@ -44,6 +44,8 @@ spec: value: "/tmp" - name: "TASK_ID" value: "api-issued_kill_wikipedia3_omjobnbc_1000-01-01T00:00:00.000Z_2023-05-14T00:00:00.000Z_2023-05-15T17:03:01.220Z" + - name: "LOAD_BROADCAST_DATASOURCE_MODE" + value: "ALL" - name: "LOAD_BROADCAST_SEGMENTS" value: "false" - name: "TASK_JSON" diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml index 90ae9970959..ec7f9a06248 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml @@ -43,6 +43,8 @@ spec: value: "/tmp" - name: "TASK_ID" value: "id" + - name: "LOAD_BROADCAST_DATASOURCE_MODE" + value: "ALL" - name: "LOAD_BROADCAST_SEGMENTS" value: "false" image: one diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabledBase.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabledBase.yaml index 0e52beac9e3..84457fb3175 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabledBase.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabledBase.yaml @@ -44,6 +44,8 @@ spec: value: "/tmp" - name: "TASK_ID" value: "id" + - name: "LOAD_BROADCAST_DATASOURCE_MODE" + value: "ALL" - name: "LOAD_BROADCAST_SEGMENTS" value: "false" - name: "TASK_JSON" diff --git a/extensions-contrib/materialized-view-maintenance/pom.xml b/extensions-contrib/materialized-view-maintenance/pom.xml index 61695919420..0d02fa81574 100644 --- a/extensions-contrib/materialized-view-maintenance/pom.xml +++ b/extensions-contrib/materialized-view-maintenance/pom.xml @@ -22,7 +22,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java index 01039375259..b23af62f630 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java @@ -48,7 +48,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.realtime.ChatHandlerProvider; -import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -211,14 +210,13 @@ public class MaterializedViewSupervisorSpec implements SupervisorSpec ); // generate DataSchema - DataSchema dataSchema = new DataSchema( - dataSourceName, - parser, - aggregators, - granularitySpec, - TransformSpec.NONE, - objectMapper - ); + DataSchema dataSchema = DataSchema.builder() + .withDataSource(dataSourceName) + .withParserMap(parser) + .withAggregators(aggregators) + .withGranularity(granularitySpec) + .withObjectMapper(objectMapper) + .build(); // generate DatasourceIngestionSpec DatasourceIngestionSpec datasourceIngestionSpec = new DatasourceIngestionSpec( diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index ff175919586..b87ff6be14c 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -53,7 +53,6 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.ChatHandlerProvider; -import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; @@ -237,14 +236,10 @@ public class MaterializedViewSupervisorTest Map runningTasks = runningTasksPair.lhs; Map runningVersion = runningTasksPair.rhs; - DataSchema dataSchema = new DataSchema( - "test_datasource", - null, - null, - null, - TransformSpec.NONE, - objectMapper - ); + DataSchema dataSchema = DataSchema.builder() + .withDataSource("test_datasource") + .withObjectMapper(objectMapper) + .build(); HadoopIOConfig hadoopIOConfig = new HadoopIOConfig(new HashMap<>(), null, null); HadoopIngestionSpec spec = new HadoopIngestionSpec(dataSchema, hadoopIOConfig, null); HadoopIndexTask task1 = new HadoopIndexTask( diff --git a/extensions-contrib/materialized-view-selection/pom.xml b/extensions-contrib/materialized-view-selection/pom.xml index d684e2ce404..4e44bfde5c6 100644 --- a/extensions-contrib/materialized-view-selection/pom.xml +++ b/extensions-contrib/materialized-view-selection/pom.xml @@ -22,7 +22,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/momentsketch/pom.xml b/extensions-contrib/momentsketch/pom.xml index c38fa5b5583..ca0a63ddc52 100644 --- a/extensions-contrib/momentsketch/pom.xml +++ b/extensions-contrib/momentsketch/pom.xml @@ -22,7 +22,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/moving-average-query/pom.xml b/extensions-contrib/moving-average-query/pom.xml index 10b3ecbd1e2..11929663456 100644 --- a/extensions-contrib/moving-average-query/pom.xml +++ b/extensions-contrib/moving-average-query/pom.xml @@ -24,7 +24,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/opentelemetry-emitter/pom.xml b/extensions-contrib/opentelemetry-emitter/pom.xml index 71a67270966..e785c27a8f4 100644 --- a/extensions-contrib/opentelemetry-emitter/pom.xml +++ b/extensions-contrib/opentelemetry-emitter/pom.xml @@ -23,7 +23,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/opentsdb-emitter/pom.xml b/extensions-contrib/opentsdb-emitter/pom.xml index 6564389328b..eb1773effd9 100644 --- a/extensions-contrib/opentsdb-emitter/pom.xml +++ b/extensions-contrib/opentsdb-emitter/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/prometheus-emitter/pom.xml b/extensions-contrib/prometheus-emitter/pom.xml index 70119c5b523..1a6eb767d97 100644 --- a/extensions-contrib/prometheus-emitter/pom.xml +++ b/extensions-contrib/prometheus-emitter/pom.xml @@ -23,7 +23,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/rabbit-stream-indexing-service/pom.xml b/extensions-contrib/rabbit-stream-indexing-service/pom.xml index 387f402bf32..8e15a592163 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/pom.xml +++ b/extensions-contrib/rabbit-stream-indexing-service/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfig.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfig.java index 0086084c570..288e28966a0 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfig.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfig.java @@ -74,7 +74,9 @@ public class RabbitStreamIndexTaskTuningConfig extends SeekableStreamIndexTaskTu @Nullable Integer numPersistThreads, @Nullable Integer recordBufferSize, @Nullable Integer recordBufferOfferTimeout, - @Nullable Integer maxRecordsPerPoll) + @Nullable Integer maxRecordsPerPoll, + @Nullable Integer maxColumnsToMerge + ) { super( appendableIndexSpec, @@ -97,7 +99,8 @@ public class RabbitStreamIndexTaskTuningConfig extends SeekableStreamIndexTaskTu logParseExceptions, maxParseExceptions, maxSavedParseExceptions, - numPersistThreads + numPersistThreads, + maxColumnsToMerge ); this.recordBufferSize = recordBufferSize; @@ -130,7 +133,8 @@ public class RabbitStreamIndexTaskTuningConfig extends SeekableStreamIndexTaskTu @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads, - @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll + @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll, + @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge ) { this( @@ -156,7 +160,9 @@ public class RabbitStreamIndexTaskTuningConfig extends SeekableStreamIndexTaskTu numPersistThreads, recordBufferSize, recordBufferOfferTimeout, - maxRecordsPerPoll); + maxRecordsPerPoll, + maxColumnsToMerge + ); } @Nullable @@ -226,7 +232,8 @@ public class RabbitStreamIndexTaskTuningConfig extends SeekableStreamIndexTaskTu getNumPersistThreads(), getRecordBufferSizeConfigured(), getRecordBufferOfferTimeout(), - getMaxRecordsPerPollConfigured() + getMaxRecordsPerPollConfigured(), + getMaxColumnsToMerge() ); } @@ -253,6 +260,7 @@ public class RabbitStreamIndexTaskTuningConfig extends SeekableStreamIndexTaskTu ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + ", numPersistThreads=" + getNumPersistThreads() + ", maxRecordsPerPole=" + getMaxRecordsPerPollConfigured() + + ", maxColumnsToMerge=" + getMaxColumnsToMerge() + '}'; } diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java index c6db20eed91..a2667026fff 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java @@ -68,6 +68,7 @@ public class RabbitStreamSupervisorTuningConfig extends RabbitStreamIndexTaskTun null, null, null, + null, null ); } @@ -99,7 +100,9 @@ public class RabbitStreamSupervisorTuningConfig extends RabbitStreamIndexTaskTun @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads, @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, - @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll) + @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll, + @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge + ) { super( appendableIndexSpec, @@ -124,7 +127,8 @@ public class RabbitStreamSupervisorTuningConfig extends RabbitStreamIndexTaskTun numPersistThreads, recordBufferSize, recordBufferOfferTimeout, - maxRecordsPerPoll + maxRecordsPerPoll, + maxColumnsToMerge ); this.workerThreads = workerThreads; this.chatRetries = (chatRetries != null ? chatRetries : DEFAULT_CHAT_RETRIES); @@ -210,6 +214,7 @@ public class RabbitStreamSupervisorTuningConfig extends RabbitStreamIndexTaskTun ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + ", numPersistThreads=" + getNumPersistThreads() + ", maxRecordsPerPoll=" + getMaxRecordsPerPollConfigured() + + ", maxColumnsToMerge=" + getMaxColumnsToMerge() + '}'; } @@ -239,7 +244,8 @@ public class RabbitStreamSupervisorTuningConfig extends RabbitStreamIndexTaskTun getRecordBufferSizeConfigured(), getRecordBufferOfferTimeout(), getMaxRecordsPerPollConfigured(), - getNumPersistThreads() + getNumPersistThreads(), + getMaxColumnsToMerge() ); } } diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java index 193b35b8af1..d507199495a 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java @@ -180,7 +180,8 @@ public class RabbitStreamIndexTaskTuningConfigTest "maxParseExceptions=0, " + "maxSavedParseExceptions=0, " + "numPersistThreads=1, " + - "maxRecordsPerPoll=null}"; + "maxRecordsPerPoll=null, " + + "maxColumnsToMerge=-1}"; Assert.assertEquals(resStr, config.toString()); diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java index 5f31d70b75d..e52ca2f29b6 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; @@ -44,7 +43,6 @@ import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.StubServiceEmitter; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -102,16 +100,19 @@ public class RabbitStreamSupervisorTest extends EasyMockSupport dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim2")); - return new DataSchema( - dataSource, - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(dimensions), - new AggregatorFactory[] {new CountAggregatorFactory("rows")}, - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.NONE, - ImmutableList.of()), - null); + return DataSchema.builder() + .withDataSource(dataSource) + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions(dimensions) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity( + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of() + ) + ) + .build(); } @BeforeClass @@ -158,7 +159,9 @@ public class RabbitStreamSupervisorTest extends EasyMockSupport null, null, null, - 100); + 100, + null + ); rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); serviceEmitter = new StubServiceEmitter("RabbitStreamSupervisorTest", "localhost"); EmittingLogger.registerEmitter(serviceEmitter); diff --git a/extensions-contrib/redis-cache/pom.xml b/extensions-contrib/redis-cache/pom.xml index 281f5372345..092ca366dd5 100644 --- a/extensions-contrib/redis-cache/pom.xml +++ b/extensions-contrib/redis-cache/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/spectator-histogram/pom.xml b/extensions-contrib/spectator-histogram/pom.xml index fd6fff411aa..854b5630940 100644 --- a/extensions-contrib/spectator-histogram/pom.xml +++ b/extensions-contrib/spectator-histogram/pom.xml @@ -24,7 +24,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/sqlserver-metadata-storage/pom.xml b/extensions-contrib/sqlserver-metadata-storage/pom.xml index 72ad3e617bb..1fd186f9dc8 100644 --- a/extensions-contrib/sqlserver-metadata-storage/pom.xml +++ b/extensions-contrib/sqlserver-metadata-storage/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/statsd-emitter/pom.xml b/extensions-contrib/statsd-emitter/pom.xml index b3c96118bf9..952f9a41fbf 100644 --- a/extensions-contrib/statsd-emitter/pom.xml +++ b/extensions-contrib/statsd-emitter/pom.xml @@ -21,7 +21,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/tdigestsketch/pom.xml b/extensions-contrib/tdigestsketch/pom.xml index 4c192d62746..896a611b7f4 100644 --- a/extensions-contrib/tdigestsketch/pom.xml +++ b/extensions-contrib/tdigestsketch/pom.xml @@ -22,7 +22,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/thrift-extensions/pom.xml b/extensions-contrib/thrift-extensions/pom.xml index 25189c25999..064def57bde 100644 --- a/extensions-contrib/thrift-extensions/pom.xml +++ b/extensions-contrib/thrift-extensions/pom.xml @@ -28,7 +28,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/time-min-max/pom.xml b/extensions-contrib/time-min-max/pom.xml index 68dd27ffda6..fbc91e33ecd 100644 --- a/extensions-contrib/time-min-max/pom.xml +++ b/extensions-contrib/time-min-max/pom.xml @@ -21,7 +21,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-contrib/virtual-columns/pom.xml b/extensions-contrib/virtual-columns/pom.xml index d70902d5f42..2d1fe62ed50 100644 --- a/extensions-contrib/virtual-columns/pom.xml +++ b/extensions-contrib/virtual-columns/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java index 9271b9b3e98..c1fed4bc503 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java @@ -99,8 +99,7 @@ public class MapVirtualColumnGroupByTest extends InitializedNullHandlingTest return 1; } }, - () -> config, - new StupidPool<>("map-virtual-column-groupby-test", () -> ByteBuffer.allocate(1024)), + GroupByQueryConfig::new, groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new DefaultObjectMapper(), @@ -109,7 +108,8 @@ public class MapVirtualColumnGroupByTest extends InitializedNullHandlingTest final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( groupingEngine, - new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool) + new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool), + new StupidPool<>("map-virtual-column-groupby-test", () -> ByteBuffer.allocate(1024)) ); runner = QueryRunnerTestHelper.makeQueryRunner( diff --git a/extensions-core/avro-extensions/pom.xml b/extensions-core/avro-extensions/pom.xml index 70f526454a3..01b5b470e90 100644 --- a/extensions-core/avro-extensions/pom.xml +++ b/extensions-core/avro-extensions/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/azure-extensions/pom.xml b/extensions-core/azure-extensions/pom.xml index 7f0f767af85..b8a7d5ace75 100644 --- a/extensions-core/azure-extensions/pom.xml +++ b/extensions-core/azure-extensions/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml index 75ff5a0868e..ba53b7d8867 100644 --- a/extensions-core/datasketches/pom.xml +++ b/extensions-core/datasketches/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java index b69b1fe8861..2fea6774e89 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactoryTest.java @@ -26,10 +26,10 @@ import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.vector.VectorCursor; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -45,13 +45,13 @@ import java.util.function.Supplier; public class ToObjectVectorColumnProcessorFactoryTest extends InitializedNullHandlingTest { - private StorageAdapter adapter; + private CursorFactory cursorFactory; @Before public void setUp() { final QueryableIndex index = TestIndex.getMMappedTestIndex(); - adapter = new QueryableIndexStorageAdapter(index); + cursorFactory = new QueryableIndexCursorFactory(index); } @Test @@ -180,7 +180,7 @@ public class ToObjectVectorColumnProcessorFactoryTest extends InitializedNullHan ) ) .build(); - return adapter.makeCursorHolder(buildSpec); + return cursorFactory.makeCursorHolder(buildSpec); } private List readColumn(final String column, final int limit) diff --git a/extensions-core/druid-aws-rds-extensions/pom.xml b/extensions-core/druid-aws-rds-extensions/pom.xml index d32411106b9..26322f93c96 100644 --- a/extensions-core/druid-aws-rds-extensions/pom.xml +++ b/extensions-core/druid-aws-rds-extensions/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/druid-basic-security/pom.xml b/extensions-core/druid-basic-security/pom.xml index a4dbda9b94a..f2b5df6c4b2 100644 --- a/extensions-core/druid-basic-security/pom.xml +++ b/extensions-core/druid-basic-security/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/druid-bloom-filter/pom.xml b/extensions-core/druid-bloom-filter/pom.xml index b57ede7b16b..9cf52315651 100644 --- a/extensions-core/druid-bloom-filter/pom.xml +++ b/extensions-core/druid-bloom-filter/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java index 3c59bfc366e..9184d2b14fe 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java @@ -37,8 +37,8 @@ import org.apache.druid.query.extraction.MapLookupExtractor; import org.apache.druid.query.extraction.TimeDimExtractionFn; import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.BaseFilterTest; @@ -92,7 +92,7 @@ public class BloomDimFilterTest extends BaseFilterTest public BloomDimFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/extensions-core/druid-catalog/pom.xml b/extensions-core/druid-catalog/pom.xml index c5bafbeb8dc..24b0efccc66 100644 --- a/extensions-core/druid-catalog/pom.xml +++ b/extensions-core/druid-catalog/pom.xml @@ -31,7 +31,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/CatalogTests.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/CatalogTests.java index d89d5f7a20c..caab4a3dad8 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/CatalogTests.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/CatalogTests.java @@ -23,8 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import org.apache.druid.catalog.storage.sql.CatalogManager; import org.apache.druid.catalog.storage.sql.SQLCatalogManager; -import org.apache.druid.data.input.InputFormat; -import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.metadata.TestDerbyConnector.DerbyConnectorRule; import org.apache.druid.server.security.Access; @@ -35,8 +33,6 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceType; -import java.util.Arrays; - public class CatalogTests { public static final String TEST_AUTHORITY = "test"; @@ -74,17 +70,6 @@ public class CatalogTests } } - public static InputFormat csvFormat() - { - return new CsvInputFormat( - Arrays.asList("x", "y", "z"), - null, // listDelimiter - false, // hasHeaderRow - false, // findColumnsFromHeader - 0 // skipHeaderRows - ); - } - public static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); public static class DbFixture diff --git a/extensions-core/druid-kerberos/pom.xml b/extensions-core/druid-kerberos/pom.xml index 613a8a5937b..8f47425c1b4 100644 --- a/extensions-core/druid-kerberos/pom.xml +++ b/extensions-core/druid-kerberos/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/druid-pac4j/pom.xml b/extensions-core/druid-pac4j/pom.xml index dd73cb18edd..4a307cdf6a4 100644 --- a/extensions-core/druid-pac4j/pom.xml +++ b/extensions-core/druid-pac4j/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml @@ -38,7 +38,7 @@ 1.7 - 9.37.2 + 8.22.1 8.22 diff --git a/extensions-core/druid-ranger-security/pom.xml b/extensions-core/druid-ranger-security/pom.xml index 83bb38ca206..a468466e00a 100644 --- a/extensions-core/druid-ranger-security/pom.xml +++ b/extensions-core/druid-ranger-security/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/ec2-extensions/pom.xml b/extensions-core/ec2-extensions/pom.xml index c65a6b451be..59b5147e898 100644 --- a/extensions-core/ec2-extensions/pom.xml +++ b/extensions-core/ec2-extensions/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/google-extensions/pom.xml b/extensions-core/google-extensions/pom.xml index 97b23ffc5fa..ae0fde8f9fb 100644 --- a/extensions-core/google-extensions/pom.xml +++ b/extensions-core/google-extensions/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/google-extensions/src/test/java/org/apache/druid/data/input/google/GoogleCloudStorageInputSourceTest.java b/extensions-core/google-extensions/src/test/java/org/apache/druid/data/input/google/GoogleCloudStorageInputSourceTest.java index ae968aa3d6f..77f0b9436fb 100644 --- a/extensions-core/google-extensions/src/test/java/org/apache/druid/data/input/google/GoogleCloudStorageInputSourceTest.java +++ b/extensions-core/google-extensions/src/test/java/org/apache/druid/data/input/google/GoogleCloudStorageInputSourceTest.java @@ -406,7 +406,7 @@ public class GoogleCloudStorageInputSourceTest extends InitializedNullHandlingTe InputSourceReader reader = inputSource.reader( someSchema, - new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0), + new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null), null ); @@ -453,7 +453,7 @@ public class GoogleCloudStorageInputSourceTest extends InitializedNullHandlingTe InputSourceReader reader = inputSource.reader( someSchema, - new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0), + new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null), null ); diff --git a/extensions-core/hdfs-storage/pom.xml b/extensions-core/hdfs-storage/pom.xml index 598882329ab..058327e3c64 100644 --- a/extensions-core/hdfs-storage/pom.xml +++ b/extensions-core/hdfs-storage/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/inputsource/hdfs/HdfsInputSourceTest.java b/extensions-core/hdfs-storage/src/test/java/org/apache/druid/inputsource/hdfs/HdfsInputSourceTest.java index 10cff01c2b9..562f94e07e1 100644 --- a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/inputsource/hdfs/HdfsInputSourceTest.java +++ b/extensions-core/hdfs-storage/src/test/java/org/apache/druid/inputsource/hdfs/HdfsInputSourceTest.java @@ -90,7 +90,8 @@ public class HdfsInputSourceTest extends InitializedNullHandlingTest null, false, null, - 0 + 0, + null ); public static class ConstructorTest diff --git a/extensions-core/histogram/pom.xml b/extensions-core/histogram/pom.xml index a1c1f91e379..6014cade8cf 100644 --- a/extensions-core/histogram/pom.xml +++ b/extensions-core/histogram/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml index d3cf5d3216f..aa4f563f55c 100644 --- a/extensions-core/kafka-extraction-namespace/pom.xml +++ b/extensions-core/kafka-extraction-namespace/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index b9427fa520b..bb7ecafa358 100644 --- a/extensions-core/kafka-indexing-service/pom.xml +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java index de4bc38fc7a..b0b358da20b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java @@ -52,7 +52,8 @@ public class KafkaIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningCon @Nullable Boolean logParseExceptions, @Nullable Integer maxParseExceptions, @Nullable Integer maxSavedParseExceptions, - @Nullable Integer numPersistThreads + @Nullable Integer numPersistThreads, + @Nullable Integer maxColumnsToMerge ) { super( @@ -76,7 +77,8 @@ public class KafkaIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningCon logParseExceptions, maxParseExceptions, maxSavedParseExceptions, - numPersistThreads + numPersistThreads, + maxColumnsToMerge ); } @@ -100,7 +102,8 @@ public class KafkaIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningCon @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, - @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads + @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads, + @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge ) { this( @@ -123,7 +126,8 @@ public class KafkaIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningCon logParseExceptions, maxParseExceptions, maxSavedParseExceptions, - numPersistThreads + numPersistThreads, + maxColumnsToMerge ); } @@ -150,7 +154,8 @@ public class KafkaIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningCon isLogParseExceptions(), getMaxParseExceptions(), getMaxSavedParseExceptions(), - getNumPersistThreads() + getNumPersistThreads(), + getMaxColumnsToMerge() ); } @@ -177,7 +182,8 @@ public class KafkaIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningCon ", maxParseExceptions=" + getMaxParseExceptions() + ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + ", numPersistThreads=" + getNumPersistThreads() + - '}'; + ", getMaxColumnsToMerge=" + getMaxColumnsToMerge() + + '}'; } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index d32d694ad54..1e0b3587409 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -65,6 +65,7 @@ public class KafkaSupervisorTuningConfig extends KafkaIndexTaskTuningConfig null, null, null, + null, null ); } @@ -93,7 +94,8 @@ public class KafkaSupervisorTuningConfig extends KafkaIndexTaskTuningConfig @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, - @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads + @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads, + @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge ) { super( @@ -116,7 +118,8 @@ public class KafkaSupervisorTuningConfig extends KafkaIndexTaskTuningConfig logParseExceptions, maxParseExceptions, maxSavedParseExceptions, - numPersistThreads + numPersistThreads, + maxColumnsToMerge ); this.workerThreads = workerThreads; this.chatRetries = (chatRetries != null ? chatRetries : DEFAULT_CHAT_RETRIES); @@ -229,7 +232,8 @@ public class KafkaSupervisorTuningConfig extends KafkaIndexTaskTuningConfig isLogParseExceptions(), getMaxParseExceptions(), getMaxSavedParseExceptions(), - getNumPersistThreads() + getNumPersistThreads(), + getMaxColumnsToMerge() ); } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java index adbb7c4b677..a29654cfd73 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java @@ -712,7 +712,8 @@ public class KafkaInputFormatTest null, false, false, - 0 + 0, + null ), "kafka.newheader.", "kafka.newkey.key", diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index f8c6b23aae9..23bdeb14acb 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -92,7 +92,6 @@ import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.filter.SelectorDimFilter; @@ -1262,28 +1261,27 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase final KafkaIndexTask task = createTask( null, - new DataSchema( - "test_ds", - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat"), - new StringDimensionSchema("kafka.topic"), - new LongDimensionSchema("kafka.offset"), - new StringDimensionSchema("kafka.header.encoding") - ) - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null - ), + DataSchema.builder() + .withDataSource("test_ds") + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat"), + new StringDimensionSchema("kafka.topic"), + new LongDimensionSchema("kafka.offset"), + new StringDimensionSchema("kafka.header.encoding") + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity( + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null) + ) + .build(), new KafkaIndexTaskIOConfig( 0, "sequence0", @@ -1337,26 +1335,25 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase final KafkaIndexTask task = createTask( null, - new DataSchema( - "test_ds", - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat"), - new StringDimensionSchema("kafka.testheader.encoding") - ) - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null - ), + DataSchema.builder() + .withDataSource("test_ds") + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat"), + new StringDimensionSchema("kafka.testheader.encoding") + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity( + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null) + ) + .build(), new KafkaIndexTaskIOConfig( 0, "sequence0", @@ -2861,6 +2858,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase logParseExceptions, maxParseExceptions, maxSavedParseExceptions, + null, null ); if (!context.containsKey(SeekableStreamSupervisor.CHECKPOINTS_CTX_KEY)) { @@ -2887,16 +2885,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase private static DataSchema cloneDataSchema(final DataSchema dataSchema) { - return new DataSchema( - dataSchema.getDataSource(), - dataSchema.getTimestampSpec(), - dataSchema.getDimensionsSpec(), - dataSchema.getAggregators(), - dataSchema.getGranularitySpec(), - dataSchema.getTransformSpec(), - dataSchema.getParserMap(), - OBJECT_MAPPER - ); + return DataSchema.builder(dataSchema).withObjectMapper(OBJECT_MAPPER).build(); } @Override diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java index 5c67f6e5021..20777b320a5 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java @@ -73,6 +73,7 @@ public class KafkaIndexTaskTuningConfigTest Assert.assertEquals(false, config.isReportParseExceptions()); Assert.assertEquals(Duration.ofMinutes(15).toMillis(), config.getHandoffConditionTimeout()); Assert.assertEquals(1, config.getNumPersistThreads()); + Assert.assertEquals(-1, config.getMaxColumnsToMerge()); } @Test @@ -123,6 +124,7 @@ public class KafkaIndexTaskTuningConfigTest config.getIndexSpecForIntermediatePersists() ); Assert.assertEquals(2, config.getNumPersistThreads()); + Assert.assertEquals(-1, config.getMaxColumnsToMerge()); } @Test @@ -152,7 +154,8 @@ public class KafkaIndexTaskTuningConfigTest null, null, null, - 2 + 2, + 5 ); KafkaIndexTaskTuningConfig copy = original.convertToTaskTuningConfig(); @@ -168,6 +171,7 @@ public class KafkaIndexTaskTuningConfigTest Assert.assertEquals(true, copy.isReportParseExceptions()); Assert.assertEquals(5L, copy.getHandoffConditionTimeout()); Assert.assertEquals(2, copy.getNumPersistThreads()); + Assert.assertEquals(5, copy.getMaxColumnsToMerge()); } @Test @@ -193,7 +197,8 @@ public class KafkaIndexTaskTuningConfigTest true, 42, 42, - 2 + 2, + -1 ); String serialized = mapper.writeValueAsString(base); @@ -219,6 +224,7 @@ public class KafkaIndexTaskTuningConfigTest Assert.assertEquals(base.getMaxParseExceptions(), deserialized.getMaxParseExceptions()); Assert.assertEquals(base.getMaxSavedParseExceptions(), deserialized.getMaxSavedParseExceptions()); Assert.assertEquals(base.getNumPersistThreads(), deserialized.getNumPersistThreads()); + Assert.assertEquals(base.getMaxColumnsToMerge(), deserialized.getMaxColumnsToMerge()); } @Test @@ -244,6 +250,7 @@ public class KafkaIndexTaskTuningConfigTest 42, 42, 2, + -1, "extra string" ); @@ -269,6 +276,7 @@ public class KafkaIndexTaskTuningConfigTest Assert.assertEquals(base.getMaxParseExceptions(), deserialized.getMaxParseExceptions()); Assert.assertEquals(base.getMaxSavedParseExceptions(), deserialized.getMaxSavedParseExceptions()); Assert.assertEquals(base.getNumPersistThreads(), deserialized.getNumPersistThreads()); + Assert.assertEquals(base.getMaxColumnsToMerge(), deserialized.getMaxColumnsToMerge()); } @Test diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java index 0a0b64396a6..9cdc0ac0edc 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java @@ -46,7 +46,6 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.TestHelper; @@ -81,45 +80,30 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); private static final String TOPIC = "sampling"; - private static final DataSchema DATA_SCHEMA = new DataSchema( - "test_ds", - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null - ); + private static final DataSchema DATA_SCHEMA = + DataSchema.builder() + .withDataSource("test_ds") + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity( + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null) + ) + .build(); - private static final DataSchema DATA_SCHEMA_KAFKA_TIMESTAMP = new DataSchema( - "test_ds", - new TimestampSpec("kafka.timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null - ); + private static final DataSchema DATA_SCHEMA_KAFKA_TIMESTAMP = + DataSchema.builder(DATA_SCHEMA) + .withTimestamp(new TimestampSpec("kafka.timestamp", "iso", null)) + .build(); private static TestingCluster zkServer; private static TestBroker kafkaServer; @@ -364,17 +348,18 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest ); InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8"); - DataSchema dataSchema = new DataSchema( - "test_ds", - objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - objectMapper - ); + DataSchema dataSchema = DataSchema.builder() + .withDataSource("test_ds") + .withParserMap( + objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class) + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .withObjectMapper(objectMapper) + .build(); KafkaSupervisorSpec supervisorSpec = new KafkaSupervisorSpec( null, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 86275d10e31..e436b8cd56a 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -30,7 +30,6 @@ import com.google.common.util.concurrent.ListenableFuture; import org.apache.curator.test.TestingCluster; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; @@ -83,7 +82,6 @@ import org.apache.druid.java.util.emitter.service.AlertBuilder; import org.apache.druid.java.util.emitter.service.AlertEvent; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.StubServiceEmitter; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.ParseExceptionReport; @@ -346,7 +344,8 @@ public class KafkaSupervisorTest extends EasyMockSupport null, null, null, - null + null, + null ); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(kafkaSupervisorIOConfig).anyTimes(); @@ -497,6 +496,7 @@ public class KafkaSupervisorTest extends EasyMockSupport null, null, null, + null, null ), null @@ -4221,6 +4221,7 @@ public class KafkaSupervisorTest extends EasyMockSupport null, null, null, + null, null ) ); @@ -4260,6 +4261,7 @@ public class KafkaSupervisorTest extends EasyMockSupport null, null, null, + null, null ); @@ -4413,6 +4415,7 @@ public class KafkaSupervisorTest extends EasyMockSupport null, null, null, + null, null ) ); @@ -4888,6 +4891,7 @@ public class KafkaSupervisorTest extends EasyMockSupport null, null, 10, + null, null ); @@ -5002,6 +5006,7 @@ public class KafkaSupervisorTest extends EasyMockSupport null, null, null, + null, null ); @@ -5128,18 +5133,19 @@ public class KafkaSupervisorTest extends EasyMockSupport dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim2")); - return new DataSchema( - dataSource, - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(dimensions), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.NONE, - ImmutableList.of() - ), - null - ); + return DataSchema.builder() + .withDataSource(dataSource) + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions(dimensions) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity( + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of() + ) + ) + .build(); } private KafkaIndexTask createKafkaIndexTask( diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java index 8e5ad243d2e..a2b7228bb46 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java @@ -56,6 +56,7 @@ public class TestModifiedKafkaIndexTaskTuningConfig extends KafkaIndexTaskTuning @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, @JsonProperty("numPersistThreads") @Nullable Integer numPersistThreads, + @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge, @JsonProperty("extra") String extra ) { @@ -79,7 +80,8 @@ public class TestModifiedKafkaIndexTaskTuningConfig extends KafkaIndexTaskTuning logParseExceptions, maxParseExceptions, maxSavedParseExceptions, - numPersistThreads + numPersistThreads, + maxColumnsToMerge ); this.extra = extra; } diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index fb2fa2234d5..26dcf1d5eb9 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java index a86541a0487..83b111e9e96 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java @@ -92,7 +92,8 @@ public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningC @Nullable Integer maxSavedParseExceptions, @Deprecated @Nullable Integer maxRecordsPerPoll, @Nullable Integer maxBytesPerPoll, - @Nullable Period intermediateHandoffPeriod + @Nullable Period intermediateHandoffPeriod, + @Nullable Integer maxColumnsToMerge ) { super( @@ -116,7 +117,8 @@ public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningC logParseExceptions, maxParseExceptions, maxSavedParseExceptions, - null + null, + maxColumnsToMerge ); this.recordBufferSize = recordBufferSize; this.recordBufferSizeBytes = recordBufferSizeBytes; @@ -161,7 +163,8 @@ public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningC @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, @JsonProperty("maxRecordsPerPoll") @Deprecated @Nullable Integer maxRecordsPerPoll, @JsonProperty("maxBytesPerPoll") @Nullable Integer maxBytesPerPoll, - @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod + @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod, + @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge ) { this( @@ -191,7 +194,8 @@ public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningC maxSavedParseExceptions, maxRecordsPerPoll, maxBytesPerPoll, - intermediateHandoffPeriod + intermediateHandoffPeriod, + maxColumnsToMerge ); } @@ -294,7 +298,8 @@ public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningC getMaxSavedParseExceptions(), getMaxRecordsPerPollConfigured(), getMaxBytesPerPollConfigured(), - getIntermediateHandoffPeriod() + getIntermediateHandoffPeriod(), + getMaxColumnsToMerge() ); } @@ -363,6 +368,7 @@ public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningC ", maxRecordsPerPoll=" + maxRecordsPerPoll + ", maxBytesPerPoll=" + maxBytesPerPoll + ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() + - '}'; + ", maxColumnsToMerge=" + getMaxColumnsToMerge() + + '}'; } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index a0a68c14bc0..1a11f8d658b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -76,6 +76,7 @@ public class KinesisSupervisorTuningConfig extends KinesisIndexTaskTuningConfig null, null, null, + null, null ); } @@ -113,7 +114,8 @@ public class KinesisSupervisorTuningConfig extends KinesisIndexTaskTuningConfig @JsonProperty("intermediateHandoffPeriod") Period intermediateHandoffPeriod, @JsonProperty("repartitionTransitionDuration") Period repartitionTransitionDuration, @JsonProperty("offsetFetchPeriod") Period offsetFetchPeriod, - @JsonProperty("useListShards") Boolean useListShards + @JsonProperty("useListShards") Boolean useListShards, + @JsonProperty("maxColumnsToMerge") Integer maxColumnsToMerge ) { super( @@ -143,7 +145,8 @@ public class KinesisSupervisorTuningConfig extends KinesisIndexTaskTuningConfig maxSavedParseExceptions, maxRecordsPerPoll, maxBytesPerPoll, - intermediateHandoffPeriod + intermediateHandoffPeriod, + maxColumnsToMerge ); this.workerThreads = workerThreads; @@ -244,6 +247,7 @@ public class KinesisSupervisorTuningConfig extends KinesisIndexTaskTuningConfig ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() + ", repartitionTransitionDuration=" + getRepartitionTransitionDuration() + ", useListShards=" + isUseListShards() + + ", maxColumnsToMerge=" + getMaxColumnsToMerge() + '}'; } @@ -277,7 +281,8 @@ public class KinesisSupervisorTuningConfig extends KinesisIndexTaskTuningConfig getMaxSavedParseExceptions(), getMaxRecordsPerPollConfigured(), getMaxBytesPerPollConfigured(), - getIntermediateHandoffPeriod() + getIntermediateHandoffPeriod(), + getMaxColumnsToMerge() ); } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/data/input/kinesis/KinesisInputFormatTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/data/input/kinesis/KinesisInputFormatTest.java index 130f31681de..7e40203c48e 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/data/input/kinesis/KinesisInputFormatTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/data/input/kinesis/KinesisInputFormatTest.java @@ -713,7 +713,8 @@ public class KinesisInputFormatTest null, false, false, - 0 + 0, + null ), "kinesis.newts.partitionKey", "kinesis.newts.timestamp" diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java index 60d8f686a28..e84581af601 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java @@ -50,7 +50,8 @@ import java.util.Collections; public class KinesisIndexTaskSerdeTest { - private static final DataSchema DATA_SCHEMA = new DataSchema("dataSource", null, null, null, null, null, null, null); + private static final DataSchema DATA_SCHEMA = + DataSchema.builder().withDataSource("dataSource").build(); private static final KinesisIndexTaskTuningConfig TUNING_CONFIG = new KinesisIndexTaskTuningConfig( null, null, @@ -78,6 +79,7 @@ public class KinesisIndexTaskSerdeTest null, null, null, + null, null ); private static final KinesisIndexTaskIOConfig IO_CONFIG = new KinesisIndexTaskIOConfig( diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 510eaa797e0..2ef39148400 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -2374,7 +2374,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase maxSavedParseExceptions, maxRecordsPerPoll, maxBytesPerPoll, - intermediateHandoffPeriod + intermediateHandoffPeriod, + null ); return createTask(taskId, dataSchema, ioConfig, tuningConfig, context); } @@ -2411,16 +2412,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase private static DataSchema cloneDataSchema(final DataSchema dataSchema) { - return new DataSchema( - dataSchema.getDataSource(), - dataSchema.getTimestampSpec(), - dataSchema.getDimensionsSpec(), - dataSchema.getAggregators(), - dataSchema.getGranularitySpec(), - dataSchema.getTransformSpec(), - dataSchema.getParserMap(), - OBJECT_MAPPER - ); + return DataSchema.builder(dataSchema).withObjectMapper(OBJECT_MAPPER).build(); } @Override diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index b61c5cf2ae4..375e26e2ed2 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -134,6 +134,8 @@ public class KinesisIndexTaskTuningConfigTest Assert.assertEquals(2, (int) config.getFetchThreads()); Assert.assertTrue(config.isSkipSequenceNumberAvailabilityCheck()); Assert.assertFalse(config.isResetOffsetAutomatically()); + Assert.assertEquals(-1, config.getMaxColumnsToMerge()); + } @Test @@ -166,7 +168,8 @@ public class KinesisIndexTaskTuningConfigTest 500, 6000, 1_000_000, - new Period("P3D") + new Period("P3D"), + 1000 ); String serialized = mapper.writeValueAsString(base); @@ -197,6 +200,7 @@ public class KinesisIndexTaskTuningConfigTest Assert.assertEquals(base.getRecordBufferSizeBytesConfigured(), deserialized.getRecordBufferSizeBytesConfigured()); Assert.assertEquals(base.getMaxRecordsPerPollConfigured(), deserialized.getMaxRecordsPerPollConfigured()); Assert.assertEquals(base.getMaxBytesPerPollConfigured(), deserialized.getMaxBytesPerPollConfigured()); + Assert.assertEquals(base.getMaxColumnsToMerge(), deserialized.getMaxColumnsToMerge()); } @Test @@ -229,7 +233,8 @@ public class KinesisIndexTaskTuningConfigTest 500, 1_000_000, 6000, - new Period("P3D") + new Period("P3D"), + 1000 ); String serialized = mapper.writeValueAsString(new TestModifiedKinesisIndexTaskTuningConfig(base, "loool")); @@ -257,6 +262,7 @@ public class KinesisIndexTaskTuningConfigTest Assert.assertEquals(base.getRecordBufferOfferTimeout(), deserialized.getRecordBufferOfferTimeout()); Assert.assertEquals(base.getRecordBufferSizeBytesConfigured(), deserialized.getRecordBufferSizeBytesConfigured()); Assert.assertEquals(base.getMaxRecordsPerPollConfigured(), deserialized.getMaxRecordsPerPollConfigured()); + Assert.assertEquals(base.getMaxColumnsToMerge(), deserialized.getMaxColumnsToMerge()); } @Test @@ -322,6 +328,7 @@ public class KinesisIndexTaskTuningConfigTest null, null, null, + null, null ); KinesisIndexTaskTuningConfig copy = original.convertToTaskTuningConfig(); @@ -345,6 +352,7 @@ public class KinesisIndexTaskTuningConfigTest Assert.assertTrue(copy.isResetOffsetAutomatically()); Assert.assertEquals(10, (int) copy.getMaxRecordsPerPollConfigured()); Assert.assertEquals(new Period().withDays(Integer.MAX_VALUE), copy.getIntermediateHandoffPeriod()); + Assert.assertEquals(-1, copy.getMaxColumnsToMerge()); } @Test diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java index 63144c6a935..102e2d8929f 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java @@ -49,7 +49,6 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; @@ -75,25 +74,25 @@ public class KinesisSamplerSpecTest extends EasyMockSupport { private static final String STREAM = "sampling"; private static final String SHARD_ID = "1"; - private static final DataSchema DATA_SCHEMA = new DataSchema( - "test_ds", - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null - ); + private static final DataSchema DATA_SCHEMA = + DataSchema.builder() + .withDataSource("test_ds") + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity( + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null) + ) + .build(); static { NullHandling.initializeForTests(); @@ -192,17 +191,18 @@ public class KinesisSamplerSpecTest extends EasyMockSupport ); InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8"); - DataSchema dataSchema = new DataSchema( - "test_ds", - objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - objectMapper - ); + DataSchema dataSchema = DataSchema.builder() + .withDataSource("test_ds") + .withParserMap( + objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class) + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .withObjectMapper(objectMapper) + .build(); KinesisSupervisorSpec supervisorSpec = new KinesisSupervisorSpec( null, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index e6ed27c9cec..24d919918f4 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -28,7 +28,6 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; @@ -79,7 +78,6 @@ import org.apache.druid.java.util.emitter.service.AlertBuilder; import org.apache.druid.java.util.emitter.service.AlertEvent; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.StubServiceEmitter; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -207,6 +205,7 @@ public class KinesisSupervisorTest extends EasyMockSupport null, null, null, + null, null ); rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); @@ -3980,6 +3979,7 @@ public class KinesisSupervisorTest extends EasyMockSupport null, null, null, + null, null ); @@ -5159,6 +5159,7 @@ public class KinesisSupervisorTest extends EasyMockSupport null, null, null, + null, null ); @@ -5479,18 +5480,19 @@ public class KinesisSupervisorTest extends EasyMockSupport dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim2")); - return new DataSchema( - dataSource, - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(dimensions), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.NONE, - ImmutableList.of() - ), - null - ); + return DataSchema.builder() + .withDataSource(dataSource) + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions(dimensions) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity( + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of() + ) + ) + .build(); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java index ac84d2105cd..b85c3edc2a6 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java @@ -63,6 +63,7 @@ public class TestModifiedKinesisIndexTaskTuningConfig extends KinesisIndexTaskTu @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll, @JsonProperty("maxBytesPerPoll") @Nullable Integer maxBytesPerPoll, @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod, + @JsonProperty("maxColumnsToMerge") @Nullable Integer maxColumnsToMerge, @JsonProperty("extra") String extra ) { @@ -93,7 +94,8 @@ public class TestModifiedKinesisIndexTaskTuningConfig extends KinesisIndexTaskTu maxSavedParseExceptions, maxRecordsPerPoll, maxBytesPerPoll, - intermediateHandoffPeriod + intermediateHandoffPeriod, + maxColumnsToMerge ); this.extra = extra; } @@ -127,7 +129,8 @@ public class TestModifiedKinesisIndexTaskTuningConfig extends KinesisIndexTaskTu base.getMaxSavedParseExceptions(), base.getMaxRecordsPerPollConfigured(), base.getMaxBytesPerPollConfigured(), - base.getIntermediateHandoffPeriod() + base.getIntermediateHandoffPeriod(), + base.getMaxColumnsToMerge() ); this.extra = extra; } diff --git a/extensions-core/kubernetes-extensions/pom.xml b/extensions-core/kubernetes-extensions/pom.xml index e51d1218464..6ef180795e2 100644 --- a/extensions-core/kubernetes-extensions/pom.xml +++ b/extensions-core/kubernetes-extensions/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/lookups-cached-global/pom.xml b/extensions-core/lookups-cached-global/pom.xml index d4e09011df9..e6969f999fd 100644 --- a/extensions-core/lookups-cached-global/pom.xml +++ b/extensions-core/lookups-cached-global/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/namespace/UriExtractionNamespace.java b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/namespace/UriExtractionNamespace.java index e5ad9cd658e..e1c103c0ca2 100644 --- a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/namespace/UriExtractionNamespace.java +++ b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/namespace/UriExtractionNamespace.java @@ -327,7 +327,7 @@ public class UriExtractionNamespace implements ExtractionNamespace this.valueColumn, Arrays.toString(columns.toArray()) ); - CSVParser csvParser = new CSVParser(null, columns, hasHeaderRow, skipHeaderRows); + CSVParser csvParser = new CSVParser(null, columns, hasHeaderRow, skipHeaderRows, false); csvParser.startFileFromBeginning(); this.parser = new DelegateParser( csvParser, @@ -355,13 +355,13 @@ public class UriExtractionNamespace implements ExtractionNamespace @JsonProperty public String getKeyColumn() { - return this.keyColumn; + return keyColumn; } @JsonProperty public String getValueColumn() { - return this.valueColumn; + return valueColumn; } @Override @@ -431,7 +431,8 @@ public class UriExtractionNamespace implements ExtractionNamespace StringUtils.emptyToNullNonDruidDataString(delimiter), StringUtils.emptyToNullNonDruidDataString(listDelimiter), hasHeaderRow, - skipHeaderRows + skipHeaderRows, + false ); delegate.startFileFromBeginning(); Preconditions.checkArgument( diff --git a/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/query/lookup/namespace/UriExtractionNamespaceTest.java b/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/query/lookup/namespace/UriExtractionNamespaceTest.java index dc50126f066..61d90b38a98 100644 --- a/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/query/lookup/namespace/UriExtractionNamespaceTest.java +++ b/extensions-core/lookups-cached-global/src/test/java/org/apache/druid/query/lookup/namespace/UriExtractionNamespaceTest.java @@ -115,6 +115,7 @@ public class UriExtractionNamespaceTest // The third row will parse to data Assert.assertEquals(ImmutableMap.of("val2", "val3"), parser.getParser().parseToMap("val1,val2,val3")); } + @Test(expected = IllegalArgumentException.class) public void testBadCSV() { diff --git a/extensions-core/lookups-cached-single/pom.xml b/extensions-core/lookups-cached-single/pom.xml index 62d84bf20ef..d1c3f9f2b72 100644 --- a/extensions-core/lookups-cached-single/pom.xml +++ b/extensions-core/lookups-cached-single/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/multi-stage-query/pom.xml b/extensions-core/multi-stage-query/pom.xml index 22e85f0b0fc..993d3a44853 100644 --- a/extensions-core/multi-stage-query/pom.xml +++ b/extensions-core/multi-stage-query/pom.xml @@ -31,7 +31,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java index d2370b05793..d316b9b6b0b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Controller.java @@ -84,7 +84,7 @@ public interface Controller * taskId, not by query/stage/worker, because system errors are associated * with a task rather than a specific query/stage/worker execution context. * - * @see ControllerClient#postWorkerError(String, MSQErrorReport) + * @see ControllerClient#postWorkerError(MSQErrorReport) */ void workerError(MSQErrorReport errorReport); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java index 428ce59cd8f..f56b752133f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerClient.java @@ -83,10 +83,7 @@ public interface ControllerClient extends Closeable /** * Client side method to inform the controller that the error has occured in the given worker. */ - void postWorkerError( - String workerId, - MSQErrorReport errorWrapper - ) throws IOException; + void postWorkerError(MSQErrorReport errorWrapper) throws IOException; /** * Client side method to inform the controller about the warnings generated by the given worker. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerContext.java index 40b114511c2..42515a1779a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerContext.java @@ -28,8 +28,10 @@ import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.input.InputSpecSlicer; import org.apache.druid.msq.input.table.SegmentsInputSlice; import org.apache.druid.msq.input.table.TableInputSpec; -import org.apache.druid.msq.kernel.QueryDefinition; import org.apache.druid.msq.kernel.controller.ControllerQueryKernelConfig; +import org.apache.druid.msq.querykit.QueryKit; +import org.apache.druid.msq.querykit.QueryKitSpec; +import org.apache.druid.query.Query; import org.apache.druid.server.DruidNode; /** @@ -41,7 +43,7 @@ public interface ControllerContext /** * Configuration for {@link org.apache.druid.msq.kernel.controller.ControllerQueryKernel}. */ - ControllerQueryKernelConfig queryKernelConfig(MSQSpec querySpec, QueryDefinition queryDef); + ControllerQueryKernelConfig queryKernelConfig(String queryId, MSQSpec querySpec); /** * Callback from the controller implementation to "register" the controller. Used in the indexing task implementation @@ -73,7 +75,7 @@ public interface ControllerContext /** * Provides an {@link InputSpecSlicer} that slices {@link TableInputSpec} into {@link SegmentsInputSlice}. */ - InputSpecSlicer newTableInputSpecSlicer(); + InputSpecSlicer newTableInputSpecSlicer(WorkerManager workerManager); /** * Provide access to segment actions in the Overlord. Only called for ingestion queries, i.e., where @@ -86,7 +88,7 @@ public interface ControllerContext * * @param queryId query ID * @param querySpec query spec - * @param queryKernelConfig config from {@link #queryKernelConfig(MSQSpec, QueryDefinition)} + * @param queryKernelConfig config from {@link #queryKernelConfig(String, MSQSpec)} * @param workerFailureListener listener that receives callbacks when workers fail */ WorkerManager newWorkerManager( @@ -100,4 +102,15 @@ public interface ControllerContext * Client for communicating with workers. */ WorkerClient newWorkerClient(); + + /** + * Create a {@link QueryKitSpec}. This method provides controller contexts a way to customize parameters around the + * number of workers and partitions. + */ + QueryKitSpec makeQueryKitSpec( + QueryKit> queryKit, + String queryId, + MSQSpec querySpec, + ControllerQueryKernelConfig queryKernelConfig + ); } 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 77a0b7d48d6..72d8216088f 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 @@ -117,6 +117,7 @@ import org.apache.druid.msq.indexing.error.TooManyBucketsFault; import org.apache.druid.msq.indexing.error.TooManySegmentsInTimeChunkFault; import org.apache.druid.msq.indexing.error.TooManyWarningsFault; import org.apache.druid.msq.indexing.error.UnknownFault; +import org.apache.druid.msq.indexing.error.WorkerFailedFault; import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault; import org.apache.druid.msq.indexing.processor.SegmentGeneratorFrameProcessorFactory; import org.apache.druid.msq.indexing.report.MSQSegmentReport; @@ -151,6 +152,7 @@ import org.apache.druid.msq.kernel.controller.ControllerStagePhase; import org.apache.druid.msq.kernel.controller.WorkerInputs; import org.apache.druid.msq.querykit.MultiQueryKit; import org.apache.druid.msq.querykit.QueryKit; +import org.apache.druid.msq.querykit.QueryKitSpec; import org.apache.druid.msq.querykit.QueryKitUtils; import org.apache.druid.msq.querykit.ShuffleSpecFactory; import org.apache.druid.msq.querykit.WindowOperatorQueryKit; @@ -224,6 +226,7 @@ import java.util.stream.StreamSupport; public class ControllerImpl implements Controller { private static final Logger log = new Logger(ControllerImpl.class); + private static final String RESULT_READER_CANCELLATION_ID = "result-reader"; private final String queryId; private final MSQSpec querySpec; @@ -364,7 +367,7 @@ public class ControllerImpl implements Controller // Execution-related: run the multi-stage QueryDefinition. final InputSpecSlicerFactory inputSpecSlicerFactory = - makeInputSpecSlicerFactory(context.newTableInputSpecSlicer()); + makeInputSpecSlicerFactory(context.newTableInputSpecSlicer(workerManager)); final Pair> queryRunResult = new RunQueryUntilDone( @@ -560,12 +563,12 @@ public class ControllerImpl implements Controller private QueryDefinition initializeQueryDefAndState(final Closer closer) { this.selfDruidNode = context.selfNode(); - this.netClient = new ExceptionWrappingWorkerClient(context.newWorkerClient()); - closer.register(netClient); + this.netClient = closer.register(new ExceptionWrappingWorkerClient(context.newWorkerClient())); + this.queryKernelConfig = context.queryKernelConfig(queryId, querySpec); + final QueryContext queryContext = querySpec.getQuery().context(); final QueryDefinition queryDef = makeQueryDefinition( - queryId(), - makeQueryControllerToolKit(), + context.makeQueryKitSpec(makeQueryControllerToolKit(), queryId, querySpec, queryKernelConfig), querySpec, context.jsonMapper(), resultsContext @@ -587,7 +590,6 @@ public class ControllerImpl implements Controller QueryValidator.validateQueryDef(queryDef); queryDefRef.set(queryDef); - queryKernelConfig = context.queryKernelConfig(querySpec, queryDef); workerManager = context.newWorkerManager( queryId, querySpec, @@ -612,7 +614,7 @@ public class ControllerImpl implements Controller ); } - final long maxParseExceptions = MultiStageQueryContext.getMaxParseExceptions(querySpec.getQuery().context()); + final long maxParseExceptions = MultiStageQueryContext.getMaxParseExceptions(queryContext); this.faultsExceededChecker = new FaultsExceededChecker( ImmutableMap.of(CannotParseExternalDataFault.CODE, maxParseExceptions) ); @@ -624,7 +626,7 @@ public class ControllerImpl implements Controller stageDefinition.getId().getStageNumber(), finalizeClusterStatisticsMergeMode( stageDefinition, - MultiStageQueryContext.getClusterStatisticsMergeMode(querySpec.getQuery().context()) + MultiStageQueryContext.getClusterStatisticsMergeMode(queryContext) ) ) ); @@ -748,6 +750,11 @@ public class ControllerImpl implements Controller } workerErrorRef.compareAndSet(null, mapQueryColumnNameToOutputColumnName(errorReport)); + + // Wake up the main controller thread. + addToKernelManipulationQueue(kernel -> { + throw new MSQException(new WorkerFailedFault(errorReport.getTaskId(), null)); + }); } /** @@ -1190,7 +1197,7 @@ public class ControllerImpl implements Controller } @SuppressWarnings("rawtypes") - private QueryKit makeQueryControllerToolKit() + private QueryKit> makeQueryControllerToolKit() { final Map, QueryKit> kitMap = ImmutableMap., QueryKit>builder() @@ -1603,9 +1610,10 @@ public class ControllerImpl implements Controller if (shardSpec != null) { if (Objects.equals(shardSpec.getType(), ShardSpec.Type.RANGE)) { List partitionDimensions = ((DimensionRangeShardSpec) shardSpec).getDimensions(); + // Effective maxRowsPerSegment is propagated as rowsPerSegment in MSQ partitionSpec = new DimensionRangePartitionsSpec( - tuningConfig.getRowsPerSegment(), null, + tuningConfig.getRowsPerSegment(), partitionDimensions, false ); @@ -1623,9 +1631,10 @@ public class ControllerImpl implements Controller ))); } } else if (clusterBy != null && !clusterBy.getColumns().isEmpty()) { + // Effective maxRowsPerSegment is propagated as rowsPerSegment in MSQ partitionSpec = new DimensionRangePartitionsSpec( - tuningConfig.getRowsPerSegment(), null, + tuningConfig.getRowsPerSegment(), clusterBy.getColumns() .stream() .map(KeyColumn::columnName).collect(Collectors.toList()), @@ -1712,8 +1721,7 @@ public class ControllerImpl implements Controller @SuppressWarnings("unchecked") private static QueryDefinition makeQueryDefinition( - final String queryId, - @SuppressWarnings("rawtypes") final QueryKit toolKit, + final QueryKitSpec queryKitSpec, final MSQSpec querySpec, final ObjectMapper jsonMapper, final ResultsContext resultsContext @@ -1722,11 +1730,11 @@ public class ControllerImpl implements Controller final MSQTuningConfig tuningConfig = querySpec.getTuningConfig(); final ColumnMappings columnMappings = querySpec.getColumnMappings(); final Query queryToPlan; - final ShuffleSpecFactory shuffleSpecFactory; + final ShuffleSpecFactory resultShuffleSpecFactory; if (MSQControllerTask.isIngestion(querySpec)) { - shuffleSpecFactory = querySpec.getDestination() - .getShuffleSpecFactory(tuningConfig.getRowsPerSegment()); + resultShuffleSpecFactory = querySpec.getDestination() + .getShuffleSpecFactory(tuningConfig.getRowsPerSegment()); if (!columnMappings.hasUniqueOutputColumnNames()) { // We do not expect to hit this case in production, because the SQL validator checks that column names @@ -1750,7 +1758,7 @@ public class ControllerImpl implements Controller queryToPlan = querySpec.getQuery(); } } else { - shuffleSpecFactory = + resultShuffleSpecFactory = querySpec.getDestination() .getShuffleSpecFactory(MultiStageQueryContext.getRowsPerPage(querySpec.getQuery().context())); queryToPlan = querySpec.getQuery(); @@ -1759,12 +1767,10 @@ public class ControllerImpl implements Controller final QueryDefinition queryDef; try { - queryDef = toolKit.makeQueryDefinition( - queryId, + queryDef = queryKitSpec.getQueryKit().makeQueryDefinition( + queryKitSpec, queryToPlan, - toolKit, - shuffleSpecFactory, - tuningConfig.getMaxNumWorkers(), + resultShuffleSpecFactory, 0 ); } @@ -1793,7 +1799,7 @@ public class ControllerImpl implements Controller // Add all query stages. // Set shuffleCheckHasMultipleValues on the stage that serves as input to the final segment-generation stage. - final QueryDefinitionBuilder builder = QueryDefinition.builder(queryId); + final QueryDefinitionBuilder builder = QueryDefinition.builder(queryKitSpec.getQueryId()); for (final StageDefinition stageDef : queryDef.getStageDefinitions()) { if (stageDef.equals(finalShuffleStageDef)) { @@ -1819,7 +1825,7 @@ public class ControllerImpl implements Controller // attaching new query results stage if the final stage does sort during shuffle so that results are ordered. StageDefinition finalShuffleStageDef = queryDef.getFinalStageDefinition(); if (finalShuffleStageDef.doesSortDuringShuffle()) { - final QueryDefinitionBuilder builder = QueryDefinition.builder(queryId); + final QueryDefinitionBuilder builder = QueryDefinition.builder(queryKitSpec.getQueryId()); builder.addAll(queryDef); builder.add(StageDefinition.builder(queryDef.getNextStageNumber()) .inputs(new StageInputSpec(queryDef.getFinalStageDefinition().getStageNumber())) @@ -1856,7 +1862,7 @@ public class ControllerImpl implements Controller } final ResultFormat resultFormat = exportMSQDestination.getResultFormat(); - final QueryDefinitionBuilder builder = QueryDefinition.builder(queryId); + final QueryDefinitionBuilder builder = QueryDefinition.builder(queryKitSpec.getQueryId()); builder.addAll(queryDef); builder.add(StageDefinition.builder(queryDef.getNextStageNumber()) .inputs(new StageInputSpec(queryDef.getFinalStageDefinition().getStageNumber())) @@ -1864,7 +1870,7 @@ public class ControllerImpl implements Controller .signature(queryDef.getFinalStageDefinition().getSignature()) .shuffleSpec(null) .processorFactory(new ExportResultsFrameProcessorFactory( - queryId, + queryKitSpec.getQueryId(), exportStorageProvider, resultFormat, columnMappings, @@ -2180,6 +2186,34 @@ public class ControllerImpl implements Controller } } + /** + * Create a result-reader executor for {@link RunQueryUntilDone#readQueryResults()}. + */ + private static FrameProcessorExecutor createResultReaderExec(final String queryId) + { + return new FrameProcessorExecutor( + MoreExecutors.listeningDecorator( + Execs.singleThreaded(StringUtils.encodeForFormat("msq-result-reader[" + queryId + "]"))) + ); + } + + /** + * Cancel any currently-running work and shut down a result-reader executor, like one created by + * {@link #createResultReaderExec(String)}. + */ + private static void closeResultReaderExec(final FrameProcessorExecutor exec) + { + try { + exec.cancel(RESULT_READER_CANCELLATION_ID); + } + catch (Exception e) { + throw new RuntimeException(e); + } + finally { + exec.shutdownNow(); + } + } + private void stopExternalFetchers() { if (workerSketchFetcher != null) { @@ -2689,12 +2723,9 @@ public class ControllerImpl implements Controller inputChannelFactory = new WorkerInputChannelFactory(netClient, () -> taskIds); } - final FrameProcessorExecutor resultReaderExec = new FrameProcessorExecutor( - MoreExecutors.listeningDecorator( - Execs.singleThreaded(StringUtils.encodeForFormat("msq-result-reader[" + queryId() + "]"))) - ); + final FrameProcessorExecutor resultReaderExec = createResultReaderExec(queryId()); + resultReaderExec.registerCancellationId(RESULT_READER_CANCELLATION_ID); - final String cancellationId = "results-reader"; ReadableConcatFrameChannel resultsChannel = null; try { @@ -2704,7 +2735,7 @@ public class ControllerImpl implements Controller inputChannelFactory, () -> ArenaMemoryAllocator.createOnHeap(5_000_000), resultReaderExec, - cancellationId, + RESULT_READER_CANCELLATION_ID, null, MultiStageQueryContext.removeNullBytes(querySpec.getQuery().context()) ); @@ -2738,7 +2769,7 @@ public class ControllerImpl implements Controller queryListener ); - queryResultsReaderFuture = resultReaderExec.runFully(resultsReader, cancellationId); + queryResultsReaderFuture = resultReaderExec.runFully(resultsReader, RESULT_READER_CANCELLATION_ID); // When results are done being read, kick the main thread. // Important: don't use FutureUtils.futureWithBaggage, because we need queryResultsReaderFuture to resolve @@ -2755,23 +2786,13 @@ public class ControllerImpl implements Controller e, () -> CloseableUtils.closeAll( finalResultsChannel, - () -> resultReaderExec.getExecutorService().shutdownNow() + () -> closeResultReaderExec(resultReaderExec) ) ); } // Result reader is set up. Register with the query-wide closer. - closer.register(() -> { - try { - resultReaderExec.cancel(cancellationId); - } - catch (Exception e) { - throw new RuntimeException(e); - } - finally { - resultReaderExec.getExecutorService().shutdownNow(); - } - }); + closer.register(() -> closeResultReaderExec(resultReaderExec)); } /** diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerMemoryParameters.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerMemoryParameters.java index 2ab016e10e4..c5131ddd84e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerMemoryParameters.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerMemoryParameters.java @@ -19,7 +19,6 @@ package org.apache.druid.msq.exec; -import com.google.common.base.Preconditions; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.NotEnoughMemoryFault; import org.apache.druid.msq.kernel.controller.ControllerQueryKernel; @@ -29,10 +28,10 @@ import org.apache.druid.msq.statistics.ClusterByStatisticsCollectorImpl; * Class for determining how much JVM heap to allocate to various purposes for {@link Controller}. * * First, look at how much of total JVM heap that is dedicated for MSQ; see - * {@link MemoryIntrospector#usableMemoryInJvm()}. + * {@link MemoryIntrospector#memoryPerTask()}. * * Then, we split up that total amount of memory into equally-sized portions per {@link Controller}; see - * {@link MemoryIntrospector#numQueriesInJvm()}. The number of controllers is based entirely on server configuration, + * {@link MemoryIntrospector#numTasksInJvm()}. The number of controllers is based entirely on server configuration, * which makes the calculation robust to different queries running simultaneously in the same JVM. * * Then, we split that up into a chunk used for input channels, and a chunk used for partition statistics. @@ -70,29 +69,28 @@ public class ControllerMemoryParameters final int maxWorkerCount ) { - final long usableMemoryInJvm = memoryIntrospector.usableMemoryInJvm(); - final int numControllersInJvm = memoryIntrospector.numQueriesInJvm(); - Preconditions.checkArgument(usableMemoryInJvm > 0, "Usable memory[%s] must be > 0", usableMemoryInJvm); - Preconditions.checkArgument(numControllersInJvm > 0, "Number of controllers[%s] must be > 0", numControllersInJvm); - Preconditions.checkArgument(maxWorkerCount > 0, "Number of workers[%s] must be > 0", maxWorkerCount); - - final long memoryPerController = usableMemoryInJvm / numControllersInJvm; - final long memoryForInputChannels = WorkerMemoryParameters.memoryNeededForInputChannels(maxWorkerCount); + final long totalMemory = memoryIntrospector.memoryPerTask(); + final long memoryForInputChannels = + WorkerMemoryParameters.computeProcessorMemoryForInputChannels( + maxWorkerCount, + WorkerMemoryParameters.DEFAULT_FRAME_SIZE + ); final int partitionStatisticsMaxRetainedBytes = (int) Math.min( - memoryPerController - memoryForInputChannels, + totalMemory - memoryForInputChannels, PARTITION_STATS_MAX_MEMORY ); if (partitionStatisticsMaxRetainedBytes < PARTITION_STATS_MIN_MEMORY) { - final long requiredMemory = memoryForInputChannels + PARTITION_STATS_MIN_MEMORY; + final long requiredTaskMemory = memoryForInputChannels + PARTITION_STATS_MIN_MEMORY; throw new MSQException( new NotEnoughMemoryFault( - memoryIntrospector.computeJvmMemoryRequiredForUsableMemory(requiredMemory), + memoryIntrospector.computeJvmMemoryRequiredForTaskMemory(requiredTaskMemory), memoryIntrospector.totalMemoryInJvm(), - usableMemoryInJvm, - numControllersInJvm, - memoryIntrospector.numProcessorsInJvm(), - 0 + memoryIntrospector.memoryPerTask(), + memoryIntrospector.numTasksInJvm(), + memoryIntrospector.numProcessingThreads(), + maxWorkerCount, + 1 ) ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java index bb782cb67d9..fd210776277 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/Limits.java @@ -24,7 +24,7 @@ public class Limits /** * Maximum number of columns that can appear in a frame signature. *

    - * Somewhat less than {@link WorkerMemoryParameters#STANDARD_FRAME_SIZE} divided by typical minimum column size: + * Somewhat less than {@link WorkerMemoryParameters#DEFAULT_FRAME_SIZE} divided by typical minimum column size: * {@link org.apache.druid.frame.allocation.AppendableMemory#DEFAULT_INITIAL_ALLOCATION_SIZE}. */ public static final int MAX_FRAME_COLUMNS = 2000; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MemoryIntrospector.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MemoryIntrospector.java index 337e36d14ef..76fcb33005a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MemoryIntrospector.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MemoryIntrospector.java @@ -19,10 +19,8 @@ package org.apache.druid.msq.exec; -import org.apache.druid.msq.kernel.WorkOrder; - /** - * Introspector used to generate {@link ControllerMemoryParameters}. + * Introspector used to generate {@link WorkerMemoryParameters} and {@link ControllerMemoryParameters}. */ public interface MemoryIntrospector { @@ -32,34 +30,23 @@ public interface MemoryIntrospector long totalMemoryInJvm(); /** - * Amount of memory usable for the multi-stage query engine in the entire JVM. - * - * This may be an expensive operation. For example, the production implementation {@link MemoryIntrospectorImpl} - * estimates size of all lookups as part of computing this value. + * Amount of memory alloted to each {@link Worker} or {@link Controller}. */ - long usableMemoryInJvm(); + long memoryPerTask(); /** - * Amount of total JVM memory required for a particular amount of usable memory to be available. - * - * This may be an expensive operation. For example, the production implementation {@link MemoryIntrospectorImpl} - * estimates size of all lookups as part of computing this value. + * Computes the amount of total JVM memory that would be required for a particular memory allotment per task, i.e., + * a particular return value from {@link #memoryPerTask()}. */ - long computeJvmMemoryRequiredForUsableMemory(long usableMemory); + long computeJvmMemoryRequiredForTaskMemory(long memoryPerTask); /** - * Maximum number of queries that run simultaneously in this JVM. - * - * On workers, this is the maximum number of {@link Worker} that run simultaneously in this JVM. See - * {@link WorkerMemoryParameters} for how memory is divided among and within {@link WorkOrder} run by a worker. - * - * On controllers, this is the maximum number of {@link Controller} that run simultaneously. See - * {@link ControllerMemoryParameters} for how memory is used by controllers. + * Maximum number of tasks ({@link Worker} or {@link Controller}) that run simultaneously in this JVM. */ - int numQueriesInJvm(); + int numTasksInJvm(); /** - * Maximum number of processing threads that can be used at once in this JVM. + * Maximum number of processing threads that can be used at once by each {@link Worker} or {@link Controller}. */ - int numProcessorsInJvm(); + int numProcessingThreads(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MemoryIntrospectorImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MemoryIntrospectorImpl.java index f7cd501ed8f..93d0b9de271 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MemoryIntrospectorImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/MemoryIntrospectorImpl.java @@ -20,12 +20,14 @@ package org.apache.druid.msq.exec; import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.query.lookup.LookupExtractorFactoryContainer; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; +import javax.annotation.Nullable; import java.util.List; /** @@ -34,37 +36,47 @@ import java.util.List; public class MemoryIntrospectorImpl implements MemoryIntrospector { private static final Logger log = new Logger(MemoryIntrospectorImpl.class); + private static final long LOOKUP_FOOTPRINT_INIT = Long.MIN_VALUE; - private final LookupExtractorFactoryContainerProvider lookupProvider; private final long totalMemoryInJvm; - private final int numQueriesInJvm; - private final int numProcessorsInJvm; private final double usableMemoryFraction; + private final int numTasksInJvm; + private final int numProcessingThreads; + + /** + * Lookup footprint per task, set the first time {@link #memoryPerTask()} is called. + */ + private volatile long lookupFootprint = LOOKUP_FOOTPRINT_INIT; + + @Nullable + private final LookupExtractorFactoryContainerProvider lookupProvider; /** * Create an introspector. * - * @param lookupProvider provider of lookups; we use this to subtract lookup size from total JVM memory when - * computing usable memory * @param totalMemoryInJvm maximum JVM heap memory * @param usableMemoryFraction fraction of JVM memory, after subtracting lookup overhead, that we consider usable - * for multi-stage queries - * @param numQueriesInJvm maximum number of {@link Controller} or {@link Worker} that may run concurrently - * @param numProcessorsInJvm size of processing thread pool, typically {@link DruidProcessingConfig#getNumThreads()} + * for {@link Controller} or {@link Worker} + * @param numTasksInJvm maximum number of {@link Controller} or {@link Worker} that may run concurrently + * @param numProcessingThreads size of processing thread pool, typically {@link DruidProcessingConfig#getNumThreads()} + * @param lookupProvider provider of lookups; we use this to subtract lookup size from total JVM memory when + * computing usable memory. Ignored if null. This is used once the first time + * {@link #memoryPerTask()} is called, then the footprint is cached. As such, it provides + * a point-in-time view only. */ public MemoryIntrospectorImpl( - final LookupExtractorFactoryContainerProvider lookupProvider, final long totalMemoryInJvm, final double usableMemoryFraction, - final int numQueriesInJvm, - final int numProcessorsInJvm + final int numTasksInJvm, + final int numProcessingThreads, + @Nullable final LookupExtractorFactoryContainerProvider lookupProvider ) { - this.lookupProvider = lookupProvider; this.totalMemoryInJvm = totalMemoryInJvm; - this.numQueriesInJvm = numQueriesInJvm; - this.numProcessorsInJvm = numProcessorsInJvm; this.usableMemoryFraction = usableMemoryFraction; + this.numTasksInJvm = numTasksInJvm; + this.numProcessingThreads = numProcessingThreads; + this.lookupProvider = lookupProvider; } @Override @@ -74,33 +86,52 @@ public class MemoryIntrospectorImpl implements MemoryIntrospector } @Override - public long usableMemoryInJvm() + public long memoryPerTask() { - final long totalMemory = totalMemoryInJvm(); - final long totalLookupFootprint = computeTotalLookupFootprint(true); return Math.max( 0, - (long) ((totalMemory - totalLookupFootprint) * usableMemoryFraction) + (long) ((totalMemoryInJvm - getTotalLookupFootprint()) * usableMemoryFraction) / numTasksInJvm ); } @Override - public long computeJvmMemoryRequiredForUsableMemory(long usableMemory) + public long computeJvmMemoryRequiredForTaskMemory(long memoryPerTask) { - final long totalLookupFootprint = computeTotalLookupFootprint(false); - return (long) Math.ceil(usableMemory / usableMemoryFraction + totalLookupFootprint); + if (memoryPerTask <= 0) { + throw new IAE("Invalid memoryPerTask[%d], expected a positive number", memoryPerTask); + } + + return (long) Math.ceil(memoryPerTask * numTasksInJvm / usableMemoryFraction) + getTotalLookupFootprint(); } @Override - public int numQueriesInJvm() + public int numTasksInJvm() { - return numQueriesInJvm; + return numTasksInJvm; } @Override - public int numProcessorsInJvm() + public int numProcessingThreads() { - return numProcessorsInJvm; + return numProcessingThreads; + } + + /** + * Get a possibly-cached value of {@link #computeTotalLookupFootprint()}. The underlying computation method is + * called just once, meaning this is not a good way to track the size of lookups over time. This is done to keep + * memory calculations as consistent as possible. + */ + private long getTotalLookupFootprint() + { + if (lookupFootprint == LOOKUP_FOOTPRINT_INIT) { + synchronized (this) { + if (lookupFootprint == LOOKUP_FOOTPRINT_INIT) { + lookupFootprint = computeTotalLookupFootprint(); + } + } + } + + return lookupFootprint; } /** @@ -108,11 +139,13 @@ public class MemoryIntrospectorImpl implements MemoryIntrospector * * Correctness of this approach depends on lookups being loaded *before* calling this method. Luckily, this is the * typical mode of operation, since by default druid.lookup.enableLookupSyncOnStartup = true. - * - * @param logFootprint whether footprint should be logged */ - private long computeTotalLookupFootprint(final boolean logFootprint) + private long computeTotalLookupFootprint() { + if (lookupProvider == null) { + return 0; + } + final List lookupNames = ImmutableList.copyOf(lookupProvider.getAllLookupNames()); long lookupFootprint = 0; @@ -131,10 +164,7 @@ public class MemoryIntrospectorImpl implements MemoryIntrospector } } - if (logFootprint) { - log.info("Lookup footprint: lookup count[%d], total bytes[%,d].", lookupNames.size(), lookupFootprint); - } - + log.info("Lookup footprint: lookup count[%d], total bytes[%,d].", lookupNames.size(), lookupFootprint); return lookupFootprint; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ProcessingBuffers.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ProcessingBuffers.java new file mode 100644 index 00000000000..b12f23be851 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ProcessingBuffers.java @@ -0,0 +1,63 @@ +/* + * 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.exec; + +import org.apache.druid.collections.NonBlockingPool; +import org.apache.druid.collections.QueueNonBlockingPool; +import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.msq.kernel.FrameContext; + +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; + +/** + * Holds a processing buffer pool, and a {@link Bouncer} used to limit concurrent access to the buffer pool. + * Thread-safe. Used by {@link RunWorkOrder} by way of {@link FrameContext#processingBuffers()}. + */ +public class ProcessingBuffers +{ + private final NonBlockingPool bufferPool; + private final Bouncer bouncer; + + public ProcessingBuffers(final NonBlockingPool bufferPool, final Bouncer bouncer) + { + this.bufferPool = bufferPool; + this.bouncer = bouncer; + } + + public static ProcessingBuffers fromCollection(final Collection bufferPool) + { + final BlockingQueue queue = new ArrayBlockingQueue<>(bufferPool.size()); + queue.addAll(bufferPool); + return new ProcessingBuffers(new QueueNonBlockingPool<>(queue), new Bouncer(queue.size())); + } + + public NonBlockingPool getBufferPool() + { + return bufferPool; + } + + public Bouncer getBouncer() + { + return bouncer; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ProcessingBuffersProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ProcessingBuffersProvider.java new file mode 100644 index 00000000000..fb77d1c3078 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ProcessingBuffersProvider.java @@ -0,0 +1,58 @@ +/* + * 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.exec; + +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.msq.kernel.FrameProcessorFactory; +import org.apache.druid.msq.kernel.QueryDefinition; + +/** + * Provides processing buffers for {@link org.apache.druid.msq.kernel.WorkOrder}. Thread-safe, shared by all + * {@link Worker} in a particular JVM. + */ +public interface ProcessingBuffersProvider +{ + /** + * Acquire buffers for a {@link Worker}. + */ + ResourceHolder acquire(int poolSize); + + /** + * Acquire buffers for a {@link Worker}, using a pool size equal to the minimum of + * {@link WorkerContext#maxConcurrentStages()} and the number of stages in the query where + * {@link FrameProcessorFactory#usesProcessingBuffers()}. (These are both caps on the number of concurrent + * stages that will need processing buffers at once.) + */ + default ResourceHolder acquire( + final QueryDefinition queryDef, + final int maxConcurrentStages + ) + { + final int poolSize = Math.min( + maxConcurrentStages, + (int) queryDef.getStageDefinitions() + .stream() + .filter(stageDef -> stageDef.getProcessorFactory().usesProcessingBuffers()) + .count() + ); + + return acquire(poolSize); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ProcessingBuffersSet.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ProcessingBuffersSet.java new file mode 100644 index 00000000000..7f81a9c4a9c --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ProcessingBuffersSet.java @@ -0,0 +1,92 @@ +/* + * 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.exec; + +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.error.DruidException; +import org.apache.druid.msq.kernel.StageDefinition; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.stream.Collectors; + +/** + * Holds a set of {@link ProcessingBuffers} for a {@link Worker}. Acquired from {@link ProcessingBuffersProvider}. + */ +public class ProcessingBuffersSet +{ + public static final ProcessingBuffersSet EMPTY = new ProcessingBuffersSet(Collections.emptyList()); + + private final BlockingQueue pool; + + public ProcessingBuffersSet(Collection buffers) + { + this.pool = new ArrayBlockingQueue<>(buffers.isEmpty() ? 1 : buffers.size()); + this.pool.addAll(buffers); + } + + /** + * Equivalent to calling {@link ProcessingBuffers#fromCollection} on each collection in the overall collection, + * then creating an instance. + */ + public static > ProcessingBuffersSet fromCollection(final Collection processingBuffers) + { + return new ProcessingBuffersSet( + processingBuffers.stream() + .map(ProcessingBuffers::fromCollection) + .collect(Collectors.toList()) + ); + } + + @Nullable + public ResourceHolder acquireForStage(final StageDefinition stageDef) + { + if (!stageDef.getProcessorFactory().usesProcessingBuffers()) { + return null; + } + + final ProcessingBuffers buffers = pool.poll(); + + if (buffers == null) { + // Never happens, because the pool acquired from ProcessingBuffersProvider must be big enough for all + // concurrent processing buffer needs. (In other words: if this does happen, it's a bug.) + throw DruidException.defensive("Processing buffers not available"); + } + + return new ResourceHolder() + { + @Override + public ProcessingBuffers get() + { + return buffers; + } + + @Override + public void close() + { + pool.add(buffers); + } + }; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java index a4d6a2180bd..3ad8bf1f29a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrder.java @@ -21,6 +21,7 @@ package org.apache.druid.msq.exec; import com.google.common.base.Function; import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -56,6 +57,7 @@ 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.java.util.common.Either; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -67,6 +69,8 @@ import org.apache.druid.msq.counters.CpuCounters; import org.apache.druid.msq.indexing.CountingOutputChannelFactory; import org.apache.druid.msq.indexing.InputChannelFactory; import org.apache.druid.msq.indexing.InputChannelsImpl; +import org.apache.druid.msq.indexing.error.CanceledFault; +import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.processor.KeyStatisticsCollectionProcessor; import org.apache.druid.msq.input.InputSlice; import org.apache.druid.msq.input.InputSliceReader; @@ -94,7 +98,6 @@ import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.shuffle.output.DurableStorageOutputChannelFactory; import org.apache.druid.msq.statistics.ClusterByStatisticsCollector; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; -import org.apache.druid.utils.CloseableUtils; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import javax.annotation.Nullable; @@ -104,7 +107,8 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; /** @@ -112,7 +116,29 @@ import java.util.stream.Collectors; */ public class RunWorkOrder { - private final String controllerTaskId; + enum State + { + /** + * Initial state. Must be in this state to call {@link #startAsync()}. + */ + INIT, + + /** + * State entered upon calling {@link #startAsync()}. + */ + STARTED, + + /** + * State entered upon calling {@link #stop(Throwable)}. + */ + STOPPING, + + /** + * State entered when a call to {@link #stop(Throwable)} concludes. + */ + STOPPED + } + private final WorkOrder workOrder; private final InputChannelFactory inputChannelFactory; private final CounterTracker counterTracker; @@ -125,7 +151,9 @@ public class RunWorkOrder private final boolean reindex; private final boolean removeNullBytes; private final ByteTracker intermediateSuperSorterLocalStorageTracker; - private final AtomicBoolean started = new AtomicBoolean(); + private final AtomicReference state = new AtomicReference<>(State.INIT); + private final CountDownLatch stopLatch = new CountDownLatch(1); + private final AtomicReference> resultForListener = new AtomicReference<>(); @MonotonicNonNull private InputSliceReader inputSliceReader; @@ -141,7 +169,6 @@ public class RunWorkOrder private ListenableFuture stageOutputChannelsFuture; public RunWorkOrder( - final String controllerTaskId, final WorkOrder workOrder, final InputChannelFactory inputChannelFactory, final CounterTracker counterTracker, @@ -154,7 +181,6 @@ public class RunWorkOrder final boolean removeNullBytes ) { - this.controllerTaskId = controllerTaskId; this.workOrder = workOrder; this.inputChannelFactory = inputChannelFactory; this.counterTracker = counterTracker; @@ -180,15 +206,16 @@ public class RunWorkOrder * Execution proceeds asynchronously after this method returns. The {@link RunWorkOrderListener} passed to the * constructor of this instance can be used to track progress. */ - public void start() throws IOException + public void startAsync() { - if (started.getAndSet(true)) { - throw new ISE("Already started"); + if (!state.compareAndSet(State.INIT, State.STARTED)) { + throw new ISE("Cannot start from state[%s]", state); } final StageDefinition stageDef = workOrder.getStageDefinition(); try { + exec.registerCancellationId(cancellationId); makeInputSliceReader(); makeWorkOutputChannelFactory(); makeShuffleOutputChannelFactory(); @@ -205,16 +232,86 @@ public class RunWorkOrder setUpCompletionCallbacks(); } catch (Throwable t) { - // If start() has problems, cancel anything that was already kicked off, and close the FrameContext. + stopUnchecked(t); + } + } + + /** + * Stops an execution that was previously initiated through {@link #startAsync()} and closes the {@link FrameContext}. + * May be called to cancel execution. Must also be called after successful execution in order to ensure that resources + * are all properly cleaned up. + * + * Blocks until execution is fully stopped. + * + * @param t error to send to {@link RunWorkOrderListener#onFailure}, if success/failure has not already been sent. + * Will also be thrown at the end of this method. + */ + public void stop(@Nullable Throwable t) throws InterruptedException + { + if (state.compareAndSet(State.INIT, State.STOPPING) + || state.compareAndSet(State.STARTED, State.STOPPING)) { + // Initiate stopping. try { exec.cancel(cancellationId); } - catch (Throwable t2) { - t.addSuppressed(t2); + catch (Throwable e2) { + if (t == null) { + t = e2; + } else { + t.addSuppressed(e2); + } } - CloseableUtils.closeAndSuppressExceptions(frameContext, t::addSuppressed); - throw t; + try { + frameContext.close(); + } + catch (Throwable e2) { + if (t == null) { + t = e2; + } else { + t.addSuppressed(e2); + } + } + + try { + // notifyListener will ignore this error if work has already succeeded. + notifyListener(Either.error(t != null ? t : new MSQException(CanceledFault.instance()))); + } + catch (Throwable e2) { + if (t == null) { + t = e2; + } else { + t.addSuppressed(e2); + } + } + + stopLatch.countDown(); + } + + stopLatch.await(); + + if (t != null) { + Throwables.throwIfInstanceOf(t, InterruptedException.class); + Throwables.throwIfUnchecked(t); + throw new RuntimeException(t); + } + } + + /** + * Calls {@link #stop(Throwable)}. If the call to {@link #stop(Throwable)} throws {@link InterruptedException}, + * this method sets the interrupt flag and throws an unchecked exception. + * + * @param t error to send to {@link RunWorkOrderListener#onFailure}, if success/failure has not already been sent. + * Will also be thrown at the end of this method. + */ + public void stopUnchecked(@Nullable final Throwable t) + { + try { + stop(t); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); } } @@ -242,7 +339,7 @@ public class RunWorkOrder workOrder.getQueryDefinition(), InputSlices.allReadablePartitions(workOrder.getInputs()), inputChannelFactory, - () -> ArenaMemoryAllocator.createOnHeap(frameContext.memoryParameters().getStandardFrameSize()), + () -> ArenaMemoryAllocator.createOnHeap(frameContext.memoryParameters().getFrameSize()), exec, cancellationId, counterTracker, @@ -270,18 +367,8 @@ public class RunWorkOrder final OutputChannelFactory baseOutputChannelFactory; if (workOrder.getStageDefinition().doesShuffle()) { - // Writing to a consumer in the same JVM (which will be set up later on in this method). Use the large frame - // size if we're writing to a SuperSorter, since we'll generate fewer temp files if we use larger frames. - // Otherwise, use the standard frame size. - final int frameSize; - - if (workOrder.getStageDefinition().getShuffleSpec().kind().isSort()) { - frameSize = frameContext.memoryParameters().getLargeFrameSize(); - } else { - frameSize = frameContext.memoryParameters().getStandardFrameSize(); - } - - baseOutputChannelFactory = new BlockingQueueOutputChannelFactory(frameSize); + // Writing to a consumer in the same JVM (which will be set up later on in this method). + baseOutputChannelFactory = new BlockingQueueOutputChannelFactory(frameContext.memoryParameters().getFrameSize()); } else { // Writing stage output. baseOutputChannelFactory = makeStageOutputChannelFactory(); @@ -353,7 +440,7 @@ public class RunWorkOrder final ListenableFuture workResultFuture = exec.runAllFully( counterTracker.trackCpu(processorManager, CpuCounters.LABEL_MAIN), maxOutstandingProcessors, - frameContext.processorBouncer(), + processorFactory.usesProcessingBuffers() ? frameContext.processingBuffers().getBouncer() : Bouncer.unlimited(), cancellationId ); @@ -394,13 +481,13 @@ public class RunWorkOrder if (shuffleSpec.partitionCount() == 1) { // Single partition; no need to write temporary files. hashOutputChannelFactory = - new BlockingQueueOutputChannelFactory(frameContext.memoryParameters().getStandardFrameSize()); + new BlockingQueueOutputChannelFactory(frameContext.memoryParameters().getFrameSize()); } else { // Multi-partition; write temporary files and then sort each one file-by-file. hashOutputChannelFactory = new FileOutputChannelFactory( frameContext.tempDir("hash-parts"), - frameContext.memoryParameters().getStandardFrameSize(), + frameContext.memoryParameters().getFrameSize(), null ); } @@ -469,19 +556,33 @@ public class RunWorkOrder writeDurableStorageSuccessFile(); } - listener.onSuccess(resultObject); + notifyListener(Either.value(resultObject)); } @Override public void onFailure(final Throwable t) { - listener.onFailure(t); + notifyListener(Either.error(t)); } }, Execs.directExecutor() ); } + /** + * Notify {@link RunWorkOrderListener} that the job is done, if not already notified. + */ + private void notifyListener(final Either result) + { + if (resultForListener.compareAndSet(null, result)) { + if (result.isError()) { + listener.onFailure(result.error()); + } else { + listener.onSuccess(result.valueOrThrow()); + } + } + } + /** * Write {@link DurableStorageUtils#SUCCESS_MARKER_FILENAME} for a particular stage, if durable storage is enabled. */ @@ -490,7 +591,7 @@ public class RunWorkOrder final DurableStorageOutputChannelFactory durableStorageOutputChannelFactory = makeDurableStorageOutputChannelFactory( frameContext.tempDir("durable"), - frameContext.memoryParameters().getStandardFrameSize(), + frameContext.memoryParameters().getFrameSize(), workOrder.getOutputChannelMode() == OutputChannelMode.DURABLE_STORAGE_QUERY_RESULTS ); @@ -510,7 +611,7 @@ public class RunWorkOrder { // Use the standard frame size, since we assume this size when computing how much is needed to merge output // files from different workers. - final int frameSize = frameContext.memoryParameters().getStandardFrameSize(); + final int frameSize = frameContext.memoryParameters().getFrameSize(); final OutputChannelMode outputChannelMode = workOrder.getOutputChannelMode(); switch (outputChannelMode) { @@ -542,7 +643,7 @@ public class RunWorkOrder private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory(final File tmpDir) { - final int frameSize = frameContext.memoryParameters().getLargeFrameSize(); + final int frameSize = frameContext.memoryParameters().getFrameSize(); final File fileChannelDirectory = new File(tmpDir, StringUtils.format("intermediate_output_stage_%06d", workOrder.getStageNumber())); final FileOutputChannelFactory fileOutputChannelFactory = @@ -571,7 +672,7 @@ public class RunWorkOrder ) { return DurableStorageOutputChannelFactory.createStandardImplementation( - controllerTaskId, + workerContext.queryId(), workOrder.getWorkerNumber(), workOrder.getStageNumber(), workerContext.workerId(), @@ -736,8 +837,8 @@ public class RunWorkOrder }, outputChannelFactory, makeSuperSorterIntermediateOutputChannelFactory(sorterTmpDir), - memoryParameters.getSuperSorterMaxActiveProcessors(), - memoryParameters.getSuperSorterMaxChannelsPerProcessor(), + memoryParameters.getSuperSorterConcurrentProcessors(), + memoryParameters.getSuperSorterMaxChannelsPerMerger(), stageDefinition.getShuffleSpec().limitHint(), cancellationId, counterTracker.sortProgress(), @@ -774,7 +875,7 @@ public class RunWorkOrder workOrder.getStageDefinition().getFrameReader(), workOrder.getStageDefinition().getClusterBy().getColumns().size(), FrameWriters.makeRowBasedFrameWriterFactory( - new ArenaMemoryAllocatorFactory(frameContext.memoryParameters().getStandardFrameSize()), + new ArenaMemoryAllocatorFactory(frameContext.memoryParameters().getFrameSize()), workOrder.getStageDefinition().getSignature(), workOrder.getStageDefinition().getSortKey(), removeNullBytes diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrderListener.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrderListener.java index 19c3c6570fe..8bffd6f8179 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrderListener.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/RunWorkOrderListener.java @@ -25,7 +25,7 @@ import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import javax.annotation.Nullable; /** - * Listener for various things that may happen during execution of {@link RunWorkOrder#start()}. Listener methods are + * Listener for various things that may happen during execution of {@link RunWorkOrder#startAsync()}. Listener methods are * fired in processing threads, so they must be thread-safe, and it is important that they run quickly. */ public interface RunWorkOrderListener diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java index c327ec340ae..04b08f9346a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/TaskDataSegmentProvider.java @@ -29,10 +29,10 @@ import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.querykit.DataSegmentProvider; +import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.Segment; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.timeline.DataSegment; @@ -70,7 +70,7 @@ public class TaskDataSegmentProvider implements DataSegmentProvider } @Override - public Supplier> fetchSegment( + public Supplier> fetchSegment( final SegmentId segmentId, final ChannelCounters channelCounters, final boolean isReindex @@ -79,7 +79,7 @@ public class TaskDataSegmentProvider implements DataSegmentProvider // Returns Supplier instead of ResourceHolder, so the Coordinator calls and segment downloads happen // in processing threads, rather than the main thread. (They happen when fetchSegmentInternal is called.) return () -> { - ResourceHolder holder = null; + ResourceHolder holder = null; while (holder == null) { holder = holders.computeIfAbsent( @@ -99,7 +99,7 @@ public class TaskDataSegmentProvider implements DataSegmentProvider * Helper used by {@link #fetchSegment(SegmentId, ChannelCounters, boolean)}. Does the actual fetching of a segment, once it * is determined that we definitely need to go out and get one. */ - private ReferenceCountingResourceHolder fetchSegmentInternal( + private ReferenceCountingResourceHolder fetchSegmentInternal( final SegmentId segmentId, final ChannelCounters channelCounters, final boolean isReindex @@ -133,7 +133,7 @@ public class TaskDataSegmentProvider implements DataSegmentProvider final int numRows = index.getNumRows(); final long size = dataSegment.getSize(); closer.register(() -> channelCounters.addFile(numRows, size)); - return new ReferenceCountingResourceHolder<>(segment, closer); + return new ReferenceCountingResourceHolder<>(new CompleteSegment(dataSegment, segment), closer); } catch (IOException | SegmentLoadingException e) { throw CloseableUtils.closeInCatch( @@ -143,13 +143,13 @@ public class TaskDataSegmentProvider implements DataSegmentProvider } } - private static class SegmentHolder implements Supplier> + private static class SegmentHolder implements Supplier> { - private final Supplier> holderSupplier; + private final Supplier> holderSupplier; private final Closeable cleanupFn; @GuardedBy("this") - private ReferenceCountingResourceHolder holder; + private ReferenceCountingResourceHolder holder; @GuardedBy("this") private boolean closing; @@ -157,7 +157,7 @@ public class TaskDataSegmentProvider implements DataSegmentProvider @GuardedBy("this") private boolean closed; - public SegmentHolder(Supplier> holderSupplier, Closeable cleanupFn) + public SegmentHolder(Supplier> holderSupplier, Closeable cleanupFn) { this.holderSupplier = holderSupplier; this.cleanupFn = cleanupFn; @@ -165,7 +165,7 @@ public class TaskDataSegmentProvider implements DataSegmentProvider @Override @Nullable - public ResourceHolder get() + public ResourceHolder get() { synchronized (this) { if (closing) { @@ -183,7 +183,7 @@ public class TaskDataSegmentProvider implements DataSegmentProvider // Then, return null so "fetchSegment" will try again. return null; } else if (holder == null) { - final ResourceHolder segmentHolder = holderSupplier.get(); + final ResourceHolder segmentHolder = holderSupplier.get(); holder = new ReferenceCountingResourceHolder<>( segmentHolder.get(), () -> { @@ -210,7 +210,7 @@ public class TaskDataSegmentProvider implements DataSegmentProvider } } ); - final ResourceHolder retVal = holder.increment(); + final ResourceHolder retVal = holder.increment(); // Store already-closed holder, so it disappears when the last reference is closed. holder.close(); return retVal; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java index 666115d774c..90082fcf0dd 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerContext.java @@ -25,8 +25,8 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.kernel.FrameContext; import org.apache.druid.msq.kernel.FrameProcessorFactory; -import org.apache.druid.msq.kernel.QueryDefinition; import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.server.DruidNode; import java.io.File; @@ -78,14 +78,15 @@ public interface WorkerContext WorkerClient makeWorkerClient(); /** - * Directory for temporary outputs. + * Directory for temporary outputs, used as a base for {@link FrameContext#tempDir()}. This directory is not + * necessarily fully owned by the worker. */ File tempDir(); /** * Create a context with useful objects required by {@link FrameProcessorFactory#makeProcessors}. */ - FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode); + FrameContext frameContext(WorkOrder workOrder); /** * Number of available processing threads. @@ -98,4 +99,9 @@ public interface WorkerContext DruidNode selfNode(); DataServerQueryHandlerFactory dataServerQueryHandlerFactory(); + + /** + * Whether to include all counters in reports. See {@link MultiStageQueryContext#CTX_INCLUDE_ALL_COUNTERS} for detail. + */ + boolean includeAllCounters(); } 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 92664feeabb..7be045542bc 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 @@ -48,7 +48,6 @@ import org.apache.druid.msq.counters.CounterSnapshotsTree; import org.apache.druid.msq.counters.CounterTracker; import org.apache.druid.msq.indexing.InputChannelFactory; import org.apache.druid.msq.indexing.MSQWorkerTask; -import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.indexing.error.CanceledFault; import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault; import org.apache.druid.msq.indexing.error.MSQErrorReport; @@ -203,7 +202,7 @@ public class WorkerImpl implements Worker log.warn("%s", logMessage); if (controllerAlive) { - controllerClient.postWorkerError(context.workerId(), errorReport); + controllerClient.postWorkerError(errorReport); } if (t != null) { @@ -367,34 +366,19 @@ public class WorkerImpl implements Worker final WorkerStageKernel kernel = kernelHolder.kernel; final WorkOrder workOrder = kernel.getWorkOrder(); final StageDefinition stageDefinition = workOrder.getStageDefinition(); - final String cancellationId = cancellationIdFor(stageDefinition.getId()); + final String cancellationId = cancellationIdFor(stageDefinition.getId(), workOrder.getWorkerNumber()); log.info( - "Processing work order for stage[%s]%s", + "Starting work order for stage[%s], workerNumber[%d]%s", stageDefinition.getId(), + workOrder.getWorkerNumber(), (log.isDebugEnabled() ? StringUtils.format(", payload[%s]", context.jsonMapper().writeValueAsString(workOrder)) : "") ); - final FrameContext frameContext = kernelHolder.processorCloser.register( - context.frameContext( - workOrder.getQueryDefinition(), - stageDefinition.getStageNumber(), - workOrder.getOutputChannelMode() - ) - ); - kernelHolder.processorCloser.register(() -> { - try { - workerExec.cancel(cancellationId); - } - catch (InterruptedException e) { - // Strange that cancellation would itself be interrupted. Log and suppress. - log.warn(e, "Cancellation interrupted for stage[%s]", stageDefinition.getId()); - Thread.currentThread().interrupt(); - } - }); + final FrameContext frameContext = context.frameContext(workOrder); - // Set up cleanup functions for this work order. + // Set up resultsCloser (called when we are done reading results). kernelHolder.resultsCloser.register(() -> FileUtils.deleteDirectory(frameContext.tempDir())); kernelHolder.resultsCloser.register(() -> removeStageOutputChannels(stageDefinition.getId())); @@ -403,13 +387,8 @@ public class WorkerImpl implements Worker final InputChannelFactory inputChannelFactory = makeBaseInputChannelFactory(workOrder, controllerClient, kernelHolder.processorCloser); - // Start working on this stage immediately. - kernel.startReading(); - - final QueryContext queryContext = task != null ? QueryContext.of(task.getContext()) : QueryContext.empty(); - final boolean includeAllCounters = MultiStageQueryContext.getIncludeAllCounters(queryContext); + final boolean includeAllCounters = context.includeAllCounters(); final RunWorkOrder runWorkOrder = new RunWorkOrder( - task.getControllerTaskId(), workOrder, inputChannelFactory, stageCounters.computeIfAbsent( @@ -421,11 +400,16 @@ public class WorkerImpl implements Worker context, frameContext, makeRunWorkOrderListener(workOrder, controllerClient, criticalWarningCodes, maxVerboseParseExceptions), - MultiStageQueryContext.isReindex(queryContext), - MultiStageQueryContext.removeNullBytes(queryContext) + MultiStageQueryContext.isReindex(workOrder.getWorkerContext()), + MultiStageQueryContext.removeNullBytes(workOrder.getWorkerContext()) ); - runWorkOrder.start(); + // Set up processorCloser (called when processing is done). + kernelHolder.processorCloser.register(() -> runWorkOrder.stopUnchecked(null)); + + // Start working on this stage immediately. + kernel.startReading(); + runWorkOrder.startAsync(); kernelHolder.partitionBoundariesFuture = runWorkOrder.getStagePartitionBoundariesFuture(); } @@ -574,6 +558,13 @@ public class WorkerImpl implements Worker return getOrCreateStageOutputHolder(stageId, partitionNumber).readRemotelyFrom(offset); } + /** + * Accept a new {@link WorkOrder} for execution. + * + * For backwards-compatibility purposes, this method populates {@link WorkOrder#getOutputChannelMode()} + * and {@link WorkOrder#getWorkerContext()} if the controller did not set them. (They are there for newer controllers, + * but not older ones.) + */ @Override public void postWorkOrder(final WorkOrder workOrder) { @@ -591,28 +582,11 @@ public class WorkerImpl implements Worker ); } - final OutputChannelMode outputChannelMode; + final WorkOrder workOrderToUse = makeWorkOrderToUse( + workOrder, + task != null && task.getContext() != null ? QueryContext.of(task.getContext()) : QueryContext.empty() + ); - // This stack of conditions can be removed once we can rely on OutputChannelMode always being in the WorkOrder. - // (It will be there for newer controllers; this is a backwards-compatibility thing.) - if (workOrder.hasOutputChannelMode()) { - outputChannelMode = workOrder.getOutputChannelMode(); - } else { - final MSQSelectDestination selectDestination = - task != null - ? MultiStageQueryContext.getSelectDestination(QueryContext.of(task.getContext())) - : MSQSelectDestination.TASKREPORT; - - outputChannelMode = ControllerQueryKernelUtils.getOutputChannelMode( - workOrder.getQueryDefinition(), - workOrder.getStageNumber(), - selectDestination, - task != null && MultiStageQueryContext.isDurableStorageEnabled(QueryContext.of(task.getContext())), - false - ); - } - - final WorkOrder workOrderToUse = workOrder.withOutputChannelMode(outputChannelMode); kernelManipulationQueue.add( kernelHolders -> kernelHolders.addKernel(WorkerStageKernel.create(workOrderToUse)) @@ -993,9 +967,9 @@ public class WorkerImpl implements Worker /** * Returns cancellation ID for a particular stage, to be used in {@link FrameProcessorExecutor#cancel(String)}. */ - private static String cancellationIdFor(final StageId stageId) + private static String cancellationIdFor(final StageId stageId, final int workerNumber) { - return stageId.toString(); + return StringUtils.format("%s_%s", stageId, workerNumber); } /** @@ -1023,6 +997,48 @@ public class WorkerImpl implements Worker ); } + /** + * Returns a work order based on the provided "originalWorkOrder", but where {@link WorkOrder#hasOutputChannelMode()} + * and {@link WorkOrder#hasWorkerContext()} are both true. If the original work order didn't have those fields, they + * are populated from the "taskContext". Otherwise the "taskContext" is ignored. + * + * This method can be removed once we can rely on these fields always being set in the WorkOrder. + * (They will be there for newer controllers; this is a backwards-compatibility method.) + * + * @param originalWorkOrder work order from controller + * @param taskContext task context + */ + static WorkOrder makeWorkOrderToUse(final WorkOrder originalWorkOrder, @Nullable final QueryContext taskContext) + { + // This condition can be removed once we can rely on QueryContext always being in the WorkOrder. + // (It will be there for newer controllers; this is a backwards-compatibility thing.) + final QueryContext queryContext; + if (originalWorkOrder.hasWorkerContext()) { + queryContext = originalWorkOrder.getWorkerContext(); + } else if (taskContext != null) { + queryContext = taskContext; + } else { + queryContext = QueryContext.empty(); + } + + // This stack of conditions can be removed once we can rely on OutputChannelMode always being in the WorkOrder. + // (It will be there for newer controllers; this is a backwards-compatibility thing.) + final OutputChannelMode outputChannelMode; + if (originalWorkOrder.hasOutputChannelMode()) { + outputChannelMode = originalWorkOrder.getOutputChannelMode(); + } else { + outputChannelMode = ControllerQueryKernelUtils.getOutputChannelMode( + originalWorkOrder.getQueryDefinition(), + originalWorkOrder.getStageNumber(), + MultiStageQueryContext.getSelectDestination(queryContext), + MultiStageQueryContext.isDurableStorageEnabled(queryContext), + false + ); + } + + return originalWorkOrder.withWorkerContext(queryContext).withOutputChannelMode(outputChannelMode); + } + /** * Log (at DEBUG level) a string explaining the status of all work assigned to this worker. */ @@ -1250,10 +1266,19 @@ public class WorkerImpl implements Worker private static class KernelHolder { private final WorkerStageKernel kernel; - private final Closer processorCloser; - private final Closer resultsCloser; private SettableFuture partitionBoundariesFuture; + /** + * Closer for processing. This is closed when all processing for a stage has completed. + */ + private final Closer processorCloser; + + /** + * Closer for results. This is closed when results for a stage are no longer needed. Always closed + * *after* {@link #processorCloser} is done closing. + */ + private final Closer resultsCloser; + public KernelHolder(WorkerStageKernel kernel) { this.kernel = kernel; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java deleted file mode 100644 index 415c93a8599..00000000000 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerManagerClient.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.exec; - -import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.msq.indexing.MSQWorkerTask; - -import java.io.Closeable; -import java.util.Map; -import java.util.Set; - -/** - * Generic interface to the "worker manager" mechanism which starts, cancels and monitors worker tasks. - */ -public interface WorkerManagerClient extends Closeable -{ - String run(String taskId, MSQWorkerTask task); - - /** - * @param workerId the task ID - * - * @return a {@code TaskLocation} associated with the task or - * {@code TaskLocation.unknown()} if no associated entry could be found - */ - TaskLocation location(String workerId); - - /** - * Fetches status map corresponding to a group of task ids - */ - Map statuses(Set taskIds); - - /** - * Cancel the task corresponding to the provided workerId - */ - void cancel(String workerId); - - @Override - void close(); -} 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 aeaae030e61..f06d9697764 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 @@ -19,92 +19,66 @@ package org.apache.druid.msq.exec; -import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; import com.google.common.primitives.Ints; -import com.google.inject.Injector; +import it.unimi.dsi.fastutil.ints.IntOpenHashSet; import it.unimi.dsi.fastutil.ints.IntSet; -import org.apache.druid.frame.processor.Bouncer; -import org.apache.druid.indexing.worker.config.WorkerConfig; -import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.SuperSorter; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.NotEnoughMemoryFault; import org.apache.druid.msq.indexing.error.TooManyWorkersFault; -import org.apache.druid.msq.input.InputSpecs; -import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.indexing.processor.KeyStatisticsCollectionProcessor; +import org.apache.druid.msq.indexing.processor.SegmentGeneratorFrameProcessorFactory; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSlices; +import org.apache.druid.msq.input.stage.ReadablePartition; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.kernel.GlobalSortMaxCountShuffleSpec; +import org.apache.druid.msq.kernel.ShuffleSpec; import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.kernel.WorkOrder; 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; -import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; -import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; -import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; +import org.apache.druid.segment.incremental.IncrementalIndex; +import javax.annotation.Nullable; +import java.util.List; import java.util.Objects; /** - * Class for determining how much JVM heap to allocate to various purposes. + * Class for determining how much JVM heap to allocate to various purposes for executing a {@link WorkOrder}. * - * First, we take a chunk out of the total JVM heap that is dedicated for MSQ; see {@link #computeUsableMemoryInJvm}. + * First, we split each worker's memory allotment, given by {@link MemoryIntrospector#memoryPerTask()}, into + * equally-sized "bundles" for each {@link WorkOrder} that may be running simultaneously within the {@link Worker} + * for that {@link WorkOrder}. * - * Then, we carve out some space for each worker that may be running in our JVM; see {@link #memoryPerWorker}. + * Within each bundle, we carve out memory required for buffering broadcast data + * (see {@link #computeBroadcastBufferMemory}) and for concurrently-running processors + * (see {@link #computeProcessorMemory}). * - * Then, we split the rest into "bundles" of equal size; see {@link #memoryPerBundle}. The number of bundles is based - * entirely on server configuration; this makes the calculation robust to different queries running simultaneously in - * the same JVM. - * - * Within each bundle, we split up memory in two different ways: one assuming it'll be used for a - * {@link org.apache.druid.frame.processor.SuperSorter}, and one assuming it'll be used for a regular - * processor. Callers can then use whichever set of allocations makes sense. (We assume no single bundle - * will be used for both purposes.) + * The remainder is called "bundle free memory", a pool of memory that can be used for {@link SuperSorter} or + * {@link SegmentGeneratorFrameProcessorFactory}. The amounts overlap, because the same {@link WorkOrder} never + * does both. */ public class WorkerMemoryParameters { - private static final Logger log = new Logger(WorkerMemoryParameters.class); - /** - * Percent of memory that we allocate to bundles. It is less than 100% because we need to leave some space - * left over for miscellaneous other stuff, and to ensure that GC pressure does not get too high. + * Default size for frames. */ - static final double USABLE_MEMORY_FRACTION = 0.75; + public static final int DEFAULT_FRAME_SIZE = 1_000_000; /** - * Percent of each bundle's memory that we allocate to appenderators. It is less than 100% because appenderators + * Amount of extra memory available for each processing thread, beyond what is needed for input and output + * channels. This memory is used for miscellaneous purposes within the various {@link FrameProcessor}. + */ + private static final long EXTRA_MEMORY_PER_PROCESSOR = 25_000_000; + + /** + * Percent of each bundle's free memory that we allocate to appenderators. It is less than 100% because appenderators * unfortunately have a variety of unaccounted-for memory usage. */ - static final double APPENDERATOR_MEMORY_FRACTION = 0.67; - - /** - * Size for "standard frames", which are used for most purposes, except inputs to super-sorters. - * - * In particular, frames that travel between workers are always the minimum size. This is helpful because it makes - * it easier to compute the amount of memory needed to merge input streams. - */ - private static final int STANDARD_FRAME_SIZE = 1_000_000; - - /** - * Size for "large frames", which are used for inputs and inner channels in to super-sorters. - * - * This is helpful because it minimizes the number of temporary files needed during super-sorting. - */ - private static final int LARGE_FRAME_SIZE = 8_000_000; - - /** - * Minimum amount of bundle memory available for processing (i.e., total bundle size minus the amount - * needed for input channels). This memory is guaranteed to be available for things like segment generation - * and broadcast data. - */ - public static final long PROCESSING_MINIMUM_BYTES = 25_000_000; - - /** - * Maximum amount of parallelism for the super-sorter. Higher amounts of concurrency tend to be wasteful. - */ - private static final int MAX_SUPER_SORTER_PROCESSORS = 4; - - /** - * Each super-sorter must have at least 1 processor with 2 input frames and 1 output frame. That's 3 total. - */ - private static final int MIN_SUPER_SORTER_FRAMES = 3; + private static final double APPENDERATOR_BUNDLE_FREE_MEMORY_FRACTION = 0.67; /** * (Very) rough estimate of the on-heap overhead of reading a column. @@ -112,256 +86,214 @@ public class WorkerMemoryParameters private static final int APPENDERATOR_MERGE_ROUGH_MEMORY_PER_COLUMN = 3_000; /** - * Maximum percent of *total* available memory (not each bundle), i.e. {@link #USABLE_MEMORY_FRACTION}, that we'll - * ever use for maxRetainedBytes of {@link ClusterByStatisticsCollectorImpl} across all workers. + * Maximum percent of each bundle's free memory that will be used for maxRetainedBytes of + * {@link ClusterByStatisticsCollectorImpl}. */ - private static final double PARTITION_STATS_MEMORY_MAX_FRACTION = 0.1; + private static final double PARTITION_STATS_MAX_BUNDLE_FREE_MEMORY_FRACTION = 0.1; /** - * Maximum number of bytes we'll ever use for maxRetainedBytes of {@link ClusterByStatisticsCollectorImpl} for - * a single worker. Acts as a limit on the value computed based on {@link #PARTITION_STATS_MEMORY_MAX_FRACTION}. + * Maximum number of bytes from each bundle's free memory that we'll ever use for maxRetainedBytes of + * {@link ClusterByStatisticsCollectorImpl}. Limits the value computed based on + * {@link #PARTITION_STATS_MAX_BUNDLE_FREE_MEMORY_FRACTION}. */ - private static final long PARTITION_STATS_MEMORY_MAX_BYTES = 300_000_000; + private static final long PARTITION_STATS_MAX_MEMORY_PER_BUNDLE = 300_000_000; /** - * Threshold in bytes below which we assume that the worker is "small". While calculating the memory requirements for - * a small worker, we try to be as conservatives with the estimates and the extra temporary space required by the - * frames, since that can add up quickly and cause OOM. + * Minimum number of bytes from each bundle's free memory that we'll use for maxRetainedBytes of + * {@link ClusterByStatisticsCollectorImpl}. */ - private static final long SMALL_WORKER_CAPACITY_THRESHOLD_BYTES = 256_000_000; + private static final long PARTITION_STATS_MIN_MEMORY_PER_BUNDLE = 10_000_000; /** - * 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. + * Fraction of each bundle's total memory that can be used to buffer broadcast inputs. This is used by + * {@link BroadcastJoinSegmentMapFnProcessor} to limit how much joinable data is stored on-heap. This is carved + * directly out of the total bundle memory, which makes its size more predictable and stable: it only depends on + * the total JVM memory, the number of tasks per JVM, and the value of maxConcurrentStages for the query. This + * stability is important, because if the broadcast buffer fills up, the query fails. So any time its size changes, + * we risk queries failing that would formerly have succeeded. */ - static final double BROADCAST_JOIN_MEMORY_FRACTION = 0.3; + private static final double BROADCAST_BUFFER_TOTAL_MEMORY_FRACTION = 0.2; /** - * Fraction of free memory per bundle that can be used by + * Multiplier to apply to {@link #BROADCAST_BUFFER_TOTAL_MEMORY_FRACTION} when determining how much free bundle + * memory is left over. This fudge factor exists because {@link BroadcastJoinSegmentMapFnProcessor} applies data + * size limits based on frame size, which we expect to expand somewhat in memory due to indexing structures in + * {@link org.apache.druid.segment.join.table.FrameBasedIndexedTable}. + */ + private static final double BROADCAST_BUFFER_OVERHEAD_RATIO = 1.5; + + /** + * Amount of memory that can be used by * {@link org.apache.druid.msq.querykit.common.SortMergeJoinFrameProcessor} to buffer frames in its trackers. */ - static final double SORT_MERGE_JOIN_MEMORY_FRACTION = 0.9; + private static final long SORT_MERGE_JOIN_MEMORY_PER_PROCESSOR = (long) (EXTRA_MEMORY_PER_PROCESSOR * 0.9); - /** - * In case {@link NotEnoughMemoryFault} is thrown, a fixed estimation overhead is added when estimating total memory required for the process. - */ - private static final long BUFFER_BYTES_FOR_ESTIMATION = 1000; - - private final long processorBundleMemory; - private final int superSorterMaxActiveProcessors; - private final int superSorterMaxChannelsPerProcessor; + private final long bundleFreeMemory; + private final int frameSize; + private final int superSorterConcurrentProcessors; + private final int superSorterMaxChannelsPerMerger; private final int partitionStatisticsMaxRetainedBytes; + private final long broadcastBufferMemory; - WorkerMemoryParameters( - final long processorBundleMemory, - final int superSorterMaxActiveProcessors, - final int superSorterMaxChannelsPerProcessor, - final int partitionStatisticsMaxRetainedBytes + public WorkerMemoryParameters( + final long bundleFreeMemory, + final int frameSize, + final int superSorterConcurrentProcessors, + final int superSorterMaxChannelsPerMerger, + final int partitionStatisticsMaxRetainedBytes, + final long broadcastBufferMemory ) { - this.processorBundleMemory = processorBundleMemory; - this.superSorterMaxActiveProcessors = superSorterMaxActiveProcessors; - this.superSorterMaxChannelsPerProcessor = superSorterMaxChannelsPerProcessor; + this.bundleFreeMemory = bundleFreeMemory; + this.frameSize = frameSize; + this.superSorterConcurrentProcessors = superSorterConcurrentProcessors; + this.superSorterMaxChannelsPerMerger = superSorterMaxChannelsPerMerger; this.partitionStatisticsMaxRetainedBytes = partitionStatisticsMaxRetainedBytes; + this.broadcastBufferMemory = broadcastBufferMemory; } /** - * Create a production instance for {@link org.apache.druid.msq.indexing.MSQWorkerTask}. + * Create a production instance for a given {@link WorkOrder}. */ - public static WorkerMemoryParameters createProductionInstanceForWorker( - final Injector injector, - final QueryDefinition queryDef, - final int stageNumber, + public static WorkerMemoryParameters createProductionInstance( + final WorkOrder workOrder, + final MemoryIntrospector memoryIntrospector, final int maxConcurrentStages ) { - final StageDefinition stageDef = queryDef.getStageDefinition(stageNumber); - final IntSet inputStageNumbers = InputSpecs.getStageNumbers(stageDef.getInputSpecs()); - final int numInputWorkers = - inputStageNumbers.intStream() - .map(inputStageNumber -> queryDef.getStageDefinition(inputStageNumber).getMaxWorkerCount()) - .sum(); - long totalLookupFootprint = computeTotalLookupFootprint(injector); - - final int numHashOutputPartitions; - if (stageDef.doesShuffle() && stageDef.getShuffleSpec().kind().isHash()) { - numHashOutputPartitions = stageDef.getShuffleSpec().partitionCount(); - } else { - numHashOutputPartitions = 0; - } - + final StageDefinition stageDef = workOrder.getStageDefinition(); return createInstance( - Runtime.getRuntime().maxMemory(), - computeNumWorkersInJvm(injector), - computeNumProcessorsInJvm(injector), + memoryIntrospector, + DEFAULT_FRAME_SIZE, + workOrder.getInputs(), + stageDef.getBroadcastInputNumbers(), + stageDef.doesShuffle() ? stageDef.getShuffleSpec() : null, maxConcurrentStages, - numInputWorkers, - numHashOutputPartitions, - totalLookupFootprint + computeFramesPerOutputChannel(workOrder.getOutputChannelMode()) ); } /** - * Returns an object specifying memory-usage parameters. + * Returns an object specifying memory-usage parameters for a {@link WorkOrder} running inside a {@link Worker}. * * Throws a {@link MSQException} with an appropriate fault if the provided combination of parameters cannot * yield a workable memory situation. * - * @param maxMemoryInJvm memory available in the entire JVM. This will be divided amongst processors. - * @param numWorkersInJvm number of workers that can run concurrently in this JVM. Generally equal to - * the task capacity. - * @param numProcessingThreadsInJvm size of the processing thread pool in the JVM. - * @param maxConcurrentStages maximum number of concurrent stages per worker. - * @param numInputWorkers total number of workers across all input stages. - * @param numHashOutputPartitions total number of output partitions, if using hash partitioning; zero if not using - * hash partitioning. - * @param totalLookupFootprint estimated size of the lookups loaded by the process. + * @param memoryIntrospector memory introspector + * @param frameSize frame size + * @param inputSlices from {@link WorkOrder#getInputs()} + * @param broadcastInputNumbers from {@link StageDefinition#getBroadcastInputNumbers()} + * @param shuffleSpec from {@link StageDefinition#getShuffleSpec()} + * @param maxConcurrentStages figure from {@link WorkerContext#maxConcurrentStages()} + * @param numFramesPerOutputChannel figure from {@link #computeFramesPerOutputChannel(OutputChannelMode)} + * + * @throws MSQException with {@link TooManyWorkersFault} or {@link NotEnoughMemoryFault} if not enough memory + * is available to generate a usable instance */ public static WorkerMemoryParameters createInstance( - final long maxMemoryInJvm, - final int numWorkersInJvm, - final int numProcessingThreadsInJvm, + final MemoryIntrospector memoryIntrospector, + final int frameSize, + final List inputSlices, + final IntSet broadcastInputNumbers, + @Nullable final ShuffleSpec shuffleSpec, final int maxConcurrentStages, - final int numInputWorkers, - final int numHashOutputPartitions, - final long totalLookupFootprint + final int numFramesPerOutputChannel ) { - Preconditions.checkArgument(maxMemoryInJvm > 0, "Max memory passed: [%s] should be > 0", maxMemoryInJvm); - Preconditions.checkArgument(numWorkersInJvm > 0, "Number of workers: [%s] in jvm should be > 0", numWorkersInJvm); - Preconditions.checkArgument( - numProcessingThreadsInJvm > 0, - "Number of processing threads [%s] should be > 0", - numProcessingThreadsInJvm + final long bundleMemory = computeBundleMemory(memoryIntrospector.memoryPerTask(), maxConcurrentStages); + final long processorMemory = computeProcessorMemory( + computeMaxSimultaneousInputChannelsPerProcessor(inputSlices, broadcastInputNumbers), + frameSize ); - Preconditions.checkArgument(numInputWorkers >= 0, "Number of input workers: [%s] should be >=0", numInputWorkers); - Preconditions.checkArgument( - totalLookupFootprint >= 0, - "Lookup memory footprint: [%s] should be >= 0", - totalLookupFootprint - ); - final long usableMemoryInJvm = computeUsableMemoryInJvm(maxMemoryInJvm, totalLookupFootprint); - final long workerMemory = memoryPerWorker(usableMemoryInJvm, numWorkersInJvm); - final long bundleMemory = - memoryPerBundle(usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm) / maxConcurrentStages; - final long bundleMemoryForInputChannels = memoryNeededForInputChannels(numInputWorkers); - final long bundleMemoryForHashPartitioning = memoryNeededForHashPartitioning(numHashOutputPartitions); - final long bundleMemoryForProcessing = - bundleMemory - bundleMemoryForInputChannels - bundleMemoryForHashPartitioning; + final boolean hasBroadcastInputs = !broadcastInputNumbers.isEmpty(); + final long broadcastBufferMemory = + hasBroadcastInputs ? computeBroadcastBufferMemoryIncludingOverhead(bundleMemory) : 0; + final int numProcessingThreads = memoryIntrospector.numProcessingThreads(); + final int maxSimultaneousWorkProcessors = Math.min(numProcessingThreads, computeNumInputPartitions(inputSlices)); + final long bundleFreeMemory = + bundleMemory - maxSimultaneousWorkProcessors * processorMemory - broadcastBufferMemory; - if (bundleMemoryForProcessing < PROCESSING_MINIMUM_BYTES) { - final int maxWorkers = computeMaxWorkers( - usableMemoryInJvm, - numWorkersInJvm, - numProcessingThreadsInJvm, - maxConcurrentStages, - numHashOutputPartitions - ); - - if (maxWorkers > 0) { - throw new MSQException(new TooManyWorkersFault(numInputWorkers, Math.min(Limits.MAX_WORKERS, maxWorkers))); - } else { - // Not enough memory for even one worker. More of a NotEnoughMemory situation than a TooManyWorkers situation. - throw new MSQException( - new NotEnoughMemoryFault( - calculateSuggestedMinMemoryFromUsableMemory( - estimateUsableMemory( - numWorkersInJvm, - numProcessingThreadsInJvm, - PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels, - maxConcurrentStages - ), totalLookupFootprint), - maxMemoryInJvm, - usableMemoryInJvm, - numWorkersInJvm, - numProcessingThreadsInJvm, - maxConcurrentStages - ) - ); - } - } - - // Compute memory breakdown for super-sorting bundles. - final int maxNumFramesForSuperSorter = Ints.checkedCast(bundleMemory / WorkerMemoryParameters.LARGE_FRAME_SIZE); - - if (maxNumFramesForSuperSorter < MIN_SUPER_SORTER_FRAMES) { + final long minimumBundleFreeMemory = computeMinimumBundleFreeMemory(frameSize, numFramesPerOutputChannel); + if (bundleFreeMemory < minimumBundleFreeMemory) { + final long requiredTaskMemory = (bundleMemory - bundleFreeMemory + minimumBundleFreeMemory) * maxConcurrentStages; throw new MSQException( new NotEnoughMemoryFault( - calculateSuggestedMinMemoryFromUsableMemory( - estimateUsableMemory( - numWorkersInJvm, - (MIN_SUPER_SORTER_FRAMES + BUFFER_BYTES_FOR_ESTIMATION) * LARGE_FRAME_SIZE, - maxConcurrentStages - ), - totalLookupFootprint - ), - maxMemoryInJvm, - usableMemoryInJvm, - numWorkersInJvm, - numProcessingThreadsInJvm, + memoryIntrospector.computeJvmMemoryRequiredForTaskMemory(requiredTaskMemory), + memoryIntrospector.totalMemoryInJvm(), + memoryIntrospector.memoryPerTask(), + memoryIntrospector.numTasksInJvm(), + memoryIntrospector.numProcessingThreads(), + computeNumInputWorkers(inputSlices), maxConcurrentStages ) ); } - final int superSorterMaxActiveProcessors = Math.min( - numProcessingThreadsInJvm, - Math.min( - maxNumFramesForSuperSorter / MIN_SUPER_SORTER_FRAMES, - MAX_SUPER_SORTER_PROCESSORS - ) - ); + // Compute memory breakdown for super-sorting bundles. + final int partitionStatsMemory = + StageDefinition.mustGatherResultKeyStatistics(shuffleSpec) ? computePartitionStatsMemory(bundleFreeMemory) : 0; + final long superSorterMemory = bundleFreeMemory - partitionStatsMemory; + final int maxOutputPartitions = computeMaxOutputPartitions(shuffleSpec); - final int isSmallWorker = usableMemoryInJvm < SMALL_WORKER_CAPACITY_THRESHOLD_BYTES ? 1 : 0; - // Apportion max frames to all processors equally, then subtract one to account for an output frame and one to account - // for the durable storage's output frame in the supersorter. The extra frame is required in case of durable storage - // since composing output channel factories keep a frame open while writing to them. - // We only account for this extra frame in the workers where the heap size is relatively small to be more - // conservative with the memory estimations. In workers with heap size larger than the frame size, we can get away - // without accounting for this extra frame, and instead better parallelize the supersorter's operations. - final int superSorterMaxChannelsPerProcessor = maxNumFramesForSuperSorter / superSorterMaxActiveProcessors - - 1 - - isSmallWorker; - if (superSorterMaxActiveProcessors <= 0) { + int superSorterConcurrentProcessors; + int superSorterMaxChannelsPerMerger = -1; + + if (maxOutputPartitions == 0) { + superSorterConcurrentProcessors = numProcessingThreads; + } else { + superSorterConcurrentProcessors = Math.min(maxOutputPartitions, numProcessingThreads); + } + + for (; superSorterConcurrentProcessors > 0; superSorterConcurrentProcessors--) { + final long memoryPerProcessor = superSorterMemory / superSorterConcurrentProcessors; + + // Each processor has at least 2 frames for inputs, plus numFramesPerOutputChannel for outputs. + // Compute whether we can support this level of parallelism, given these constraints. + final int minMemoryForInputsPerProcessor = 2 * frameSize; + final int memoryForOutputsPerProcessor = numFramesPerOutputChannel * frameSize; + + if (memoryPerProcessor >= minMemoryForInputsPerProcessor + memoryForOutputsPerProcessor) { + final long memoryForInputsPerProcessor = memoryPerProcessor - memoryForOutputsPerProcessor; + superSorterMaxChannelsPerMerger = Ints.checkedCast(memoryForInputsPerProcessor / frameSize); + break; + } + } + + if (superSorterConcurrentProcessors == 0) { + // Couldn't support any level of concurrency. Not expected, since we should have accounted for at least a + // minimally-sized SuperSorter by way of the calculation in "computeMinimumBundleFreeMemory". Return a + // NotEnoughMemoryFault with no suggestedServerMemory, since at this point, we aren't sure what will work. throw new MSQException( new NotEnoughMemoryFault( - calculateSuggestedMinMemoryFromUsableMemory( - estimateUsableMemory( - numWorkersInJvm, - numProcessingThreadsInJvm, - PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels, - maxConcurrentStages - ), totalLookupFootprint), - maxMemoryInJvm, - usableMemoryInJvm, - numWorkersInJvm, - numProcessingThreadsInJvm, + 0, + memoryIntrospector.totalMemoryInJvm(), + memoryIntrospector.memoryPerTask(), + memoryIntrospector.numTasksInJvm(), + memoryIntrospector.numProcessingThreads(), + computeNumInputWorkers(inputSlices), maxConcurrentStages ) ); } return new WorkerMemoryParameters( - bundleMemoryForProcessing, - superSorterMaxActiveProcessors, - superSorterMaxChannelsPerProcessor, - - // 100% of worker memory is devoted to partition statistics - Ints.checkedCast(workerMemory / maxConcurrentStages) + bundleFreeMemory, + frameSize, + superSorterConcurrentProcessors, + superSorterMaxChannelsPerMerger, + Math.min(Integer.MAX_VALUE, partitionStatsMemory / numProcessingThreads), + hasBroadcastInputs ? computeBroadcastBufferMemory(bundleMemory) : 0 ); } - public int getSuperSorterMaxActiveProcessors() + public int getSuperSorterConcurrentProcessors() { - return superSorterMaxActiveProcessors; + return superSorterConcurrentProcessors; } - public int getSuperSorterMaxChannelsPerProcessor() + public int getSuperSorterMaxChannelsPerMerger() { - return superSorterMaxChannelsPerProcessor; + return superSorterMaxChannelsPerMerger; } public long getAppenderatorMaxBytesInMemory() @@ -376,24 +308,27 @@ public class WorkerMemoryParameters return Ints.checkedCast(Math.max(2, getAppenderatorMemory() / 2 / APPENDERATOR_MERGE_ROUGH_MEMORY_PER_COLUMN)); } - public int getStandardFrameSize() + public int getFrameSize() { - return STANDARD_FRAME_SIZE; + return frameSize; } - public int getLargeFrameSize() + /** + * Memory available for buffering broadcast data. Used to restrict the amount of memory used by + * {@link BroadcastJoinSegmentMapFnProcessor}. + */ + public long getBroadcastBufferMemory() { - return LARGE_FRAME_SIZE; - } - - public long getBroadcastJoinMemory() - { - return (long) (processorBundleMemory * BROADCAST_JOIN_MEMORY_FRACTION); + return broadcastBufferMemory; } + /** + * Fraction of each processor's memory that can be used by + * {@link org.apache.druid.msq.querykit.common.SortMergeJoinFrameProcessor} to buffer frames in its trackers. + */ public long getSortMergeJoinMemory() { - return (long) (processorBundleMemory * SORT_MERGE_JOIN_MEMORY_FRACTION); + return SORT_MERGE_JOIN_MEMORY_PER_PROCESSOR; } public int getPartitionStatisticsMaxRetainedBytes() @@ -406,7 +341,7 @@ public class WorkerMemoryParameters */ private long getAppenderatorMemory() { - return (long) (processorBundleMemory * APPENDERATOR_MEMORY_FRACTION); + return (long) (bundleFreeMemory * APPENDERATOR_BUNDLE_FREE_MEMORY_FRACTION); } @Override @@ -419,20 +354,24 @@ public class WorkerMemoryParameters return false; } WorkerMemoryParameters that = (WorkerMemoryParameters) o; - return processorBundleMemory == that.processorBundleMemory - && superSorterMaxActiveProcessors == that.superSorterMaxActiveProcessors - && superSorterMaxChannelsPerProcessor == that.superSorterMaxChannelsPerProcessor - && partitionStatisticsMaxRetainedBytes == that.partitionStatisticsMaxRetainedBytes; + return bundleFreeMemory == that.bundleFreeMemory + && frameSize == that.frameSize + && superSorterConcurrentProcessors == that.superSorterConcurrentProcessors + && superSorterMaxChannelsPerMerger == that.superSorterMaxChannelsPerMerger + && partitionStatisticsMaxRetainedBytes == that.partitionStatisticsMaxRetainedBytes + && broadcastBufferMemory == that.broadcastBufferMemory; } @Override public int hashCode() { return Objects.hash( - processorBundleMemory, - superSorterMaxActiveProcessors, - superSorterMaxChannelsPerProcessor, - partitionStatisticsMaxRetainedBytes + bundleFreeMemory, + frameSize, + superSorterConcurrentProcessors, + superSorterMaxChannelsPerMerger, + partitionStatisticsMaxRetainedBytes, + broadcastBufferMemory ); } @@ -440,206 +379,205 @@ public class WorkerMemoryParameters public String toString() { return "WorkerMemoryParameters{" + - "processorBundleMemory=" + processorBundleMemory + - ", superSorterMaxActiveProcessors=" + superSorterMaxActiveProcessors + - ", superSorterMaxChannelsPerProcessor=" + superSorterMaxChannelsPerProcessor + + "bundleFreeMemory=" + bundleFreeMemory + + ", frameSize=" + frameSize + + ", superSorterConcurrentProcessors=" + superSorterConcurrentProcessors + + ", superSorterMaxChannelsPerMerger=" + superSorterMaxChannelsPerMerger + ", partitionStatisticsMaxRetainedBytes=" + partitionStatisticsMaxRetainedBytes + + ", broadcastBufferMemory=" + broadcastBufferMemory + '}'; } /** - * Computes the highest value of numInputWorkers, for the given parameters, that can be passed to - * {@link #createInstance} without resulting in a {@link TooManyWorkersFault}. - * - * Returns 0 if no number of workers would be OK. + * Compute the memory allocated to each {@link WorkOrder} within a {@link Worker}. */ - static int computeMaxWorkers( - final long usableMemoryInJvm, - final int numWorkersInJvm, - final int numProcessingThreadsInJvm, - final int maxConcurrentStages, - final int numHashOutputPartitions + static long computeBundleMemory(final long memoryPerWorker, final int maxConcurrentStages) + { + return memoryPerWorker / maxConcurrentStages; + } + + /** + * Compute the memory allocated to {@link KeyStatisticsCollectionProcessor} within each bundle. + */ + static int computePartitionStatsMemory(final long bundleFreeMemory) + { + return Ints.checkedCast( + Math.max( + (long) Math.min( + bundleFreeMemory * PARTITION_STATS_MAX_BUNDLE_FREE_MEMORY_FRACTION, + PARTITION_STATS_MAX_MEMORY_PER_BUNDLE + ), + PARTITION_STATS_MIN_MEMORY_PER_BUNDLE + ) + ); + } + + /** + * Compute the memory limit passed to {@link BroadcastJoinSegmentMapFnProcessor} within each worker bundle. This + * is somewhat lower than {@link #computeBroadcastBufferMemoryIncludingOverhead}, because we expect some overhead on + * top of this limit due to indexing structures. This overhead isn't accounted for by the processor + * {@link BroadcastJoinSegmentMapFnProcessor} itself. + */ + static long computeBroadcastBufferMemory(final long bundleMemory) + { + return (long) (bundleMemory * BROADCAST_BUFFER_TOTAL_MEMORY_FRACTION); + } + + /** + * Memory allocated to {@link BroadcastJoinSegmentMapFnProcessor} within each worker bundle, including + * expected overhead. + */ + static long computeBroadcastBufferMemoryIncludingOverhead(final long bundleMemory) + { + return (long) (computeBroadcastBufferMemory(bundleMemory) * BROADCAST_BUFFER_OVERHEAD_RATIO); + } + + /** + * Memory allocated to each processor within a bundle, including fixed overheads and buffered input and output frames. + * + * @param maxSimultaneousInputChannelsPerProcessor figure from {@link #computeMaxSimultaneousInputChannelsPerProcessor} + * @param frameSize frame size + */ + static long computeProcessorMemory(final int maxSimultaneousInputChannelsPerProcessor, final int frameSize) + { + return EXTRA_MEMORY_PER_PROCESSOR + + computeProcessorMemoryForInputChannels(maxSimultaneousInputChannelsPerProcessor, frameSize) + + frameSize /* output frame */; + } + + /** + * Memory allocated to each processor for reading its inputs. + * + * @param maxSimultaneousInputChannelsPerProcessor figure from {@link #computeMaxSimultaneousInputChannelsPerProcessor} + * @param frameSize frame size + */ + static long computeProcessorMemoryForInputChannels( + final int maxSimultaneousInputChannelsPerProcessor, + final int frameSize ) { - final long bundleMemory = memoryPerBundle(usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm); - - // Compute number of workers that gives us PROCESSING_MINIMUM_BYTES of memory per bundle per concurrent stage, while - // accounting for memoryNeededForInputChannels + memoryNeededForHashPartitioning. - final int isHashing = numHashOutputPartitions > 0 ? 1 : 0; - final long bundleMemoryPerStage = bundleMemory / maxConcurrentStages; - final long maxWorkers = - (bundleMemoryPerStage - PROCESSING_MINIMUM_BYTES) / ((long) STANDARD_FRAME_SIZE * (1 + isHashing)) - 1; - return Math.max(0, Ints.checkedCast(maxWorkers)); + return (long) maxSimultaneousInputChannelsPerProcessor * frameSize; } /** - * Computes the amount of memory needed to read a single partition from a given number of workers. + * Number of input partitions across all {@link StageInputSlice}. */ - static long memoryNeededForInputChannels(final int numInputWorkers) + static int computeNumInputPartitions(final List inputSlices) { - // Workers that read sorted inputs must open all channels at once to do an N-way merge. Calculate memory needs. - // Requirement: one input frame per worker, one buffered output frame. - return (long) STANDARD_FRAME_SIZE * (numInputWorkers + 1); - } + int retVal = 0; - /** - * Maximum number of workers that may exist in the current JVM. - */ - private static int computeNumWorkersInJvm(final Injector injector) - { - final AppenderatorsManager appenderatorsManager = injector.getInstance(AppenderatorsManager.class); - - if (appenderatorsManager instanceof UnifiedIndexerAppenderatorsManager) { - // CliIndexer - return injector.getInstance(WorkerConfig.class).getCapacity(); - } else { - // CliPeon - return 1; + for (final StageInputSlice slice : InputSlices.allStageSlices(inputSlices)) { + retVal += Iterables.size(slice.getPartitions()); } + + return retVal; } /** - * Maximum number of concurrent processors that exist in the current JVM. - */ - private static int computeNumProcessorsInJvm(final Injector injector) - { - return injector.getInstance(Bouncer.class).getMaxCount(); - } - - /** - * Compute the memory allocated to each worker. Includes anything that exists outside of processing bundles. + * Maximum number of input channels that a processor may have open at once, given the provided worker assignment. * - * Today, we only look at one thing: the amount of memory taken up by - * {@link org.apache.druid.msq.statistics.ClusterByStatisticsCollector}. This is the single largest source of memory - * usage outside processing bundles. + * To compute this, we take the maximum number of workers associated with some partition for each slice. Then we sum + * those maxes up for all broadcast slices, and for all non-broadcast slices, and take the max between those two. + * The idea is that processors first read broadcast data, then read non-broadcast data, and during both phases + * they should have at most one partition open from each slice at once. + * + * @param inputSlices object from {@link WorkOrder#getInputs()} + * @param broadcastInputNumbers object from {@link StageDefinition#getBroadcastInputNumbers()} */ - private static long memoryPerWorker( - final long usableMemoryInJvm, - final int numWorkersInJvm + static int computeMaxSimultaneousInputChannelsPerProcessor( + final List inputSlices, + final IntSet broadcastInputNumbers ) { - final long memoryForWorkers = (long) Math.min( - usableMemoryInJvm * PARTITION_STATS_MEMORY_MAX_FRACTION, - numWorkersInJvm * PARTITION_STATS_MEMORY_MAX_BYTES - ); + long totalNonBroadcastInputChannels = 0; + long totalBroadcastInputChannels = 0; - return memoryForWorkers / numWorkersInJvm; - } + final List allStageSlices = InputSlices.allStageSlices(inputSlices); - /** - * Compute the memory allocated to each processing bundle. Any computation changes done to this method should also be - * done in its corresponding method {@link WorkerMemoryParameters#estimateUsableMemory} - */ - private static long memoryPerBundle( - final long usableMemoryInJvm, - final int numWorkersInJvm, - final int numProcessingThreadsInJvm - ) - { - // One bundle per worker + one per processor. The worker bundles are used for sorting (SuperSorter) and the - // processing bundles are used for reading input and doing per-partition processing. - final int bundleCount = numWorkersInJvm + numProcessingThreadsInJvm; + for (int inputNumber = 0; inputNumber < allStageSlices.size(); inputNumber++) { + final StageInputSlice slice = allStageSlices.get(inputNumber); - // Need to subtract memoryForWorkers off the top of usableMemoryInJvm, since this is reserved for - // statistics collection. - final long memoryForWorkers = numWorkersInJvm * memoryPerWorker(usableMemoryInJvm, numWorkersInJvm); - final long memoryForBundles = usableMemoryInJvm - memoryForWorkers; + int maxWorkers = 0; + for (final ReadablePartition partition : slice.getPartitions()) { + maxWorkers = Math.max(maxWorkers, partition.getWorkerNumbers().size()); + } - // Divide up the usable memory per bundle. - return memoryForBundles / bundleCount; - } - - /** - * Used for estimating the usable memory for better exception messages when {@link NotEnoughMemoryFault} is thrown. - */ - private static long estimateUsableMemory( - final int numWorkersInJvm, - final int numProcessingThreadsInJvm, - final long estimatedEachBundleMemory, - final int maxConcurrentStages - ) - { - final int bundleCount = numWorkersInJvm + numProcessingThreadsInJvm; - return estimateUsableMemory(numWorkersInJvm, estimatedEachBundleMemory * bundleCount, maxConcurrentStages); - } - - /** - * Add overheads to the estimated bundle memoery for all the workers. Checkout {@link WorkerMemoryParameters#memoryPerWorker(long, int)} - * for the overhead calculation outside the processing bundles. - */ - private static long estimateUsableMemory( - final int numWorkersInJvm, - final long estimatedTotalBundleMemory, - final int maxConcurrentStages - ) - { - // Currently, we only add the partition stats overhead since it will be the single largest overhead per worker. - final long estimateStatOverHeadPerWorker = PARTITION_STATS_MEMORY_MAX_BYTES; - final long requiredUsableMemory = estimatedTotalBundleMemory + (estimateStatOverHeadPerWorker * numWorkersInJvm); - return requiredUsableMemory * maxConcurrentStages; - } - - private static long memoryNeededForHashPartitioning(final int numOutputPartitions) - { - // One standard frame for each processor output. - // May be zero, since numOutputPartitions is zero if not using hash partitioning. - return (long) STANDARD_FRAME_SIZE * numOutputPartitions; - } - - /** - * Amount of heap memory available for our usage. Any computation changes done to this method should also be done in - * its corresponding method {@link WorkerMemoryParameters#calculateSuggestedMinMemoryFromUsableMemory} - */ - private static long computeUsableMemoryInJvm(final long maxMemory, final long totalLookupFootprint) - { - // Always report at least one byte, to simplify the math in createInstance. - return Math.max( - 1, - (long) ((maxMemory - totalLookupFootprint) * USABLE_MEMORY_FRACTION) - ); - } - - /** - * Estimate amount of heap memory for the given workload to use in case usable memory is provided. This method is used - * for better exception messages when {@link NotEnoughMemoryFault} is thrown. - */ - private static long calculateSuggestedMinMemoryFromUsableMemory(long usuableMemeory, final long totalLookupFootprint) - { - return (long) ((usuableMemeory / USABLE_MEMORY_FRACTION) + totalLookupFootprint); - } - - /** - * Total estimated lookup footprint. Obtained by calling {@link LookupExtractor#estimateHeapFootprint()} on - * all available lookups. - */ - private static long computeTotalLookupFootprint(final Injector injector) - { - // Subtract memory taken up by lookups. Correctness of this operation depends on lookups being loaded *before* - // we create this instance. Luckily, this is the typical mode of operation, since by default - // druid.lookup.enableLookupSyncOnStartup = true. - final LookupExtractorFactoryContainerProvider lookupManager = - injector.getInstance(LookupExtractorFactoryContainerProvider.class); - - int lookupCount = 0; - long lookupFootprint = 0; - - for (final String lookupName : lookupManager.getAllLookupNames()) { - final LookupExtractorFactoryContainer container = lookupManager.get(lookupName).orElse(null); - - if (container != null) { - try { - final LookupExtractor extractor = container.getLookupExtractorFactory().get(); - lookupFootprint += extractor.estimateHeapFootprint(); - lookupCount++; - } - catch (Exception e) { - log.noStackTrace().warn(e, "Failed to load lookup [%s] for size estimation. Skipping.", lookupName); - } + if (broadcastInputNumbers.contains(inputNumber)) { + totalBroadcastInputChannels += maxWorkers; + } else { + totalNonBroadcastInputChannels += maxWorkers; } } - log.debug("Lookup footprint: %d lookups with %,d total bytes.", lookupCount, lookupFootprint); + return Ints.checkedCast(Math.max(totalBroadcastInputChannels, totalNonBroadcastInputChannels)); + } - return lookupFootprint; + + /** + * Distinct number of input workers. + */ + static int computeNumInputWorkers(final List inputSlices) + { + final IntSet workerNumbers = new IntOpenHashSet(); + + for (final StageInputSlice slice : InputSlices.allStageSlices(inputSlices)) { + for (final ReadablePartition partition : slice.getPartitions()) { + workerNumbers.addAll(partition.getWorkerNumbers()); + } + } + + return workerNumbers.size(); + } + + /** + * Maximum number of output channels for a shuffle spec, or 0 if not knowable in advance. + */ + static int computeMaxOutputPartitions(@Nullable final ShuffleSpec shuffleSpec) + { + if (shuffleSpec == null) { + return 0; + } else { + switch (shuffleSpec.kind()) { + case HASH: + case HASH_LOCAL_SORT: + case MIX: + return shuffleSpec.partitionCount(); + + case GLOBAL_SORT: + if (shuffleSpec instanceof GlobalSortMaxCountShuffleSpec) { + return ((GlobalSortMaxCountShuffleSpec) shuffleSpec).getMaxPartitions(); + } + // Fall through + + default: + return 0; + } + } + } + + /** + * Maximum number of output channels for a shuffle spec, or 0 if not knowable in advance. + */ + static int computeFramesPerOutputChannel(final OutputChannelMode outputChannelMode) + { + // If durable storage is enabled, we need one extra frame per output channel. + return outputChannelMode.isDurable() ? 2 : 1; + } + + /** + * Minimum number of bytes for a bundle's free memory allotment. This must be enough to reasonably produce and + * persist an {@link IncrementalIndex}, or to run a {@link SuperSorter} with 1 thread and 2 frames. + */ + static long computeMinimumBundleFreeMemory(final int frameSize, final int numFramesPerOutputChannel) + { + // Some for partition statistics. + long minMemory = PARTITION_STATS_MIN_MEMORY_PER_BUNDLE; + + // Some for a minimally-sized super-sorter. + minMemory += (long) (2 + numFramesPerOutputChannel) * frameSize; + + // That's enough. Don't consider the possibility that the bundle may be used for producing IncrementalIndex, + // because PARTITION_STATS_MIN_MEMORY_PER_BUNDLE more or less covers that. + return minMemory; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/IndexerMemoryManagementModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/IndexerMemoryManagementModule.java index 92f16a631d9..61f03e40ab6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/IndexerMemoryManagementModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/IndexerMemoryManagementModule.java @@ -22,13 +22,15 @@ package org.apache.druid.msq.guice; import com.google.inject.Binder; import com.google.inject.Provides; import org.apache.druid.discovery.NodeRole; -import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.annotations.LoadScope; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.initialization.DruidModule; import org.apache.druid.msq.exec.MemoryIntrospector; import org.apache.druid.msq.exec.MemoryIntrospectorImpl; +import org.apache.druid.msq.exec.ProcessingBuffersProvider; +import org.apache.druid.msq.indexing.IndexerProcessingBuffersProvider; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.utils.JvmUtils; @@ -42,37 +44,51 @@ import org.apache.druid.utils.JvmUtils; public class IndexerMemoryManagementModule implements DruidModule { /** - * Allocate up to 75% of memory for MSQ-related stuff (if all running tasks are MSQ tasks). + * Allocate up to 60% of memory for the MSQ framework (if all running tasks are MSQ tasks). This does not include the + * memory allocated to {@link #PROCESSING_MEMORY_FRACTION}. */ - private static final double USABLE_MEMORY_FRACTION = 0.75; + private static final double MSQ_MEMORY_FRACTION = 0.60; + + /** + * Allocate up to 15% of memory for processing buffers for MSQ tasks. + */ + private static final double PROCESSING_MEMORY_FRACTION = 0.15; @Override public void configure(Binder binder) { - // Nothing to do. + TaskMemoryManagementConfig.bind(binder); } @Provides - @LazySingleton - public Bouncer makeProcessorBouncer(final DruidProcessingConfig processingConfig) - { - return new Bouncer(processingConfig.getNumThreads()); - } - - @Provides - @LazySingleton + @ManageLifecycle public MemoryIntrospector createMemoryIntrospector( final LookupExtractorFactoryContainerProvider lookupProvider, + final TaskMemoryManagementConfig taskMemoryManagementConfig, final DruidProcessingConfig processingConfig, final WorkerConfig workerConfig ) { return new MemoryIntrospectorImpl( - lookupProvider, JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes(), - USABLE_MEMORY_FRACTION, + MSQ_MEMORY_FRACTION, workerConfig.getCapacity(), - processingConfig.getNumThreads() + PeonMemoryManagementModule.getNumThreads(taskMemoryManagementConfig, processingConfig), + lookupProvider + ); + } + + @Provides + @LazySingleton + public ProcessingBuffersProvider createProcessingBuffersProvider( + final MemoryIntrospector memoryIntrospector, + final WorkerConfig workerConfig + ) + { + return new IndexerProcessingBuffersProvider( + (long) (JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes() * PROCESSING_MEMORY_FRACTION), + workerConfig.getCapacity(), + memoryIntrospector.numProcessingThreads() ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java index 5d837940e19..b7239980d52 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQSqlModule.java @@ -27,6 +27,8 @@ import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.annotations.LoadScope; import org.apache.druid.initialization.DruidModule; import org.apache.druid.metadata.input.InputSourceModule; +import org.apache.druid.msq.indexing.destination.MSQTerminalStageSpecFactory; +import org.apache.druid.msq.indexing.destination.SegmentGenerationTerminalStageSpecFactory; import org.apache.druid.msq.sql.MSQTaskSqlEngine; import org.apache.druid.sql.SqlStatementFactory; import org.apache.druid.sql.SqlToolbox; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/PeonMemoryManagementModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/PeonMemoryManagementModule.java index 9e814c08278..39265434584 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/PeonMemoryManagementModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/PeonMemoryManagementModule.java @@ -21,22 +21,30 @@ package org.apache.druid.msq.guice; import com.google.inject.Binder; import com.google.inject.Provides; +import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.discovery.NodeRole; -import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.annotations.Global; import org.apache.druid.guice.annotations.LoadScope; import org.apache.druid.initialization.DruidModule; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.msq.exec.MemoryIntrospector; import org.apache.druid.msq.exec.MemoryIntrospectorImpl; +import org.apache.druid.msq.exec.ProcessingBuffersProvider; +import org.apache.druid.msq.indexing.PeonProcessingBuffersProvider; +import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.utils.JvmUtils; +import java.nio.ByteBuffer; + /** * Provides {@link MemoryIntrospector} for single-task-per-JVM model. * * @see IndexerMemoryManagementModule for multi-task-per-JVM model used on {@link org.apache.druid.cli.CliIndexer} */ @LoadScope(roles = NodeRole.PEON_JSON_NAME) + public class PeonMemoryManagementModule implements DruidModule { /** @@ -45,41 +53,61 @@ public class PeonMemoryManagementModule implements DruidModule private static final int NUM_WORKERS_IN_JVM = 1; /** - * Peons may have more than one processing thread, but we currently only use one of them. - */ - private static final int NUM_PROCESSING_THREADS = 1; - - /** - * Allocate 75% of memory for MSQ-related stuff. + * Allocate 75% of memory for the MSQ framework. */ private static final double USABLE_MEMORY_FRACTION = 0.75; @Override public void configure(Binder binder) { - // Nothing to do. - } - - @Provides - @LazySingleton - public Bouncer makeProcessorBouncer() - { - return new Bouncer(NUM_PROCESSING_THREADS); + TaskMemoryManagementConfig.bind(binder); } @Provides @LazySingleton public MemoryIntrospector createMemoryIntrospector( final LookupExtractorFactoryContainerProvider lookupProvider, - final Bouncer bouncer + final DruidProcessingConfig processingConfig, + final TaskMemoryManagementConfig taskMemoryManagementConfig ) { return new MemoryIntrospectorImpl( - lookupProvider, JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes(), USABLE_MEMORY_FRACTION, NUM_WORKERS_IN_JVM, - bouncer.getMaxCount() + getNumThreads(taskMemoryManagementConfig, processingConfig), + lookupProvider ); } + + @Provides + @LazySingleton + public ProcessingBuffersProvider createProcessingBuffersProvider( + @Global final NonBlockingPool processingPool, + final MemoryIntrospector memoryIntrospector + ) + { + return new PeonProcessingBuffersProvider( + processingPool, + memoryIntrospector.numProcessingThreads() + ); + } + + public static int getNumThreads( + final TaskMemoryManagementConfig taskMemoryManagementConfig, + final DruidProcessingConfig processingConfig + ) + { + if (taskMemoryManagementConfig.getMaxThreads() == TaskMemoryManagementConfig.UNLIMITED) { + return processingConfig.getNumThreads(); + } else if (taskMemoryManagementConfig.getMaxThreads() > 0) { + return Math.min(taskMemoryManagementConfig.getMaxThreads(), processingConfig.getNumThreads()); + } else { + throw new IAE( + "Invalid value of %s.maxThreads[%d]", + TaskMemoryManagementConfig.BASE_PROPERTY, + taskMemoryManagementConfig.getMaxThreads() + ); + } + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/TaskMemoryManagementConfig.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/TaskMemoryManagementConfig.java new file mode 100644 index 00000000000..d8dc278aa16 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/TaskMemoryManagementConfig.java @@ -0,0 +1,51 @@ +/* + * 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.guice; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.inject.Binder; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.java.util.common.StringUtils; + +/** + * Server configuration for {@link PeonMemoryManagementModule} and {@link IndexerMemoryManagementModule}. + */ +public class TaskMemoryManagementConfig +{ + public static final String BASE_PROPERTY = StringUtils.format("%s.task.memory", MSQIndexingModule.BASE_MSQ_KEY); + public static final int UNLIMITED = -1; + + @JsonProperty("maxThreads") + private int maxThreads = 1; + + public static void bind(final Binder binder) + { + JsonConfigProvider.bind( + binder, + BASE_PROPERTY, + TaskMemoryManagementConfig.class + ); + } + + public int getMaxThreads() + { + return maxThreads; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java index 1037aa6c2af..c148e7fc1bb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerControllerContext.java @@ -45,11 +45,13 @@ import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.msq.indexing.error.MSQWarnings; import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.input.InputSpecSlicer; -import org.apache.druid.msq.input.table.TableInputSpecSlicer; -import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.kernel.controller.ControllerQueryKernelConfig; +import org.apache.druid.msq.querykit.QueryKit; +import org.apache.druid.msq.querykit.QueryKitSpec; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.DruidMetrics; +import org.apache.druid.query.Query; import org.apache.druid.query.QueryContext; import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.indexing.OverlordClient; @@ -66,6 +68,8 @@ import java.util.concurrent.TimeUnit; */ public class IndexerControllerContext implements ControllerContext { + public static final int DEFAULT_MAX_CONCURRENT_STAGES = 1; + private static final Logger log = new Logger(IndexerControllerContext.class); private final MSQControllerTask task; @@ -74,6 +78,7 @@ public class IndexerControllerContext implements ControllerContext private final ServiceClientFactory clientFactory; private final OverlordClient overlordClient; private final ServiceMetricEvent.Builder metricBuilder; + private final MemoryIntrospector memoryIntrospector; public IndexerControllerContext( final MSQControllerTask task, @@ -89,27 +94,27 @@ public class IndexerControllerContext implements ControllerContext this.clientFactory = clientFactory; this.overlordClient = overlordClient; this.metricBuilder = new ServiceMetricEvent.Builder(); + this.memoryIntrospector = injector.getInstance(MemoryIntrospector.class); IndexTaskUtils.setTaskDimensions(metricBuilder, task); } @Override public ControllerQueryKernelConfig queryKernelConfig( - final MSQSpec querySpec, - final QueryDefinition queryDef + final String queryId, + final MSQSpec querySpec ) { - final MemoryIntrospector memoryIntrospector = injector.getInstance(MemoryIntrospector.class); final ControllerMemoryParameters memoryParameters = ControllerMemoryParameters.createProductionInstance( memoryIntrospector, - queryDef.getFinalStageDefinition().getMaxWorkerCount() + querySpec.getTuningConfig().getMaxNumWorkers() ); final ControllerQueryKernelConfig config = makeQueryKernelConfig(querySpec, memoryParameters); log.debug( "Query[%s] using %s[%s], %s[%s], %s[%s].", - queryDef.getQueryId(), + queryId, MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, config.isDurableStorage(), MultiStageQueryContext.CTX_FAULT_TOLERANCE, @@ -146,11 +151,11 @@ public class IndexerControllerContext implements ControllerContext } @Override - public InputSpecSlicer newTableInputSpecSlicer() + public InputSpecSlicer newTableInputSpecSlicer(final WorkerManager workerManager) { final SegmentSource includeSegmentSource = MultiStageQueryContext.getSegmentSources(task.getQuerySpec().getQuery().context()); - return new TableInputSpecSlicer( + return new IndexerTableInputSpecSlicer( toolbox.getCoordinatorClient(), toolbox.getTaskActionClient(), includeSegmentSource @@ -200,8 +205,31 @@ public class IndexerControllerContext implements ControllerContext ); } + @Override + public QueryKitSpec makeQueryKitSpec( + final QueryKit> queryKit, + final String queryId, + final MSQSpec querySpec, + final ControllerQueryKernelConfig queryKernelConfig + ) + { + return new QueryKitSpec( + queryKit, + queryId, + querySpec.getTuningConfig().getMaxNumWorkers(), + querySpec.getTuningConfig().getMaxNumWorkers(), + + // Assume tasks are symmetric: workers have the same number of processors available as a controller. + // Create one partition per processor per task, for maximum parallelism. + MultiStageQueryContext.getTargetPartitionsPerWorkerWithDefault( + querySpec.getQuery().context(), + memoryIntrospector.numProcessingThreads() + ) + ); + } + /** - * Helper method for {@link #queryKernelConfig(MSQSpec, QueryDefinition)}. Also used in tests. + * Helper method for {@link #queryKernelConfig(String, MSQSpec)}. Also used in tests. */ public static ControllerQueryKernelConfig makeQueryKernelConfig( final MSQSpec querySpec, @@ -209,7 +237,8 @@ public class IndexerControllerContext implements ControllerContext ) { final QueryContext queryContext = querySpec.getQuery().context(); - final int maxConcurrentStages = MultiStageQueryContext.getMaxConcurrentStages(queryContext); + final int maxConcurrentStages = + MultiStageQueryContext.getMaxConcurrentStagesWithDefault(queryContext, DEFAULT_MAX_CONCURRENT_STAGES); final boolean isFaultToleranceEnabled = MultiStageQueryContext.isFaultToleranceEnabled(queryContext); final boolean isDurableStorageEnabled; @@ -247,9 +276,44 @@ public class IndexerControllerContext implements ControllerContext .destination(querySpec.getDestination()) .maxConcurrentStages(maxConcurrentStages) .maxRetainedPartitionSketchBytes(memoryParameters.getPartitionStatisticsMaxRetainedBytes()) + .workerContextMap(makeWorkerContextMap(querySpec, isDurableStorageEnabled, maxConcurrentStages)) .build(); } + /** + * Helper method for {@link #makeQueryKernelConfig} and {@link #makeTaskContext}. Makes the worker context map, + * i.e., the map that will become {@link WorkOrder#getWorkerContext()}. + */ + public static Map makeWorkerContextMap( + final MSQSpec querySpec, + final boolean durableStorageEnabled, + final int maxConcurrentStages + ) + { + final QueryContext queryContext = querySpec.getQuery().context(); + final long maxParseExceptions = MultiStageQueryContext.getMaxParseExceptions(queryContext); + final boolean removeNullBytes = MultiStageQueryContext.removeNullBytes(queryContext); + final boolean includeAllCounters = MultiStageQueryContext.getIncludeAllCounters(queryContext); + final ImmutableMap.Builder builder = ImmutableMap.builder(); + + builder + .put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, durableStorageEnabled) + .put(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, maxParseExceptions) + .put(MultiStageQueryContext.CTX_IS_REINDEX, MSQControllerTask.isReplaceInputDataSourceTask(querySpec)) + .put(MultiStageQueryContext.CTX_MAX_CONCURRENT_STAGES, maxConcurrentStages) + .put(MultiStageQueryContext.CTX_REMOVE_NULL_BYTES, removeNullBytes) + .put(MultiStageQueryContext.CTX_INCLUDE_ALL_COUNTERS, includeAllCounters); + + if (querySpec.getDestination().toSelectDestination() != null) { + builder.put( + MultiStageQueryContext.CTX_SELECT_DESTINATION, + querySpec.getDestination().toSelectDestination().getName() + ); + } + + return builder.build(); + } + /** * Helper method for {@link #newWorkerManager}, split out to be used in tests. * @@ -262,17 +326,16 @@ public class IndexerControllerContext implements ControllerContext ) { final ImmutableMap.Builder taskContextOverridesBuilder = ImmutableMap.builder(); - final long maxParseExceptions = MultiStageQueryContext.getMaxParseExceptions(querySpec.getQuery().context()); - final boolean removeNullBytes = MultiStageQueryContext.removeNullBytes(querySpec.getQuery().context()); - final boolean includeAllCounters = MultiStageQueryContext.getIncludeAllCounters(querySpec.getQuery().context()); - taskContextOverridesBuilder - .put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, queryKernelConfig.isDurableStorage()) - .put(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, maxParseExceptions) - .put(MultiStageQueryContext.CTX_IS_REINDEX, MSQControllerTask.isReplaceInputDataSourceTask(querySpec)) - .put(MultiStageQueryContext.CTX_MAX_CONCURRENT_STAGES, queryKernelConfig.getMaxConcurrentStages()) - .put(MultiStageQueryContext.CTX_REMOVE_NULL_BYTES, removeNullBytes) - .put(MultiStageQueryContext.CTX_INCLUDE_ALL_COUNTERS, includeAllCounters); + // Put worker context into the task context. That way, workers can get these context keys either from + // WorkOrder#getContext or Task#getContext. + taskContextOverridesBuilder.putAll( + makeWorkerContextMap( + querySpec, + queryKernelConfig.isDurableStorage(), + queryKernelConfig.getMaxConcurrentStages() + ) + ); // Put the lookup loading info in the task context to facilitate selective loading of lookups. if (controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE) != null) { @@ -288,13 +351,6 @@ public class IndexerControllerContext implements ControllerContext ); } - if (querySpec.getDestination().toSelectDestination() != null) { - taskContextOverridesBuilder.put( - MultiStageQueryContext.CTX_SELECT_DESTINATION, - querySpec.getDestination().toSelectDestination().getName() - ); - } - // propagate the controller's tags to the worker task for enhanced metrics reporting @SuppressWarnings("unchecked") Map tags = (Map) controllerTaskContext.get(DruidMetrics.TAGS); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java index fb6e4a0079f..e8f3739facb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerFrameContext.java @@ -20,9 +20,11 @@ package org.apache.druid.msq.indexing; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; +import org.apache.druid.msq.exec.ProcessingBuffers; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.kernel.FrameContext; @@ -35,6 +37,7 @@ import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.loading.DataSegmentPusher; +import javax.annotation.Nullable; import java.io.File; public class IndexerFrameContext implements FrameContext @@ -43,6 +46,8 @@ public class IndexerFrameContext implements FrameContext private final IndexerWorkerContext context; private final IndexIO indexIO; private final DataSegmentProvider dataSegmentProvider; + @Nullable + private final ResourceHolder processingBuffers; private final WorkerMemoryParameters memoryParameters; private final WorkerStorageParameters storageParameters; private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory; @@ -52,6 +57,7 @@ public class IndexerFrameContext implements FrameContext IndexerWorkerContext context, IndexIO indexIO, DataSegmentProvider dataSegmentProvider, + @Nullable ResourceHolder processingBuffers, DataServerQueryHandlerFactory dataServerQueryHandlerFactory, WorkerMemoryParameters memoryParameters, WorkerStorageParameters storageParameters @@ -61,6 +67,7 @@ public class IndexerFrameContext implements FrameContext this.context = context; this.indexIO = indexIO; this.dataSegmentProvider = dataSegmentProvider; + this.processingBuffers = processingBuffers; this.memoryParameters = memoryParameters; this.storageParameters = storageParameters; this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory; @@ -135,15 +142,19 @@ public class IndexerFrameContext implements FrameContext } @Override - public WorkerMemoryParameters memoryParameters() + public ProcessingBuffers processingBuffers() { - return memoryParameters; + if (processingBuffers != null) { + return processingBuffers.get(); + } else { + throw new ISE("No processing buffers"); + } } @Override - public Bouncer processorBouncer() + public WorkerMemoryParameters memoryParameters() { - return context.injector().getInstance(Bouncer.class); + return memoryParameters; } @Override @@ -155,6 +166,8 @@ public class IndexerFrameContext implements FrameContext @Override public void close() { - // Nothing to close. + if (processingBuffers != null) { + processingBuffers.close(); + } } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerProcessingBuffersProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerProcessingBuffersProvider.java new file mode 100644 index 00000000000..dcf499c3f2f --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerProcessingBuffersProvider.java @@ -0,0 +1,89 @@ +/* + * 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.indexing; + +import org.apache.druid.cli.CliIndexer; +import org.apache.druid.collections.ReferenceCountingResourceHolder; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.exec.ProcessingBuffersProvider; +import org.apache.druid.msq.exec.ProcessingBuffersSet; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +/** + * Production implementation of {@link ProcessingBuffersProvider} for tasks in {@link CliIndexer}. + */ +public class IndexerProcessingBuffersProvider implements ProcessingBuffersProvider +{ + private static final int MIN_BUFFER_SIZE = 1_000_000; + + private final long heapMemoryToUse; + private final int taskCapacity; + private final int numThreads; + + public IndexerProcessingBuffersProvider(final long heapMemoryToUse, final int taskCapacity, final int numThreads) + { + this.heapMemoryToUse = heapMemoryToUse; + this.taskCapacity = taskCapacity; + this.numThreads = numThreads; + } + + @Override + public ResourceHolder acquire(int poolSize) + { + if (poolSize == 0) { + return new ReferenceCountingResourceHolder<>(ProcessingBuffersSet.EMPTY, () -> {}); + } + + final long heapMemoryPerWorker = heapMemoryToUse / taskCapacity; + final int numThreadsPerWorker = (int) Math.min( + numThreads, + heapMemoryPerWorker / MIN_BUFFER_SIZE + ); + + if (numThreadsPerWorker < 1) { + // Should not happen unless the CliIndexer has an unreasonable configuration. + // CliIndexer typically has well in excess of 1 MB (min buffer size) of heap per task. + throw new ISE("Cannot acquire buffers, available heap memory is not enough for task capacity[%d]", taskCapacity); + } + + // bufferPools has one list per "poolSize"; each of those lists has "bufferCount" buffers of size "sliceSize". + final List> bufferPools = new ArrayList<>(poolSize); + final int sliceSize = (int) Math.min(Integer.MAX_VALUE, heapMemoryPerWorker / numThreadsPerWorker); + + for (int i = 0; i < poolSize; i++) { + final List bufferPool = new ArrayList<>(numThreadsPerWorker); + bufferPools.add(bufferPool); + + for (int j = 0; j < numThreadsPerWorker; j++) { + bufferPool.add(ByteBuffer.allocate(sliceSize)); + } + } + + // bufferPools is built, return it as a ProcessingBuffersSet. + return new ReferenceCountingResourceHolder<>( + ProcessingBuffersSet.fromCollection(bufferPools), + () -> {} // Garbage collection will reclaim the buffers, since they are on-heap + ); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerTableInputSpecSlicer.java similarity index 96% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerTableInputSpecSlicer.java index 916dd3c1db3..48283bdd78a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/TableInputSpecSlicer.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerTableInputSpecSlicer.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.input.table; +package org.apache.druid.msq.indexing; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -35,6 +35,12 @@ import org.apache.druid.msq.input.InputSpec; import org.apache.druid.msq.input.InputSpecSlicer; import org.apache.druid.msq.input.NilInputSlice; import org.apache.druid.msq.input.SlicerUtils; +import org.apache.druid.msq.input.table.DataSegmentWithLocation; +import org.apache.druid.msq.input.table.DataServerRequestDescriptor; +import org.apache.druid.msq.input.table.DataServerSelector; +import org.apache.druid.msq.input.table.RichSegmentDescriptor; +import org.apache.druid.msq.input.table.SegmentsInputSlice; +import org.apache.druid.msq.input.table.TableInputSpec; import org.apache.druid.query.filter.DimFilterUtils; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.DataSegment; @@ -60,15 +66,15 @@ import java.util.stream.StreamSupport; /** * Slices {@link TableInputSpec} into {@link SegmentsInputSlice} in tasks. */ -public class TableInputSpecSlicer implements InputSpecSlicer +public class IndexerTableInputSpecSlicer implements InputSpecSlicer { - private static final Logger log = new Logger(TableInputSpecSlicer.class); + private static final Logger log = new Logger(IndexerTableInputSpecSlicer.class); private final CoordinatorClient coordinatorClient; private final TaskActionClient taskActionClient; private final SegmentSource includeSegmentSource; - public TableInputSpecSlicer( + public IndexerTableInputSpecSlicer( CoordinatorClient coordinatorClient, TaskActionClient taskActionClient, SegmentSource includeSegmentSource diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java index 0b3063ef48b..fbb0bff9556 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/IndexerWorkerContext.java @@ -24,6 +24,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Injector; import com.google.inject.Key; +import org.apache.druid.collections.ResourceHolder; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.guice.annotations.Smile; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; @@ -34,7 +35,8 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; import org.apache.druid.msq.exec.MemoryIntrospector; -import org.apache.druid.msq.exec.OutputChannelMode; +import org.apache.druid.msq.exec.ProcessingBuffersProvider; +import org.apache.druid.msq.exec.ProcessingBuffersSet; import org.apache.druid.msq.exec.TaskDataSegmentProvider; import org.apache.druid.msq.exec.Worker; import org.apache.druid.msq.exec.WorkerClient; @@ -45,7 +47,7 @@ import org.apache.druid.msq.indexing.client.IndexerControllerClient; import org.apache.druid.msq.indexing.client.IndexerWorkerClient; import org.apache.druid.msq.indexing.client.WorkerChatHandler; import org.apache.druid.msq.kernel.FrameContext; -import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryToolChestWarehouse; @@ -79,12 +81,16 @@ public class IndexerWorkerContext implements WorkerContext private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory; private final ServiceClientFactory clientFactory; private final MemoryIntrospector memoryIntrospector; + private final ProcessingBuffersProvider processingBuffersProvider; private final int maxConcurrentStages; private final boolean includeAllCounters; @GuardedBy("this") private ServiceLocator controllerLocator; + // Written under synchronized(this) using double-checked locking. + private volatile ResourceHolder processingBuffersSet; + public IndexerWorkerContext( final MSQWorkerTask task, final TaskToolbox toolbox, @@ -94,6 +100,7 @@ public class IndexerWorkerContext implements WorkerContext final TaskDataSegmentProvider dataSegmentProvider, final ServiceClientFactory clientFactory, final MemoryIntrospector memoryIntrospector, + final ProcessingBuffersProvider processingBuffersProvider, final DataServerQueryHandlerFactory dataServerQueryHandlerFactory ) { @@ -105,10 +112,14 @@ public class IndexerWorkerContext implements WorkerContext this.dataSegmentProvider = dataSegmentProvider; this.clientFactory = clientFactory; this.memoryIntrospector = memoryIntrospector; + this.processingBuffersProvider = processingBuffersProvider; this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory; final QueryContext queryContext = QueryContext.of(task.getContext()); - this.maxConcurrentStages = MultiStageQueryContext.getMaxConcurrentStages(queryContext); + this.maxConcurrentStages = MultiStageQueryContext.getMaxConcurrentStagesWithDefault( + queryContext, + IndexerControllerContext.DEFAULT_MAX_CONCURRENT_STAGES + ); this.includeAllCounters = MultiStageQueryContext.getIncludeAllCounters(queryContext); } @@ -127,6 +138,7 @@ public class IndexerWorkerContext implements WorkerContext final MemoryIntrospector memoryIntrospector = injector.getInstance(MemoryIntrospector.class); final OverlordClient overlordClient = injector.getInstance(OverlordClient.class).withRetryPolicy(StandardRetryPolicy.unlimited()); + final ProcessingBuffersProvider processingBuffersProvider = injector.getInstance(ProcessingBuffersProvider.class); final ObjectMapper smileMapper = injector.getInstance(Key.get(ObjectMapper.class, Smile.class)); final QueryToolChestWarehouse warehouse = injector.getInstance(QueryToolChestWarehouse.class); @@ -139,6 +151,7 @@ public class IndexerWorkerContext implements WorkerContext new TaskDataSegmentProvider(toolbox.getCoordinatorClient(), segmentCacheManager, indexIO), serviceClientFactory, memoryIntrospector, + processingBuffersProvider, new DataServerQueryHandlerFactory( toolbox.getCoordinatorClient(), serviceClientFactory, @@ -191,6 +204,14 @@ public class IndexerWorkerContext implements WorkerContext } } }); + closer.register(() -> { + synchronized (this) { + if (processingBuffersSet != null) { + processingBuffersSet.close(); + processingBuffersSet = null; + } + } + }); // Register the periodic controller checker final ExecutorService periodicControllerCheckerExec = Execs.singleThreaded("controller-status-checker-%s"); @@ -281,23 +302,39 @@ public class IndexerWorkerContext implements WorkerContext } @Override - public FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode) + public FrameContext frameContext(WorkOrder workOrder) { + if (processingBuffersSet == null) { + synchronized (this) { + if (processingBuffersSet == null) { + processingBuffersSet = processingBuffersProvider.acquire( + workOrder.getQueryDefinition(), + maxConcurrentStages() + ); + } + } + } + + final WorkerMemoryParameters memoryParameters = + WorkerMemoryParameters.createProductionInstance(workOrder, memoryIntrospector, maxConcurrentStages); + log.info("Memory parameters for stage[%s]: %s", workOrder.getStageDefinition().getId(), memoryParameters); + return new IndexerFrameContext( - queryDef.getStageDefinition(stageNumber).getId(), + workOrder.getStageDefinition().getId(), this, indexIO, dataSegmentProvider, + processingBuffersSet.get().acquireForStage(workOrder.getStageDefinition()), dataServerQueryHandlerFactory, - WorkerMemoryParameters.createProductionInstanceForWorker(injector, queryDef, stageNumber, maxConcurrentStages), - WorkerStorageParameters.createProductionInstance(injector, outputChannelMode) + memoryParameters, + WorkerStorageParameters.createProductionInstance(injector, workOrder.getOutputChannelMode()) ); } @Override public int threadCount() { - return memoryIntrospector.numProcessorsInJvm(); + return memoryIntrospector.numProcessingThreads(); } @Override @@ -312,6 +349,12 @@ public class IndexerWorkerContext implements WorkerContext return dataServerQueryHandlerFactory; } + @Override + public boolean includeAllCounters() + { + return includeAllCounters; + } + private synchronized ServiceLocator makeControllerLocator(final String controllerId) { if (controllerLocator == null) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index dc19e33282c..417fdb60d0f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -30,7 +30,6 @@ import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SecondaryPartitionType; import org.apache.druid.indexing.common.TaskToolbox; @@ -286,19 +285,10 @@ public class MSQCompactionRunner implements CompactionRunner private static Integer getRowsPerSegment(CompactionTask compactionTask) { - Integer rowsPerSegment = PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT; - if (compactionTask.getTuningConfig() != null) { - PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec(); - if (partitionsSpec instanceof DynamicPartitionsSpec) { - rowsPerSegment = partitionsSpec.getMaxRowsPerSegment(); - } else if (partitionsSpec instanceof DimensionRangePartitionsSpec) { - DimensionRangePartitionsSpec dimensionRangePartitionsSpec = (DimensionRangePartitionsSpec) partitionsSpec; - rowsPerSegment = dimensionRangePartitionsSpec.getTargetRowsPerSegment() != null - ? dimensionRangePartitionsSpec.getTargetRowsPerSegment() - : dimensionRangePartitionsSpec.getMaxRowsPerSegment(); - } + if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) { + return compactionTask.getTuningConfig().getPartitionsSpec().getMaxRowsPerSegment(); } - return rowsPerSegment; + return PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT; } private static RowSignature getRowSignature(DataSchema dataSchema) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java index c3f6feaab24..4ddc8274b9d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java @@ -59,6 +59,7 @@ import org.apache.druid.rpc.ServiceClientFactory; import org.apache.druid.rpc.StandardRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.server.coordination.BroadcastDatasourceLoadingSpec; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; @@ -374,4 +375,10 @@ public class MSQControllerTask extends AbstractTask implements ClientTaskQuery, { return LookupLoadingSpec.NONE; } + + @Override + public BroadcastDatasourceLoadingSpec getBroadcastDatasourceLoadingSpec() + { + return BroadcastDatasourceLoadingSpec.NONE; + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/PeonProcessingBuffersProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/PeonProcessingBuffersProvider.java new file mode 100644 index 00000000000..264c7af112f --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/PeonProcessingBuffersProvider.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.msq.indexing; + +import org.apache.druid.cli.CliPeon; +import org.apache.druid.collections.NonBlockingPool; +import org.apache.druid.collections.ReferenceCountingResourceHolder; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.msq.exec.ProcessingBuffersProvider; +import org.apache.druid.msq.exec.ProcessingBuffersSet; +import org.apache.druid.utils.CloseableUtils; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Production implementation of {@link ProcessingBuffersProvider} for tasks in {@link CliPeon}. + */ +public class PeonProcessingBuffersProvider implements ProcessingBuffersProvider +{ + private final AtomicBoolean acquired = new AtomicBoolean(false); + private final NonBlockingPool bufferPool; + private final int bufferCount; + + public PeonProcessingBuffersProvider( + final NonBlockingPool bufferPool, + final int bufferCount + ) + { + this.bufferPool = bufferPool; + this.bufferCount = bufferCount; + } + + @Override + public ResourceHolder acquire(int poolSize) + { + if (poolSize == 0) { + return new ReferenceCountingResourceHolder<>(ProcessingBuffersSet.EMPTY, () -> {}); + } + + if (!acquired.compareAndSet(false, true)) { + // We expect a single task in the JVM for CliPeon. + throw DruidException.defensive("Expected a single call to acquire() for[%s]", getClass().getName()); + } + + final Closer closer = Closer.create(); + + try { + // bufferPools has one list per "poolSize"; each of those lists has "bufferCount" buffers. + // Build these by acquiring "bufferCount" processing buffers and slicing each one up into "poolSize" slices. + final List> bufferPools = new ArrayList<>(); + for (int i = 0; i < poolSize; i++) { + bufferPools.add(new ArrayList<>(bufferCount)); + } + + for (int i = 0; i < bufferCount; i++) { + final ResourceHolder bufferHolder = closer.register(bufferPool.take()); + final ByteBuffer buffer = bufferHolder.get().duplicate(); + final int sliceSize = buffer.capacity() / poolSize; + + for (int j = 0; j < poolSize; j++) { + buffer.position(sliceSize * j).limit(sliceSize * (j + 1)); + bufferPools.get(j).add(buffer.slice()); + } + } + + // bufferPools is built, return it as a ProcessingBuffersSet. + return new ReferenceCountingResourceHolder<>( + ProcessingBuffersSet.fromCollection(bufferPools), + closer + ); + } + catch (Throwable e) { + throw CloseableUtils.closeAndWrapInCatch(e, closer); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerControllerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerControllerClient.java index 1e31de71a8a..1a420d69b6c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerControllerClient.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerControllerClient.java @@ -125,11 +125,11 @@ public class IndexerControllerClient implements ControllerClient } @Override - public void postWorkerError(String workerId, MSQErrorReport errorWrapper) throws IOException + public void postWorkerError(MSQErrorReport errorWrapper) throws IOException { final String path = StringUtils.format( "/workerError/%s", - StringUtils.urlEncode(workerId) + StringUtils.urlEncode(errorWrapper.getTaskId()) ); doRequest( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerWorkerManagerClient.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerWorkerManagerClient.java deleted file mode 100644 index 927130e0ca7..00000000000 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/client/IndexerWorkerManagerClient.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * 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.indexing.client; - -import com.google.common.collect.ImmutableSet; -import org.apache.druid.client.indexing.TaskStatusResponse; -import org.apache.druid.common.guava.FutureUtils; -import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.msq.exec.WorkerManagerClient; -import org.apache.druid.msq.indexing.MSQWorkerTask; -import org.apache.druid.rpc.indexing.OverlordClient; - -import java.util.Map; -import java.util.Set; - -/** - * Worker manager client backed by the Indexer service. Glues together - * three different mechanisms to provide the single multi-stage query interface. - */ -public class IndexerWorkerManagerClient implements WorkerManagerClient -{ - private final OverlordClient overlordClient; - private final TaskLocationFetcher locationFetcher = new TaskLocationFetcher(); - - public IndexerWorkerManagerClient(final OverlordClient overlordClient) - { - this.overlordClient = overlordClient; - } - - @Override - public String run(String taskId, MSQWorkerTask task) - { - FutureUtils.getUnchecked(overlordClient.runTask(taskId, task), true); - return taskId; - } - - @Override - public void cancel(String taskId) - { - FutureUtils.getUnchecked(overlordClient.cancelTask(taskId), true); - } - - @Override - public Map statuses(Set taskIds) - { - return FutureUtils.getUnchecked(overlordClient.taskStatuses(taskIds), true); - } - - @Override - public TaskLocation location(String workerId) - { - return locationFetcher.getLocation(workerId); - } - - @Override - public void close() - { - // Nothing to do. The OverlordServiceClient is closed by the JVM lifecycle. - } - - private class TaskLocationFetcher - { - TaskLocation getLocation(String workerId) - { - final TaskStatus taskStatus = FutureUtils.getUnchecked( - overlordClient.taskStatuses(ImmutableSet.of(workerId)), - true - ).get(workerId); - - if (taskStatus != null - && !TaskLocation.unknown().equals(taskStatus.getLocation())) { - return taskStatus.getLocation(); - } - - // Retry with the single status API - final TaskStatusResponse statusResponse = FutureUtils.getUnchecked( - overlordClient.taskStatus(workerId), - true - ); - if (statusResponse == null || statusResponse.getStatus() == null) { - return TaskLocation.unknown(); - } else { - return statusResponse.getStatus().getLocation(); - } - } - } -} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQTerminalStageSpecFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQTerminalStageSpecFactory.java similarity index 91% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQTerminalStageSpecFactory.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQTerminalStageSpecFactory.java index d5a1bd79455..d7179f13225 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQTerminalStageSpecFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQTerminalStageSpecFactory.java @@ -17,9 +17,8 @@ * under the License. */ -package org.apache.druid.msq.guice; +package org.apache.druid.msq.indexing.destination; -import org.apache.druid.msq.indexing.destination.TerminalStageSpec; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rel.DruidQuery; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SegmentGenerationTerminalStageSpecFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationTerminalStageSpecFactory.java similarity index 87% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SegmentGenerationTerminalStageSpecFactory.java rename to extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationTerminalStageSpecFactory.java index f761c0616a6..09c3c514ec0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/SegmentGenerationTerminalStageSpecFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationTerminalStageSpecFactory.java @@ -17,10 +17,8 @@ * under the License. */ -package org.apache.druid.msq.guice; +package org.apache.druid.msq.indexing.destination; -import org.apache.druid.msq.indexing.destination.SegmentGenerationStageSpec; -import org.apache.druid.msq.indexing.destination.TerminalStageSpec; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rel.DruidQuery; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationUtils.java index 09d79534337..b37a29f53e2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationUtils.java @@ -53,14 +53,12 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.rel.DruidQuery; import org.apache.druid.utils.CollectionUtils; import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashSet; import java.util.List; @@ -96,14 +94,13 @@ public final class SegmentGenerationUtils destination.getDimensionSchemas() ); - return new DataSchema( - destination.getDataSource(), - new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null), - dimensionsAndAggregators.lhs, - dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0]), - makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery, jsonMapper), - new TransformSpec(null, Collections.emptyList()) - ); + return DataSchema.builder() + .withDataSource(destination.getDataSource()) + .withTimestamp(new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null)) + .withDimensions(dimensionsAndAggregators.lhs) + .withAggregators(dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0])) + .withGranularity(makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery, jsonMapper)) + .build(); } private static GranularitySpec makeGranularitySpecForIngestion( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportLimiterPublisher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportLimiterPublisher.java index 9a8b3f79f6d..a820ed7995a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportLimiterPublisher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportLimiterPublisher.java @@ -97,7 +97,7 @@ public class MSQWarningReportLimiterPublisher implements MSQWarningReportPublish // Send the warning as an error if it is disallowed altogether if (criticalWarningCodes.contains(errorCode)) { try { - controllerClient.postWorkerError(workerId, MSQErrorReport.fromException(workerId, host, stageNumber, e)); + controllerClient.postWorkerError(MSQErrorReport.fromException(workerId, host, stageNumber, e)); } catch (IOException postException) { throw new RE(postException, "Failed to post the worker error [%s] to the controller", errorCode); @@ -117,10 +117,4 @@ public class MSQWarningReportLimiterPublisher implements MSQWarningReportPublish } delegate.publishException(stageNumber, e); } - - @Override - public void close() throws IOException - { - delegate.close(); - } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportPublisher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportPublisher.java index 9dbb51b6569..882ce4e98a6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportPublisher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportPublisher.java @@ -19,17 +19,11 @@ package org.apache.druid.msq.indexing.error; -import java.io.Closeable; -import java.io.IOException; - /** * Provides an interface for a worker to publish warnings to an external source. * For example, the worker uses this interface to send warnings to the controller. */ -public interface MSQWarningReportPublisher extends Closeable +public interface MSQWarningReportPublisher { void publishException(int stageNumber, Throwable e); - - @Override - void close() throws IOException; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportSimplePublisher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportSimplePublisher.java index 1353f404041..0a7554045da 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportSimplePublisher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportSimplePublisher.java @@ -63,10 +63,4 @@ public class MSQWarningReportSimplePublisher implements MSQWarningReportPublishe throw new RuntimeException(e2); } } - - @Override - public void close() - { - - } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java index 6f4b36da1ee..d4360a09d1a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.java.util.common.StringUtils; import java.util.Objects; @@ -36,6 +37,7 @@ public class NotEnoughMemoryFault extends BaseMSQFault private final long usableMemory; private final int serverWorkers; private final int serverThreads; + private final int inputWorkers; private final int maxConcurrentStages; @JsonCreator @@ -45,22 +47,33 @@ public class NotEnoughMemoryFault extends BaseMSQFault @JsonProperty("usableMemory") final long usableMemory, @JsonProperty("serverWorkers") final int serverWorkers, @JsonProperty("serverThreads") final int serverThreads, + @JsonProperty("inputWorkers") final int inputWorkers, @JsonProperty("maxConcurrentStages") final int maxConcurrentStages ) { super( CODE, - "Not enough memory. Required at least %,d bytes. (total = %,d bytes; usable = %,d bytes; " - + "worker capacity = %,d; processing threads = %,d; concurrent stages = %,d). " + "Not enough memory. " + + (suggestedServerMemory > 0 + ? StringUtils.format("Minimum bytes[%,d] is needed for the current configuration. ", suggestedServerMemory) + : "") + + "(total bytes[%,d]; " + + "usable bytes[%,d]; " + + "input workers[%,d]; " + + "concurrent stages[%,d]; " + + "server worker capacity[%,d]; " + + "server processing threads[%,d]). " + "Increase JVM memory with the -Xmx option" + + (inputWorkers > 1 ? ", or reduce maxNumTasks for this query" : "") + + (maxConcurrentStages > 1 ? ", or reduce maxConcurrentStages for this query" : "") + (serverWorkers > 1 ? ", or reduce worker capacity on this server" : "") - + (maxConcurrentStages > 1 ? ", or reduce maxConcurrentStages for this query" : ""), - suggestedServerMemory, + + (serverThreads > 1 ? ", or reduce processing threads on this server" : ""), serverMemory, usableMemory, + inputWorkers, + maxConcurrentStages, serverWorkers, - serverThreads, - maxConcurrentStages + serverThreads ); this.suggestedServerMemory = suggestedServerMemory; @@ -68,10 +81,12 @@ public class NotEnoughMemoryFault extends BaseMSQFault this.usableMemory = usableMemory; this.serverWorkers = serverWorkers; this.serverThreads = serverThreads; + this.inputWorkers = inputWorkers; this.maxConcurrentStages = maxConcurrentStages; } @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) public long getSuggestedServerMemory() { return suggestedServerMemory; @@ -101,6 +116,13 @@ public class NotEnoughMemoryFault extends BaseMSQFault return serverThreads; } + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public int getInputWorkers() + { + return inputWorkers; + } + @JsonProperty @JsonInclude(JsonInclude.Include.NON_DEFAULT) public int getMaxConcurrentStages() @@ -126,6 +148,7 @@ public class NotEnoughMemoryFault extends BaseMSQFault && usableMemory == that.usableMemory && serverWorkers == that.serverWorkers && serverThreads == that.serverThreads + && inputWorkers == that.inputWorkers && maxConcurrentStages == that.maxConcurrentStages; } @@ -139,6 +162,7 @@ public class NotEnoughMemoryFault extends BaseMSQFault usableMemory, serverWorkers, serverThreads, + inputWorkers, maxConcurrentStages ); } @@ -148,10 +172,11 @@ public class NotEnoughMemoryFault extends BaseMSQFault { return "NotEnoughMemoryFault{" + "suggestedServerMemory=" + suggestedServerMemory + - " bytes, serverMemory=" + serverMemory + - " bytes, usableMemory=" + usableMemory + - " bytes, serverWorkers=" + serverWorkers + + ", serverMemory=" + serverMemory + + ", usableMemory=" + usableMemory + + ", serverWorkers=" + serverWorkers + ", serverThreads=" + serverThreads + + ", inputWorkers=" + inputWorkers + ", maxConcurrentStages=" + maxConcurrentStages + '}'; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyRowsWithSameKeyFault.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyRowsWithSameKeyFault.java index 60d355579b6..be284ae502d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyRowsWithSameKeyFault.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/TooManyRowsWithSameKeyFault.java @@ -44,9 +44,8 @@ public class TooManyRowsWithSameKeyFault extends BaseMSQFault { super( CODE, - "Too many rows with the same key[%s] during sort-merge join (bytes buffered[%,d], limit[%,d]). " - + "Try increasing heap memory available to workers, " - + "or adjusting your query to process fewer rows with this key.", + "Too many rows with the same key[%s] on both sides of sort-merge join (bytes buffered[%,d], limit[%,d]). " + + "Try adjusting your query such that there are fewer rows with this key on at least one side of the join.", key, numBytes, maxBytes diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java index a4b52abee58..92e5066dc4d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/processor/SegmentGeneratorFrameProcessor.java @@ -35,9 +35,7 @@ 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.frame.segment.FrameStorageAdapter; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.counters.SegmentGenerationProgressCounter; import org.apache.druid.msq.exec.MSQTasks; @@ -46,8 +44,8 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.realtime.appenderator.Appenderator; @@ -181,8 +179,8 @@ public class SegmentGeneratorFrameProcessor implements FrameProcessor( - ProcessorManagers.of(workers) - .withAccumulation( - new HashSet<>(), - (acc, segment) -> { - if (segment != null) { - acc.add(segment); - } + // Run at most one segmentGenerator per work order, since segment generation memory is carved out + // per-worker, not per-processor. See WorkerMemoryParameters for how the memory limits are calculated. + new ConcurrencyLimitedProcessorManager<>(ProcessorManagers.of(workers), 1) + .withAccumulation( + new HashSet<>(), + (acc, segment) -> { + if (segment != null) { + acc.add(segment); + } - return acc; - } - ), + return acc; + } + ), OutputChannels.none() ); } + @Override + public boolean usesProcessingBuffers() + { + return false; + } + @Override public TypeReference> getResultTypeReference() { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecs.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecs.java index 78241257710..250f320118a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecs.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/InputSpecs.java @@ -35,6 +35,9 @@ public class InputSpecs // No instantiation. } + /** + * Returns the set of input stages, from {@link StageInputSpec}, for a given list of {@link InputSpec}. + */ public static IntSet getStageNumbers(final List specs) { final IntSet retVal = new IntRBTreeSet(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java index 4b68a3bf1b0..2a863fa5525 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/external/ExternalInputSliceReader.java @@ -45,6 +45,7 @@ import org.apache.druid.msq.input.ReadableInputs; import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.msq.util.DimensionSchemaUtils; +import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.RowBasedSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.column.ColumnHolder; @@ -163,7 +164,7 @@ public class ExternalInputSliceReader implements InputSliceReader signature ); return new SegmentWithDescriptor( - () -> ResourceHolder.fromCloseable(segment), + () -> ResourceHolder.fromCloseable(new CompleteSegment(null, segment)), new RichSegmentDescriptor(segmentId.toDescriptor(), null) ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java index ef58c7723b3..8a05ce1527e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java @@ -30,6 +30,7 @@ import org.apache.druid.msq.input.ReadableInputs; import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.query.InlineDataSource; +import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.InlineSegmentWrangler; import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.timeline.SegmentId; @@ -74,7 +75,7 @@ public class InlineInputSliceReader implements InputSliceReader segmentWrangler.getSegmentsForIntervals(dataSource, Intervals.ONLY_ETERNITY), segment -> ReadableInput.segment( new SegmentWithDescriptor( - () -> ResourceHolder.fromCloseable(segment), + () -> ResourceHolder.fromCloseable(new CompleteSegment(null, segment)), DUMMY_SEGMENT_DESCRIPTOR ) ) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java index 2b327f216f7..85f0b10718d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/lookup/LookupInputSliceReader.java @@ -32,6 +32,7 @@ import org.apache.druid.msq.input.ReadableInputs; import org.apache.druid.msq.input.table.RichSegmentDescriptor; import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.query.LookupDataSource; +import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.timeline.SegmentId; @@ -98,7 +99,7 @@ public class LookupInputSliceReader implements InputSliceReader throw new ISE("Lookup[%s] has multiple segments; cannot read", lookupName); } - return ResourceHolder.fromCloseable(segment); + return ResourceHolder.fromCloseable(new CompleteSegment(null, segment)); }, new RichSegmentDescriptor(SegmentId.dummy(lookupName).toDescriptor(), null) ) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java index b9026c7b9fb..343f7994d1e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentWithDescriptor.java @@ -21,6 +21,7 @@ package org.apache.druid.msq.input.table; import com.google.common.base.Preconditions; import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.Segment; import java.util.Objects; @@ -31,7 +32,7 @@ import java.util.function.Supplier; */ public class SegmentWithDescriptor { - private final Supplier> segmentSupplier; + private final Supplier> segmentSupplier; private final RichSegmentDescriptor descriptor; /** @@ -42,7 +43,7 @@ public class SegmentWithDescriptor * @param descriptor segment descriptor */ public SegmentWithDescriptor( - final Supplier> segmentSupplier, + final Supplier> segmentSupplier, final RichSegmentDescriptor descriptor ) { @@ -59,7 +60,7 @@ public class SegmentWithDescriptor * It is not necessary to call {@link Segment#close()} on the returned segment. Calling {@link ResourceHolder#close()} * is enough. */ - public ResourceHolder getOrLoad() + public ResourceHolder getOrLoad() { return segmentSupplier.get(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java index dd59dfebd80..6c4ec10d6df 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/SegmentsInputSlice.java @@ -32,7 +32,7 @@ import java.util.Objects; /** * Input slice representing a set of segments to read. *
    - * Sliced from {@link TableInputSpec} by {@link TableInputSpecSlicer}. + * Sliced from {@link TableInputSpec}. *
    * Similar to {@link org.apache.druid.query.spec.MultipleSpecificSegmentSpec} from native queries. *
    diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java index da962a9d393..1b80f72f86f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/FrameContext.java @@ -20,9 +20,9 @@ package org.apache.druid.msq.kernel; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; -import org.apache.druid.msq.exec.OutputChannelMode; +import org.apache.druid.msq.exec.ProcessingBuffers; +import org.apache.druid.msq.exec.WorkerImpl; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.querykit.DataSegmentProvider; @@ -40,7 +40,7 @@ import java.io.File; * Provides services and objects for the functioning of the frame processors. Scoped to a specific stage of a * specific query, i.e., one {@link WorkOrder}. * - * Generated by {@link org.apache.druid.msq.exec.WorkerContext#frameContext(QueryDefinition, int, OutputChannelMode)}. + * Generated by {@link org.apache.druid.msq.exec.WorkerContext#frameContext(WorkOrder)}. */ public interface FrameContext extends Closeable { @@ -54,6 +54,9 @@ public interface FrameContext extends Closeable DataServerQueryHandlerFactory dataServerQueryHandlerFactory(); + /** + * Temporary directory, fully owned by this particular stage. + */ File tempDir(); ObjectMapper jsonMapper(); @@ -66,7 +69,7 @@ public interface FrameContext extends Closeable IndexMergerV9 indexMerger(); - Bouncer processorBouncer(); + ProcessingBuffers processingBuffers(); WorkerMemoryParameters memoryParameters(); @@ -76,4 +79,11 @@ public interface FrameContext extends Closeable { return new File(tempDir(), name); } + + /** + * Releases resources used in processing. This is called when processing has completed, but before results are + * cleaned up. Specifically, it is called by {@link WorkerImpl.KernelHolder#processorCloser}. + */ + @Override + void close(); } 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 fbf02d46e34..1bdba5ee22e 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 @@ -78,6 +78,11 @@ public interface FrameProcessorFactory boolean removeNullBytes ) throws IOException; + /** + * Whether processors from this factory use {@link org.apache.druid.msq.exec.ProcessingBuffers}. + */ + boolean usesProcessingBuffers(); + @Nullable TypeReference getResultTypeReference(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java index 19a7978abba..cd2bb6a81f4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/StageDefinition.java @@ -146,6 +146,13 @@ public class StageDefinition } } + public static boolean mustGatherResultKeyStatistics(@Nullable final ShuffleSpec shuffleSpec) + { + return shuffleSpec != null + && shuffleSpec.kind() == ShuffleKind.GLOBAL_SORT + && ((GlobalSortShuffleSpec) shuffleSpec).mustGatherResultKeyStatistics(); + } + public static StageDefinitionBuilder builder(final int stageNumber) { return new StageDefinitionBuilder(stageNumber); @@ -302,14 +309,10 @@ public class StageDefinition * For eg: we know there's exactly one partition in query shapes like `select with limit`. *

    * In such cases, we return a false. - * - * @return */ public boolean mustGatherResultKeyStatistics() { - return shuffleSpec != null - && shuffleSpec.kind() == ShuffleKind.GLOBAL_SORT - && ((GlobalSortShuffleSpec) shuffleSpec).mustGatherResultKeyStatistics(); + return mustGatherResultKeyStatistics(shuffleSpec); } public Either generatePartitionBoundariesForShuffle( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java index 0c857870210..2a45605826b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/WorkOrder.java @@ -26,9 +26,11 @@ import com.google.common.base.Preconditions; import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.exec.OutputChannelMode; import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.query.QueryContext; import javax.annotation.Nullable; import java.util.List; +import java.util.Map; import java.util.Objects; /** @@ -51,9 +53,18 @@ public class WorkOrder @Nullable private final List workerIds; + /** + * Always non-null for newer controllers. This is marked nullable for backwards-compatibility reasons. + */ @Nullable private final OutputChannelMode outputChannelMode; + /** + * Always non-null for newer controllers. This is marked nullable for backwards-compatibility reasons. + */ + @Nullable + private final QueryContext workerContext; + @JsonCreator @SuppressWarnings("rawtypes") public WorkOrder( @@ -63,7 +74,8 @@ public class WorkOrder @JsonProperty("input") final List workerInputs, @JsonProperty("extra") @Nullable final ExtraInfoHolder extraInfoHolder, @JsonProperty("workers") @Nullable final List workerIds, - @JsonProperty("output") @Nullable final OutputChannelMode outputChannelMode + @JsonProperty("output") @Nullable final OutputChannelMode outputChannelMode, + @JsonProperty("context") @Nullable final Map workerContext ) { this.queryDefinition = Preconditions.checkNotNull(queryDefinition, "queryDefinition"); @@ -73,6 +85,7 @@ public class WorkOrder this.extraInfoHolder = extraInfoHolder; this.workerIds = workerIds; this.outputChannelMode = outputChannelMode; + this.workerContext = workerContext != null ? QueryContext.of(workerContext) : null; } @JsonProperty("query") @@ -124,6 +137,10 @@ public class WorkOrder return outputChannelMode != null; } + /** + * Retrieves the output channel mode set by the controller. Null means the controller didn't set it, which means + * we're dealing with an older controller. + */ @Nullable @JsonProperty("output") @JsonInclude(JsonInclude.Include.NON_NULL) @@ -132,6 +149,29 @@ public class WorkOrder return outputChannelMode; } + public boolean hasWorkerContext() + { + return workerContext != null; + } + + /** + * Retrieves the query context set by the controller. Null means the controller didn't set it, which means + * we're dealing with an older controller. + */ + @Nullable + public QueryContext getWorkerContext() + { + return workerContext; + } + + @Nullable + @JsonProperty("context") + @JsonInclude(JsonInclude.Include.NON_NULL) + public Map getContextForSerialization() + { + return workerContext != null ? workerContext.asMap() : null; + } + @Nullable public Object getExtraInfo() { @@ -155,7 +195,26 @@ public class WorkOrder workerInputs, extraInfoHolder, workerIds, - newOutputChannelMode + newOutputChannelMode, + workerContext != null ? workerContext.asMap() : null + ); + } + } + + public WorkOrder withWorkerContext(final QueryContext newContext) + { + if (Objects.equals(newContext, this.workerContext)) { + return this; + } else { + return new WorkOrder( + queryDefinition, + stageNumber, + workerNumber, + workerInputs, + extraInfoHolder, + workerIds, + outputChannelMode, + newContext.asMap() ); } } @@ -176,7 +235,8 @@ public class WorkOrder && Objects.equals(workerInputs, workOrder.workerInputs) && Objects.equals(extraInfoHolder, workOrder.extraInfoHolder) && Objects.equals(workerIds, workOrder.workerIds) - && Objects.equals(outputChannelMode, workOrder.outputChannelMode); + && Objects.equals(outputChannelMode, workOrder.outputChannelMode) + && Objects.equals(workerContext, workOrder.workerContext); } @Override @@ -189,7 +249,8 @@ public class WorkOrder workerInputs, extraInfoHolder, workerIds, - outputChannelMode + outputChannelMode, + workerContext ); } @@ -204,6 +265,7 @@ public class WorkOrder ", extraInfoHolder=" + extraInfoHolder + ", workerIds=" + workerIds + ", outputChannelMode=" + outputChannelMode + + ", context=" + workerContext + '}'; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java index 05e0f722ccd..62a13326909 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernel.java @@ -302,7 +302,8 @@ public class ControllerQueryKernel workerInputs.inputsForWorker(workerNumber), extraInfoHolder, config.getWorkerIds(), - outputChannelMode + outputChannelMode, + config.getWorkerContextMap() ); QueryValidator.validateWorkOrder(workOrder); @@ -676,11 +677,10 @@ public class ControllerQueryKernel { if (stageOutputChannelModes.get(stageId) == OutputChannelMode.MEMORY) { if (getStageDefinition(stageId).doesSortDuringShuffle()) { - // Stages that sort during shuffle go through a READING_INPUT phase followed by a POST_READING phase - // (once all input is read). These stages start producing output once POST_READING starts. - return newPhase == ControllerStagePhase.POST_READING; + // Sorting stages start producing output when they finish reading their input. + return newPhase.isDoneReadingInput(); } else { - // Can read results immediately. + // Non-sorting stages start producing output immediately. return newPhase == ControllerStagePhase.NEW; } } else { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelConfig.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelConfig.java index 5c754aedd4f..f7516c63c92 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelConfig.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelConfig.java @@ -21,9 +21,12 @@ package org.apache.druid.msq.kernel.controller; import org.apache.druid.java.util.common.IAE; import org.apache.druid.msq.indexing.destination.MSQDestination; +import org.apache.druid.msq.kernel.WorkOrder; import javax.annotation.Nullable; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; /** @@ -37,22 +40,22 @@ public class ControllerQueryKernelConfig private final boolean durableStorage; private final boolean faultTolerance; private final MSQDestination destination; - @Nullable - private final String controllerId; - + private final String controllerHost; @Nullable private final List workerIds; + private final Map workerContextMap; - private ControllerQueryKernelConfig( + ControllerQueryKernelConfig( int maxRetainedPartitionSketchBytes, int maxConcurrentStages, boolean pipeline, boolean durableStorage, boolean faultTolerance, MSQDestination destination, - @Nullable String controllerId, - @Nullable List workerIds + @Nullable String controllerHost, + @Nullable List workerIds, + Map workerContextMap ) { if (maxRetainedPartitionSketchBytes <= 0) { @@ -85,8 +88,9 @@ public class ControllerQueryKernelConfig this.durableStorage = durableStorage; this.faultTolerance = faultTolerance; this.destination = destination; - this.controllerId = controllerId; + this.controllerHost = controllerHost; this.workerIds = workerIds; + this.workerContextMap = workerContextMap; } public static Builder builder() @@ -130,6 +134,14 @@ public class ControllerQueryKernelConfig return workerIds; } + /** + * Map to include in {@link WorkOrder}, as {@link WorkOrder#getWorkerContext()}. + */ + public Map getWorkerContextMap() + { + return workerContextMap; + } + @Override public boolean equals(Object o) { @@ -145,8 +157,10 @@ public class ControllerQueryKernelConfig && pipeline == that.pipeline && durableStorage == that.durableStorage && faultTolerance == that.faultTolerance - && Objects.equals(controllerId, that.controllerId) - && Objects.equals(workerIds, that.workerIds); + && Objects.equals(destination, that.destination) + && Objects.equals(controllerHost, that.controllerHost) + && Objects.equals(workerIds, that.workerIds) + && Objects.equals(workerContextMap, that.workerContextMap); } @Override @@ -158,8 +172,10 @@ public class ControllerQueryKernelConfig pipeline, durableStorage, faultTolerance, - controllerId, - workerIds + destination, + controllerHost, + workerIds, + workerContextMap ); } @@ -171,9 +187,11 @@ public class ControllerQueryKernelConfig ", maxConcurrentStages=" + maxConcurrentStages + ", pipeline=" + pipeline + ", durableStorage=" + durableStorage + - ", faultTolerant=" + faultTolerance + - ", controllerId='" + controllerId + '\'' + + ", faultTolerance=" + faultTolerance + + ", destination=" + destination + + ", controllerHost='" + controllerHost + '\'' + ", workerIds=" + workerIds + + ", workerContextMap=" + workerContextMap + '}'; } @@ -185,8 +203,9 @@ public class ControllerQueryKernelConfig private boolean durableStorage; private boolean faultTolerant; private MSQDestination destination; - private String controllerId; + private String controllerHost; private List workerIds; + private Map workerContextMap = Collections.emptyMap(); /** * Use {@link #builder()}. @@ -231,9 +250,9 @@ public class ControllerQueryKernelConfig return this; } - public Builder controllerId(final String controllerId) + public Builder controllerHost(final String controllerHost) { - this.controllerId = controllerId; + this.controllerHost = controllerHost; return this; } @@ -243,6 +262,12 @@ public class ControllerQueryKernelConfig return this; } + public Builder workerContextMap(final Map workerContextMap) + { + this.workerContextMap = workerContextMap; + return this; + } + public ControllerQueryKernelConfig build() { return new ControllerQueryKernelConfig( @@ -252,8 +277,9 @@ public class ControllerQueryKernelConfig durableStorage, faultTolerant, destination, - controllerId, - workerIds + controllerHost, + workerIds, + workerContextMap ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java index b838092ca71..992b90c0285 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStageKernel.java @@ -182,6 +182,12 @@ public class WorkerStageKernel throw new NullPointerException("resultObject must not be null"); } + if (phase.isTerminal()) { + // Ignore RESULTS_COMPLETE if work is already finished. This can happen if we transition to FINISHED early + // due to a downstream stage including a limit. + return; + } + transitionTo(WorkerStagePhase.RESULTS_COMPLETE); this.resultObject = resultObject; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java index 4e59e7d17a8..10543beeb06 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/kernel/worker/WorkerStagePhase.java @@ -19,6 +19,8 @@ package org.apache.druid.msq.kernel.worker; +import org.apache.druid.msq.exec.ProcessingBuffers; + /** * Phases that a stage can be in, as far as the worker is concerned. * @@ -99,6 +101,8 @@ public enum WorkerStagePhase /** * Whether this phase indicates a stage is running and consuming its full complement of resources. * + * Importantly, stages that are not running are not holding {@link ProcessingBuffers}. + * * There are still some resources that can be consumed by stages that are not running. For example, in the * {@link #FINISHED} state, stages can still have data on disk that has not been cleaned-up yet, some pointers * to that data that still reside in memory, and some counters in memory available for collection by the controller. 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 4cf23387633..013b6d4c93c 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 @@ -352,7 +352,7 @@ public abstract class BaseLeafFrameProcessorFactory extends BaseFrameProcessorFa return BroadcastJoinSegmentMapFnProcessor.create( query, broadcastInputs, - frameContext.memoryParameters().getBroadcastJoinMemory() + frameContext.memoryParameters().getBroadcastBufferMemory() ); } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessor.java index ab160f7319d..cbb79c45702 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessor.java @@ -83,7 +83,7 @@ public class BroadcastJoinSegmentMapFnProcessor implements FrameProcessor query, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java index 91ee4a48788..488479407cf 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentProvider.java @@ -21,7 +21,7 @@ package org.apache.druid.msq.querykit; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.msq.counters.ChannelCounters; -import org.apache.druid.segment.Segment; +import org.apache.druid.segment.CompleteSegment; import org.apache.druid.timeline.SegmentId; import java.util.function.Supplier; @@ -32,10 +32,16 @@ public interface DataSegmentProvider * Returns a supplier that fetches the segment corresponding to the provided segmentId from deep storage. The segment * is not actually fetched until you call {@link Supplier#get()}. Once you call this, make sure to also call * {@link ResourceHolder#close()}. - *
    + * * It is not necessary to call {@link ResourceHolder#close()} if you never call {@link Supplier#get()}. + * + * @param segmentId segment ID to fetch + * @param channelCounters counters to increment when the segment is closed + * @param isReindex true if this is a DML command (INSERT or REPLACE) writing into the same table it is + * reading from; false otherwise. When true, implementations must only allow reading from + * segments that are currently-used according to the Coordinator. */ - Supplier> fetchSegment( + Supplier> fetchSegment( SegmentId segmentId, ChannelCounters channelCounters, boolean isReindex diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentTimelineView.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentTimelineView.java deleted file mode 100644 index cc010a104c6..00000000000 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSegmentTimelineView.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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 org.apache.druid.query.planning.DataSourceAnalysis; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.TimelineLookup; -import org.joda.time.Interval; - -import java.util.List; -import java.util.Optional; - -public interface DataSegmentTimelineView -{ - /** - * Returns the timeline for a datasource, if it 'exists'. The analysis object passed in must represent a scan-based - * datasource of a single table. (i.e., {@link DataSourceAnalysis#getBaseTableDataSource()} must be present.) - * - * @param dataSource table data source name - * @param intervals relevant intervals. The returned timeline will *at least* include all segments that overlap - * these intervals. It may also include more. Empty means the timeline may not contain any - * segments at all. - * - * @return timeline, if it 'exists' - * - * @throws IllegalStateException if 'analysis' does not represent a scan-based datasource of a single table - */ - Optional> getTimeline( - String dataSource, - List intervals - ); -} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java index e6ddb4d723d..21848813e5d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java @@ -123,8 +123,7 @@ public class DataSourcePlan /** * Build a plan. * - * @param queryKit query kit reference for recursive planning - * @param queryId query ID + * @param queryKitSpec reference for recursive planning * @param queryContext query context * @param dataSource datasource to plan * @param querySegmentSpec intervals for mandatory pruning. Must be {@link MultipleIntervalSegmentSpec}. The returned @@ -132,20 +131,17 @@ public class DataSourcePlan * @param filter filter for best-effort pruning. The returned plan may or may not be filtered to this * filter. Query processing must still apply the filter to generated correct results. * @param filterFields which fields from the filter to consider for pruning, or null to consider all fields. - * @param maxWorkerCount maximum number of workers for subqueries * @param minStageNumber starting stage number for subqueries * @param broadcast whether the plan should broadcast data for this datasource */ @SuppressWarnings("rawtypes") public static DataSourcePlan forDataSource( - final QueryKit queryKit, - final String queryId, + final QueryKitSpec queryKitSpec, final QueryContext queryContext, final DataSource dataSource, final QuerySegmentSpec querySegmentSpec, @Nullable DimFilter filter, @Nullable Set filterFields, - final int maxWorkerCount, final int minStageNumber, final boolean broadcast ) @@ -180,47 +176,38 @@ public class DataSourcePlan return forLookup((LookupDataSource) dataSource, broadcast); } else if (dataSource instanceof FilteredDataSource) { return forFilteredDataSource( - queryKit, - queryId, + queryKitSpec, queryContext, (FilteredDataSource) dataSource, querySegmentSpec, - maxWorkerCount, minStageNumber, broadcast ); } else if (dataSource instanceof UnnestDataSource) { return forUnnest( - queryKit, - queryId, + queryKitSpec, queryContext, (UnnestDataSource) dataSource, querySegmentSpec, - maxWorkerCount, minStageNumber, broadcast ); } else if (dataSource instanceof QueryDataSource) { checkQuerySegmentSpecIsEternity(dataSource, querySegmentSpec); return forQuery( - queryKit, - queryId, + queryKitSpec, (QueryDataSource) dataSource, - maxWorkerCount, minStageNumber, - broadcast, - queryContext + broadcast ); } else if (dataSource instanceof UnionDataSource) { return forUnion( - queryKit, - queryId, + queryKitSpec, queryContext, (UnionDataSource) dataSource, querySegmentSpec, filter, filterFields, - maxWorkerCount, minStageNumber, broadcast ); @@ -234,25 +221,21 @@ public class DataSourcePlan switch (deducedJoinAlgorithm) { case BROADCAST: return forBroadcastHashJoin( - queryKit, - queryId, + queryKitSpec, queryContext, (JoinDataSource) dataSource, querySegmentSpec, filter, filterFields, - maxWorkerCount, minStageNumber, broadcast ); case SORT_MERGE: return forSortMergeJoin( - queryKit, - queryId, + queryKitSpec, (JoinDataSource) dataSource, querySegmentSpec, - maxWorkerCount, minStageNumber, broadcast ); @@ -414,23 +397,18 @@ public class DataSourcePlan } private static DataSourcePlan forQuery( - final QueryKit queryKit, - final String queryId, + final QueryKitSpec queryKitSpec, final QueryDataSource dataSource, - final int maxWorkerCount, final int minStageNumber, - final boolean broadcast, - @Nullable final QueryContext parentContext + final boolean broadcast ) { - final QueryDefinition subQueryDef = queryKit.makeQueryDefinition( - queryId, + final QueryDefinition subQueryDef = queryKitSpec.getQueryKit().makeQueryDefinition( + queryKitSpec, // Subqueries ignore SQL_INSERT_SEGMENT_GRANULARITY, even if set in the context. It's only used for the // outermost query, and setting it for the subquery makes us erroneously add bucketing where it doesn't belong. dataSource.getQuery().withOverriddenContext(CONTEXT_MAP_NO_SEGMENT_GRANULARITY), - queryKit, - ShuffleSpecFactories.globalSortWithMaxPartitionCount(maxWorkerCount), - maxWorkerCount, + ShuffleSpecFactories.globalSortWithMaxPartitionCount(queryKitSpec.getNumPartitionsForShuffle()), minStageNumber ); @@ -445,25 +423,21 @@ public class DataSourcePlan } private static DataSourcePlan forFilteredDataSource( - final QueryKit queryKit, - final String queryId, + final QueryKitSpec queryKitSpec, final QueryContext queryContext, final FilteredDataSource dataSource, final QuerySegmentSpec querySegmentSpec, - final int maxWorkerCount, final int minStageNumber, final boolean broadcast ) { final DataSourcePlan basePlan = forDataSource( - queryKit, - queryId, + queryKitSpec, queryContext, dataSource.getBase(), querySegmentSpec, null, null, - maxWorkerCount, minStageNumber, broadcast ); @@ -485,26 +459,22 @@ public class DataSourcePlan * Build a plan for Unnest data source */ private static DataSourcePlan forUnnest( - final QueryKit queryKit, - final String queryId, + final QueryKitSpec queryKitSpec, final QueryContext queryContext, final UnnestDataSource dataSource, final QuerySegmentSpec querySegmentSpec, - final int maxWorkerCount, final int minStageNumber, final boolean broadcast ) { // Find the plan for base data source by recursing final DataSourcePlan basePlan = forDataSource( - queryKit, - queryId, + queryKitSpec, queryContext, dataSource.getBase(), querySegmentSpec, null, null, - maxWorkerCount, minStageNumber, broadcast ); @@ -529,14 +499,12 @@ public class DataSourcePlan } private static DataSourcePlan forUnion( - final QueryKit queryKit, - final String queryId, + final QueryKitSpec queryKitSpec, final QueryContext queryContext, final UnionDataSource unionDataSource, final QuerySegmentSpec querySegmentSpec, @Nullable DimFilter filter, @Nullable Set filterFields, - final int maxWorkerCount, final int minStageNumber, final boolean broadcast ) @@ -544,21 +512,19 @@ public class DataSourcePlan // This is done to prevent loss of generality since MSQ can plan any type of DataSource. List children = unionDataSource.getDataSources(); - final QueryDefinitionBuilder subqueryDefBuilder = QueryDefinition.builder(queryId); + final QueryDefinitionBuilder subqueryDefBuilder = QueryDefinition.builder(queryKitSpec.getQueryId()); final List newChildren = new ArrayList<>(); final List inputSpecs = new ArrayList<>(); final IntSet broadcastInputs = new IntOpenHashSet(); for (DataSource child : children) { DataSourcePlan childDataSourcePlan = forDataSource( - queryKit, - queryId, + queryKitSpec, queryContext, child, querySegmentSpec, filter, filterFields, - maxWorkerCount, Math.max(minStageNumber, subqueryDefBuilder.getNextStageNumber()), broadcast ); @@ -582,30 +548,26 @@ public class DataSourcePlan * Build a plan for broadcast hash-join. */ private static DataSourcePlan forBroadcastHashJoin( - final QueryKit queryKit, - final String queryId, + final QueryKitSpec queryKitSpec, final QueryContext queryContext, final JoinDataSource dataSource, final QuerySegmentSpec querySegmentSpec, @Nullable final DimFilter filter, @Nullable final Set filterFields, - final int maxWorkerCount, final int minStageNumber, final boolean broadcast ) { - final QueryDefinitionBuilder subQueryDefBuilder = QueryDefinition.builder(queryId); + final QueryDefinitionBuilder subQueryDefBuilder = QueryDefinition.builder(queryKitSpec.getQueryId()); final DataSourceAnalysis analysis = dataSource.getAnalysis(); final DataSourcePlan basePlan = forDataSource( - queryKit, - queryId, + queryKitSpec, queryContext, analysis.getBaseDataSource(), querySegmentSpec, filter, filter == null ? null : DimFilterUtils.onlyBaseFields(filterFields, analysis), - maxWorkerCount, Math.max(minStageNumber, subQueryDefBuilder.getNextStageNumber()), broadcast ); @@ -618,14 +580,12 @@ public class DataSourcePlan for (int i = 0; i < analysis.getPreJoinableClauses().size(); i++) { final PreJoinableClause clause = analysis.getPreJoinableClauses().get(i); final DataSourcePlan clausePlan = forDataSource( - queryKit, - queryId, + queryKitSpec, queryContext, clause.getDataSource(), new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY), null, // Don't push down query filters for right-hand side: needs some work to ensure it works properly. null, - maxWorkerCount, Math.max(minStageNumber, subQueryDefBuilder.getNextStageNumber()), true // Always broadcast right-hand side of the join. ); @@ -655,11 +615,9 @@ public class DataSourcePlan * Build a plan for sort-merge join. */ private static DataSourcePlan forSortMergeJoin( - final QueryKit queryKit, - final String queryId, + final QueryKitSpec queryKitSpec, final JoinDataSource dataSource, final QuerySegmentSpec querySegmentSpec, - final int maxWorkerCount, final int minStageNumber, final boolean broadcast ) @@ -672,19 +630,16 @@ public class DataSourcePlan SortMergeJoinFrameProcessorFactory.validateCondition(dataSource.getConditionAnalysis()) ); - final QueryDefinitionBuilder subQueryDefBuilder = QueryDefinition.builder(queryId); + final QueryDefinitionBuilder subQueryDefBuilder = QueryDefinition.builder(queryKitSpec.getQueryId()); // Plan the left input. // We're confident that we can cast dataSource.getLeft() to QueryDataSource, because DruidJoinQueryRel creates // subqueries when the join algorithm is sortMerge. final DataSourcePlan leftPlan = forQuery( - queryKit, - queryId, + queryKitSpec, (QueryDataSource) dataSource.getLeft(), - maxWorkerCount, Math.max(minStageNumber, subQueryDefBuilder.getNextStageNumber()), - false, - null + false ); leftPlan.getSubQueryDefBuilder().ifPresent(subQueryDefBuilder::addAll); @@ -692,13 +647,10 @@ public class DataSourcePlan // We're confident that we can cast dataSource.getRight() to QueryDataSource, because DruidJoinQueryRel creates // subqueries when the join algorithm is sortMerge. final DataSourcePlan rightPlan = forQuery( - queryKit, - queryId, + queryKitSpec, (QueryDataSource) dataSource.getRight(), - maxWorkerCount, Math.max(minStageNumber, subQueryDefBuilder.getNextStageNumber()), - false, - null + false ); rightPlan.getSubQueryDefBuilder().ifPresent(subQueryDefBuilder::addAll); @@ -707,8 +659,9 @@ public class DataSourcePlan ((StageInputSpec) Iterables.getOnlyElement(leftPlan.getInputSpecs())).getStageNumber() ); + final int hashPartitionCount = queryKitSpec.getNumPartitionsForShuffle(); final List leftPartitionKey = partitionKeys.get(0); - leftBuilder.shuffleSpec(new HashShuffleSpec(new ClusterBy(leftPartitionKey, 0), maxWorkerCount)); + leftBuilder.shuffleSpec(new HashShuffleSpec(new ClusterBy(leftPartitionKey, 0), hashPartitionCount)); leftBuilder.signature(QueryKitUtils.sortableSignature(leftBuilder.getSignature(), leftPartitionKey)); // Build up the right stage. @@ -717,7 +670,7 @@ public class DataSourcePlan ); final List rightPartitionKey = partitionKeys.get(1); - rightBuilder.shuffleSpec(new HashShuffleSpec(new ClusterBy(rightPartitionKey, 0), maxWorkerCount)); + rightBuilder.shuffleSpec(new HashShuffleSpec(new ClusterBy(rightPartitionKey, 0), hashPartitionCount)); rightBuilder.signature(QueryKitUtils.sortableSignature(rightBuilder.getSignature(), rightPartitionKey)); // Compute join signature. @@ -745,7 +698,7 @@ public class DataSourcePlan Iterables.getOnlyElement(rightPlan.getInputSpecs()) ) ) - .maxWorkerCount(maxWorkerCount) + .maxWorkerCount(queryKitSpec.getMaxNonLeafWorkerCount()) .signature(joinSignatureBuilder.build()) .processorFactory( new SortMergeJoinFrameProcessorFactory( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/MultiQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/MultiQueryKit.java index a795f649605..3129bbfacb9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/MultiQueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/MultiQueryKit.java @@ -41,11 +41,9 @@ public class MultiQueryKit implements QueryKit> @Override public QueryDefinition makeQueryDefinition( - String queryId, + QueryKitSpec queryKitSpec, Query query, - QueryKit> toolKitForSubQueries, ShuffleSpecFactory resultShuffleSpecFactory, - int maxWorkerCount, int minStageNumber ) { @@ -54,11 +52,9 @@ public class MultiQueryKit implements QueryKit> if (specificToolKit != null) { //noinspection unchecked return specificToolKit.makeQueryDefinition( - queryId, + queryKitSpec, query, - this, resultShuffleSpecFactory, - maxWorkerCount, minStageNumber ); } else { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKit.java index b259022bba5..118091ccbd4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKit.java @@ -30,23 +30,17 @@ public interface QueryKit> /** * Creates a {@link QueryDefinition} from a {@link Query}. * - * @param queryId query ID of the resulting {@link QueryDefinition} + * @param queryKitSpec collection of parameters necessary for planning {@link QueryDefinition} * @param query native query to translate - * @param toolKitForSubQueries kit that is used to translate native subqueries; i.e., - * {@link org.apache.druid.query.QueryDataSource}. Typically a {@link MultiQueryKit}. * @param resultShuffleSpecFactory shuffle spec factory for the final output of this query. - * @param maxWorkerCount maximum number of workers: becomes - * {@link org.apache.druid.msq.kernel.StageDefinition#getMaxWorkerCount()} * @param minStageNumber lowest stage number to use for any generated stages. Useful if the resulting * {@link QueryDefinition} is going to be added to an existing * {@link org.apache.druid.msq.kernel.QueryDefinitionBuilder}. */ QueryDefinition makeQueryDefinition( - String queryId, + QueryKitSpec queryKitSpec, QueryType query, - QueryKit> toolKitForSubQueries, ShuffleSpecFactory resultShuffleSpecFactory, - int maxWorkerCount, int minStageNumber ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitSpec.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitSpec.java new file mode 100644 index 00000000000..7cae4ed7d7b --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/QueryKitSpec.java @@ -0,0 +1,109 @@ +/* + * 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 org.apache.druid.msq.input.InputSpec; +import org.apache.druid.msq.input.InputSpecs; +import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.query.Query; + +import java.util.List; + +/** + * Collection of parameters for {@link QueryKit#makeQueryDefinition}. + */ +public class QueryKitSpec +{ + private final QueryKit> queryKit; + private final String queryId; + private final int maxLeafWorkerCount; + private final int maxNonLeafWorkerCount; + private final int targetPartitionsPerWorker; + + /** + * @param queryKit kit that is used to translate native subqueries; i.e., + * {@link org.apache.druid.query.QueryDataSource}. Typically a {@link MultiQueryKit}. + * @param queryId queryId of the resulting {@link QueryDefinition} + * @param maxLeafWorkerCount maximum number of workers for leaf stages: becomes + * {@link org.apache.druid.msq.kernel.StageDefinition#getMaxWorkerCount()} + * @param maxNonLeafWorkerCount maximum number of workers for non-leaf stages: becomes + * {@link org.apache.druid.msq.kernel.StageDefinition#getMaxWorkerCount()} + * @param targetPartitionsPerWorker preferred number of partitions per worker for subqueries + */ + public QueryKitSpec( + QueryKit> queryKit, + String queryId, + int maxLeafWorkerCount, + int maxNonLeafWorkerCount, + int targetPartitionsPerWorker + ) + { + this.queryId = queryId; + this.queryKit = queryKit; + this.maxLeafWorkerCount = maxLeafWorkerCount; + this.maxNonLeafWorkerCount = maxNonLeafWorkerCount; + this.targetPartitionsPerWorker = targetPartitionsPerWorker; + } + + /** + * Instance of {@link QueryKit} for recursive calls. + */ + public QueryKit> getQueryKit() + { + return queryKit; + } + + /** + * Query ID to use when building {@link QueryDefinition}. + */ + public String getQueryId() + { + return queryId; + } + + /** + * Maximum worker count for a stage with the given inputs. Will use {@link #maxNonLeafWorkerCount} if there are + * any stage inputs, {@link #maxLeafWorkerCount} otherwise. + */ + public int getMaxWorkerCount(final List inputSpecs) + { + if (InputSpecs.getStageNumbers(inputSpecs).isEmpty()) { + return maxLeafWorkerCount; + } else { + return maxNonLeafWorkerCount; + } + } + + /** + * Maximum number of workers for non-leaf stages (where there are some stage inputs). + */ + public int getMaxNonLeafWorkerCount() + { + return maxNonLeafWorkerCount; + } + + /** + * Number of partitions to generate during a shuffle. + */ + public int getNumPartitionsForShuffle() + { + return maxNonLeafWorkerCount * targetPartitionsPerWorker; + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java index 9852f4f4098..6ad7742672f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorFactory.java @@ -174,6 +174,11 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor ); } + @Override + public boolean usesProcessingBuffers() + { + return false; + } @Override public boolean equals(Object o) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java index b3686359d2a..02542f8e736 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryKit.java @@ -35,7 +35,6 @@ import org.apache.druid.msq.kernel.QueryDefinitionBuilder; import org.apache.druid.msq.kernel.ShuffleSpec; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.util.MultiStageQueryContext; -import org.apache.druid.query.Query; import org.apache.druid.query.operator.ColumnWithDirection; import org.apache.druid.query.operator.NaivePartitioningOperatorFactory; import org.apache.druid.query.operator.NaiveSortOperatorFactory; @@ -63,11 +62,9 @@ public class WindowOperatorQueryKit implements QueryKit @Override public QueryDefinition makeQueryDefinition( - String queryId, + QueryKitSpec queryKitSpec, WindowOperatorQuery originalQuery, - QueryKit> queryKit, ShuffleSpecFactory resultShuffleSpecFactory, - int maxWorkerCount, int minStageNumber ) { @@ -89,20 +86,22 @@ public class WindowOperatorQueryKit implements QueryKit log.info("Created operatorList with operator factories: [%s]", operatorList); final DataSourcePlan dataSourcePlan = DataSourcePlan.forDataSource( - queryKit, - queryId, + queryKitSpec, originalQuery.context(), originalQuery.getDataSource(), originalQuery.getQuerySegmentSpec(), originalQuery.getFilter(), null, - maxWorkerCount, minStageNumber, false ); - ShuffleSpec nextShuffleSpec = findShuffleSpecForNextWindow(operatorList.get(0), maxWorkerCount); - final QueryDefinitionBuilder queryDefBuilder = makeQueryDefinitionBuilder(queryId, dataSourcePlan, nextShuffleSpec); + ShuffleSpec nextShuffleSpec = findShuffleSpecForNextWindow( + operatorList.get(0), + queryKitSpec.getNumPartitionsForShuffle() + ); + final QueryDefinitionBuilder queryDefBuilder = + makeQueryDefinitionBuilder(queryKitSpec.getQueryId(), dataSourcePlan, nextShuffleSpec); final int firstStageNumber = Math.max(minStageNumber, queryDefBuilder.getNextStageNumber()); final WindowOperatorQuery queryToRun = (WindowOperatorQuery) originalQuery.withDataSource(dataSourcePlan.getNewDataSource()); @@ -130,7 +129,7 @@ public class WindowOperatorQueryKit implements QueryKit StageDefinition.builder(firstStageNumber) .inputs(new StageInputSpec(firstStageNumber - 1)) .signature(finalWindowStageRowSignature) - .maxWorkerCount(maxWorkerCount) + .maxWorkerCount(queryKitSpec.getMaxNonLeafWorkerCount()) .shuffleSpec(finalWindowStageShuffleSpec) .processorFactory(new WindowOperatorQueryFrameProcessorFactory( queryToRun, @@ -192,7 +191,8 @@ public class WindowOperatorQueryKit implements QueryKit stageRowSignature = finalWindowStageRowSignature; nextShuffleSpec = finalWindowStageShuffleSpec; } else { - nextShuffleSpec = findShuffleSpecForNextWindow(operatorList.get(i + 1), maxWorkerCount); + nextShuffleSpec = + findShuffleSpecForNextWindow(operatorList.get(i + 1), queryKitSpec.getNumPartitionsForShuffle()); if (nextShuffleSpec == null) { stageRowSignature = intermediateSignature; } else { @@ -229,7 +229,7 @@ public class WindowOperatorQueryKit implements QueryKit StageDefinition.builder(firstStageNumber + i) .inputs(new StageInputSpec(firstStageNumber + i - 1)) .signature(stageRowSignature) - .maxWorkerCount(maxWorkerCount) + .maxWorkerCount(queryKitSpec.getMaxNonLeafWorkerCount()) .shuffleSpec(nextShuffleSpec) .processorFactory(new WindowOperatorQueryFrameProcessorFactory( queryToRun, @@ -285,7 +285,7 @@ public class WindowOperatorQueryKit implements QueryKit return operatorList; } - private ShuffleSpec findShuffleSpecForNextWindow(List operatorFactories, int maxWorkerCount) + private ShuffleSpec findShuffleSpecForNextWindow(List operatorFactories, int partitionCount) { NaivePartitioningOperatorFactory partition = null; NaiveSortOperatorFactory sort = null; @@ -325,7 +325,7 @@ public class WindowOperatorQueryKit implements QueryKit keyColsOfWindow.add(kc); } - return new HashShuffleSpec(new ClusterBy(keyColsOfWindow, 0), maxWorkerCount); + return new HashShuffleSpec(new ClusterBy(keyColsOfWindow, 0), partitionCount); } /** 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 d04a75011fa..a0332edb902 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 @@ -140,6 +140,12 @@ public class OffsetLimitFrameProcessorFactory extends BaseFrameProcessorFactory ); } + @Override + public boolean usesProcessingBuffers() + { + return false; + } + @Override public boolean equals(Object o) { 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 9eb95a468fd..55391b13861 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 @@ -197,6 +197,12 @@ public class SortMergeJoinFrameProcessorFactory extends BaseFrameProcessorFactor ); } + @Override + public boolean usesProcessingBuffers() + { + return false; + } + /** * Extracts key columns from a {@link JoinConditionAnalysis}. The returned list has two elements: 0 is the * left-hand side, 1 is the right-hand side. Each sub-list has one element for each equi-condition. 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 851fc21c52e..ab683c8329a 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 @@ -129,4 +129,10 @@ public class GroupByPostShuffleFrameProcessorFactory extends BaseFrameProcessorF OutputChannels.wrapReadOnly(ImmutableList.copyOf(outputChannels.values())) ); } + + @Override + public boolean usesProcessingBuffers() + { + return false; + } } 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 a859ea8cd53..ad456a45f5b 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,6 +20,7 @@ package org.apache.druid.msq.querykit.groupby; import com.google.common.collect.Iterables; +import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.frame.Frame; import org.apache.druid.frame.channel.FrameWithPartition; @@ -53,13 +54,14 @@ import org.apache.druid.query.groupby.epinephelinae.RowBasedGrouperHelper; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.Segment; +import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.timeline.SegmentId; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.List; import java.util.function.Function; @@ -72,6 +74,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor private static final Logger log = new Logger(GroupByPreShuffleFrameProcessor.class); private final GroupByQuery query; private final GroupingEngine groupingEngine; + private final NonBlockingPool bufferPool; private final ColumnSelectorFactory frameWriterColumnSelectorFactory; private final Closer closer = Closer.create(); @@ -84,6 +87,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor public GroupByPreShuffleFrameProcessor( final GroupByQuery query, final GroupingEngine groupingEngine, + final NonBlockingPool bufferPool, final ReadableInput baseInput, final Function segmentMapFn, final ResourceHolder outputChannelHolder, @@ -98,6 +102,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor ); this.query = query; this.groupingEngine = groupingEngine; + this.bufferPool = bufferPool; this.frameWriterColumnSelectorFactory = RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory( query, () -> resultYielder.get(), @@ -147,14 +152,15 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException { if (resultYielder == null) { - final ResourceHolder segmentHolder = closer.register(segment.getOrLoad()); - final SegmentReference mappedSegment = mapSegment(segmentHolder.get()); + final ResourceHolder segmentHolder = closer.register(segment.getOrLoad()); + final SegmentReference mappedSegment = mapSegment(segmentHolder.get().getSegment()); final Sequence rowSequence = groupingEngine.process( query.withQuerySegmentSpec(new SpecificSegmentSpec(segment.getDescriptor())), - mappedSegment.asStorageAdapter(), + mappedSegment.asCursorFactory(), mappedSegment.as(TimeBoundaryInspector.class), + bufferPool, null ); @@ -187,8 +193,9 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor final Sequence rowSequence = groupingEngine.process( query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)), - mappedSegment.asStorageAdapter(), + mappedSegment.asCursorFactory(), mappedSegment.as(TimeBoundaryInspector.class), + bufferPool, null ); 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 71f1e531b40..5ae163c1fd0 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 @@ -65,10 +65,17 @@ public class GroupByPreShuffleFrameProcessorFactory extends BaseLeafFrameProcess return new GroupByPreShuffleFrameProcessor( query, frameContext.groupingEngine(), + frameContext.processingBuffers().getBufferPool(), baseInput, segmentMapFn, outputChannelHolder, frameWriterFactoryHolder ); } + + @Override + public boolean usesProcessingBuffers() + { + return true; + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java index 7e4ebf5e7fa..db56bd02f74 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByQueryKit.java @@ -34,12 +34,12 @@ import org.apache.druid.msq.kernel.ShuffleSpec; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.querykit.DataSourcePlan; import org.apache.druid.msq.querykit.QueryKit; +import org.apache.druid.msq.querykit.QueryKitSpec; import org.apache.druid.msq.querykit.QueryKitUtils; import org.apache.druid.msq.querykit.ShuffleSpecFactories; import org.apache.druid.msq.querykit.ShuffleSpecFactory; import org.apache.druid.msq.querykit.common.OffsetLimitFrameProcessorFactory; import org.apache.druid.query.DimensionComparisonUtils; -import org.apache.druid.query.Query; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.having.AlwaysHavingSpec; @@ -66,26 +66,22 @@ public class GroupByQueryKit implements QueryKit @Override public QueryDefinition makeQueryDefinition( - final String queryId, + final QueryKitSpec queryKitSpec, final GroupByQuery originalQuery, - final QueryKit> queryKit, final ShuffleSpecFactory resultShuffleSpecFactory, - final int maxWorkerCount, final int minStageNumber ) { validateQuery(originalQuery); - final QueryDefinitionBuilder queryDefBuilder = QueryDefinition.builder(queryId); + final QueryDefinitionBuilder queryDefBuilder = QueryDefinition.builder(queryKitSpec.getQueryId()); final DataSourcePlan dataSourcePlan = DataSourcePlan.forDataSource( - queryKit, - queryId, + queryKitSpec, originalQuery.context(), originalQuery.getDataSource(), originalQuery.getQuerySegmentSpec(), originalQuery.getFilter(), null, - maxWorkerCount, minStageNumber, false ); @@ -139,9 +135,10 @@ public class GroupByQueryKit implements QueryKit // __time in such queries is generated using either an aggregator (e.g. sum(metric) as __time) or using a // post-aggregator (e.g. TIMESTAMP '2000-01-01' as __time) // For example: INSERT INTO foo SELECT COUNT(*), TIMESTAMP '2000-01-01' AS __time FROM bar PARTITIONED BY DAY - shuffleSpecFactoryPreAggregation = intermediateClusterBy.isEmpty() - ? ShuffleSpecFactories.singlePartition() - : ShuffleSpecFactories.globalSortWithMaxPartitionCount(maxWorkerCount); + shuffleSpecFactoryPreAggregation = + intermediateClusterBy.isEmpty() + ? ShuffleSpecFactories.singlePartition() + : ShuffleSpecFactories.globalSortWithMaxPartitionCount(queryKitSpec.getNumPartitionsForShuffle()); if (doLimitOrOffset) { shuffleSpecFactoryPostAggregation = ShuffleSpecFactories.singlePartitionWithLimit(postAggregationLimitHint); @@ -166,7 +163,10 @@ public class GroupByQueryKit implements QueryKit .broadcastInputs(dataSourcePlan.getBroadcastInputs()) .signature(intermediateSignature) .shuffleSpec(shuffleSpecFactoryPreAggregation.build(intermediateClusterBy, true)) - .maxWorkerCount(dataSourcePlan.isSingleWorker() ? 1 : maxWorkerCount) + .maxWorkerCount( + dataSourcePlan.isSingleWorker() + ? 1 + : queryKitSpec.getMaxWorkerCount(dataSourcePlan.getInputSpecs())) .processorFactory(new GroupByPreShuffleFrameProcessorFactory(queryToRun)) ); @@ -186,7 +186,7 @@ public class GroupByQueryKit implements QueryKit StageDefinition.builder(firstStageNumber + 1) .inputs(new StageInputSpec(firstStageNumber)) .signature(resultSignature) - .maxWorkerCount(maxWorkerCount) + .maxWorkerCount(queryKitSpec.getMaxNonLeafWorkerCount()) .shuffleSpec( shuffleSpecFactoryPostAggregation != null ? shuffleSpecFactoryPostAggregation.build(resultClusterBy, false) @@ -387,7 +387,10 @@ public class GroupByQueryKit implements QueryKit for (final OrderByColumnSpec column : defaultLimitSpec.getColumns()) { final Optional type = resultSignature.getColumnType(column.getDimension()); - if (!type.isPresent() || !DimensionComparisonUtils.isNaturalComparator(type.get().getType(), column.getDimensionComparator())) { + if (!type.isPresent() || !DimensionComparisonUtils.isNaturalComparator( + type.get().getType(), + column.getDimensionComparator() + )) { throw new ISE( "Must use natural comparator for column [%s] of type [%s]", column.getDimension(), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java index 665cfab89ee..2c4a92efef6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java @@ -31,8 +31,7 @@ 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.frame.segment.FrameStorageAdapter; -import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.frame.segment.FrameSegment; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.msq.exec.ResultsContext; import org.apache.druid.segment.BaseObjectColumnValueSelector; @@ -40,13 +39,14 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.Segment; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.run.SqlResults; import org.apache.druid.sql.http.ResultFormat; import org.apache.druid.storage.StorageConnector; +import org.apache.druid.timeline.SegmentId; import java.io.IOException; import java.io.OutputStream; @@ -151,8 +151,8 @@ public class ExportResultsFrameProcessor implements FrameProcessor private void exportFrame(final Frame frame) { - final StorageAdapter adapter = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + final Segment segment = new FrameSegment(frame, frameReader, SegmentId.dummy("test")); + try (final CursorHolder cursorHolder = segment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { exportWriter.writeRowEnd(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessorFactory.java index 930ba886155..fe2598a9514 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessorFactory.java @@ -117,6 +117,12 @@ public class ExportResultsFrameProcessorFactory implements FrameProcessorFactory return resultsContext; } + @Override + public boolean usesProcessingBuffers() + { + return false; + } + @Override public ProcessorsAndChannels makeProcessors( StageDefinition stageDefinition, 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 17fc6c94817..b9befa8374d 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 @@ -115,4 +115,10 @@ public class QueryResultFrameProcessorFactory extends BaseFrameProcessorFactory OutputChannels.wrapReadOnly(ImmutableList.copyOf(outputChannels.values())) ); } + + @Override + public boolean usesProcessingBuffers() + { + return false; + } } 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 f402aa60430..e5fa0a03d62 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 @@ -64,13 +64,14 @@ import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanQueryEngine; import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; 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; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.RowSignature; @@ -245,17 +246,18 @@ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment) throws IOException { if (cursor == null) { - final ResourceHolder segmentHolder = closer.register(segment.getOrLoad()); + final ResourceHolder segmentHolder = closer.register(segment.getOrLoad()); - final StorageAdapter adapter = mapSegment(segmentHolder.get()).asStorageAdapter(); - if (adapter == null) { + final Segment mappedSegment = mapSegment(segmentHolder.get().getSegment()); + final CursorFactory cursorFactory = mappedSegment.asCursorFactory(); + if (cursorFactory == null) { throw new ISE( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." ); } final CursorHolder cursorHolder = closer.register( - adapter.makeCursorHolder(ScanQueryEngine.makeCursorBuildSpec(query, null)) + cursorFactory.makeCursorHolder(ScanQueryEngine.makeCursorBuildSpec(query, null)) ); final Cursor nextCursor = cursorHolder.asCursor(); @@ -263,7 +265,7 @@ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor // No cursors! return ReturnOrAwait.returnObject(Unit.instance()); } else { - final long rowsFlushed = setNextCursor(nextCursor, segmentHolder.get()); + final long rowsFlushed = setNextCursor(nextCursor, segmentHolder.get().getSegment()); assert rowsFlushed == 0; // There's only ever one cursor when running with a segment } } @@ -292,15 +294,16 @@ public class ScanQueryFrameProcessor extends BaseLeafFrameProcessor final Frame frame = inputChannel.read(); final FrameSegment frameSegment = new FrameSegment(frame, inputFrameReader, SegmentId.dummy("scan")); - final StorageAdapter adapter = mapSegment(frameSegment).asStorageAdapter(); - if (adapter == null) { + final Segment mappedSegment = mapSegment(frameSegment); + final CursorFactory cursorFactory = mappedSegment.asCursorFactory(); + if (cursorFactory == null) { throw new ISE( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." ); } final CursorHolder cursorHolder = closer.register( - adapter.makeCursorHolder(ScanQueryEngine.makeCursorBuildSpec(query, null)) + cursorFactory.makeCursorHolder(ScanQueryEngine.makeCursorBuildSpec(query, null)) ); final Cursor nextCursor = cursorHolder.asCursor(); 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 e3d3619dd95..97ade19f5bc 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 @@ -85,4 +85,10 @@ public class ScanQueryFrameProcessorFactory extends BaseLeafFrameProcessorFactor frameWriterFactoryHolder ); } + + @Override + public boolean usesProcessingBuffers() + { + return false; + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java index f4f50106e81..8d23e289bb6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ScanQueryKit.java @@ -33,13 +33,13 @@ import org.apache.druid.msq.kernel.ShuffleSpec; import org.apache.druid.msq.kernel.StageDefinition; import org.apache.druid.msq.querykit.DataSourcePlan; import org.apache.druid.msq.querykit.QueryKit; +import org.apache.druid.msq.querykit.QueryKitSpec; import org.apache.druid.msq.querykit.QueryKitUtils; import org.apache.druid.msq.querykit.ShuffleSpecFactories; import org.apache.druid.msq.querykit.ShuffleSpecFactory; import org.apache.druid.msq.querykit.common.OffsetLimitFrameProcessorFactory; import org.apache.druid.query.Order; import org.apache.druid.query.OrderBy; -import org.apache.druid.query.Query; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -86,24 +86,20 @@ public class ScanQueryKit implements QueryKit // partition without a ClusterBy, we don't need to necessarily create it via the resultShuffleSpecFactory provided @Override public QueryDefinition makeQueryDefinition( - final String queryId, + final QueryKitSpec queryKitSpec, final ScanQuery originalQuery, - final QueryKit> queryKit, final ShuffleSpecFactory resultShuffleSpecFactory, - final int maxWorkerCount, final int minStageNumber ) { - final QueryDefinitionBuilder queryDefBuilder = QueryDefinition.builder(queryId); + final QueryDefinitionBuilder queryDefBuilder = QueryDefinition.builder(queryKitSpec.getQueryId()); final DataSourcePlan dataSourcePlan = DataSourcePlan.forDataSource( - queryKit, - queryId, + queryKitSpec, originalQuery.context(), originalQuery.getDataSource(), originalQuery.getQuerySegmentSpec(), originalQuery.getFilter(), null, - maxWorkerCount, minStageNumber, false ); @@ -177,7 +173,10 @@ public class ScanQueryKit implements QueryKit .broadcastInputs(dataSourcePlan.getBroadcastInputs()) .shuffleSpec(scanShuffleSpec) .signature(signatureToUse) - .maxWorkerCount(dataSourcePlan.isSingleWorker() ? 1 : maxWorkerCount) + .maxWorkerCount( + dataSourcePlan.isSingleWorker() + ? 1 + : queryKitSpec.getMaxWorkerCount(dataSourcePlan.getInputSpecs())) .processorFactory(new ScanQueryFrameProcessorFactory(queryToRun)) ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/BaseWorkerClientImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/BaseWorkerClientImpl.java index fd1a0323d0f..d6e7d412aca 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/BaseWorkerClientImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/rpc/BaseWorkerClientImpl.java @@ -26,6 +26,7 @@ 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.channel.ChannelClosedForWritesException; import org.apache.druid.frame.channel.ReadableByteChunksFrameChannel; import org.apache.druid.frame.file.FrameFileHttpResponseHandler; import org.apache.druid.frame.file.FrameFilePartialFetch; @@ -219,12 +220,18 @@ public abstract class BaseWorkerClientImpl implements WorkerClient public void onSuccess(FrameFilePartialFetch partialFetch) { if (partialFetch.isExceptionCaught()) { - // Exception while reading channel. Recoverable. - log.noStackTrace().info( - partialFetch.getExceptionCaught(), - "Encountered exception while reading channel [%s]", - channel.getId() - ); + if (partialFetch.getExceptionCaught() instanceof ChannelClosedForWritesException) { + // Channel was closed. Stop trying. + retVal.setException(partialFetch.getExceptionCaught()); + return; + } else { + // Exception while reading channel. Recoverable. + log.noStackTrace().warn( + partialFetch.getExceptionCaught(), + "Attempting recovery after exception while reading channel[%s]", + channel.getId() + ); + } } // Empty fetch means this is the last fetch for the channel. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java index 830fb87e1b2..202c1c591b1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java @@ -33,7 +33,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.msq.exec.MSQTasks; -import org.apache.druid.msq.guice.MSQTerminalStageSpecFactory; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; @@ -42,6 +41,7 @@ import org.apache.druid.msq.indexing.destination.DurableStorageMSQDestination; import org.apache.druid.msq.indexing.destination.ExportMSQDestination; import org.apache.druid.msq.indexing.destination.MSQDestination; import org.apache.druid.msq.indexing.destination.MSQSelectDestination; +import org.apache.druid.msq.indexing.destination.MSQTerminalStageSpecFactory; import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.util.MSQTaskQueryMakerUtils; import org.apache.druid.msq.util.MultiStageQueryContext; @@ -68,6 +68,7 @@ import org.apache.druid.sql.calcite.table.RowSignatures; import org.apache.druid.sql.destination.ExportDestination; import org.apache.druid.sql.destination.IngestDestination; import org.apache.druid.sql.destination.TableDestination; +import org.apache.druid.sql.hook.DruidHook; import org.apache.druid.sql.http.ResultFormat; import org.joda.time.Interval; @@ -117,6 +118,8 @@ public class MSQTaskQueryMaker implements QueryMaker public QueryResponse runQuery(final DruidQuery druidQuery) { Hook.QUERY_PLAN.run(druidQuery.getQuery()); + plannerContext.dispatchHook(DruidHook.NATIVE_PLAN, druidQuery.getQuery()); + String taskId = MSQTasks.controllerTaskId(plannerContext.getSqlQueryId()); // SQL query context: context provided by the user, and potentially modified by handlers during planning. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java index 9e07a909f4e..bdebe32a16f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java @@ -42,7 +42,7 @@ import org.apache.druid.error.InvalidSqlInput; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.msq.guice.MSQTerminalStageSpecFactory; +import org.apache.druid.msq.indexing.destination.MSQTerminalStageSpecFactory; import org.apache.druid.msq.querykit.QueryKitUtils; import org.apache.druid.msq.util.ArrayIngestMode; import org.apache.druid.msq.util.DimensionSchemaUtils; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index ed6a7c0e7b9..4ed98dca594 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -119,7 +119,6 @@ public class MultiStageQueryContext public static final SegmentSource DEFAULT_INCLUDE_SEGMENT_SOURCE = SegmentSource.NONE; public static final String CTX_MAX_CONCURRENT_STAGES = "maxConcurrentStages"; - public static final int DEFAULT_MAX_CONCURRENT_STAGES = 1; public static final String CTX_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; private static final boolean DEFAULT_DURABLE_SHUFFLE_STORAGE = false; public static final String CTX_SELECT_DESTINATION = "selectDestination"; @@ -190,6 +189,12 @@ public class MultiStageQueryContext public static final String CTX_SKIP_TYPE_VERIFICATION = "skipTypeVerification"; + /** + * Number of partitions to target per worker when creating shuffle specs that involve specific numbers of + * partitions. This helps us utilize more parallelism when workers are multi-threaded. + */ + public static final String CTX_TARGET_PARTITIONS_PER_WORKER = "targetPartitionsPerWorker"; + private static final Pattern LOOKS_LIKE_JSON_ARRAY = Pattern.compile("^\\s*\\[.*", Pattern.DOTALL); public static String getMSQMode(final QueryContext queryContext) @@ -200,11 +205,14 @@ public class MultiStageQueryContext ); } - public static int getMaxConcurrentStages(final QueryContext queryContext) + public static int getMaxConcurrentStagesWithDefault( + final QueryContext queryContext, + final int defaultMaxConcurrentStages + ) { return queryContext.getInt( CTX_MAX_CONCURRENT_STAGES, - DEFAULT_MAX_CONCURRENT_STAGES + defaultMaxConcurrentStages ); } @@ -330,16 +338,6 @@ public class MultiStageQueryContext ); } - @Nullable - public static MSQSelectDestination getSelectDestinationOrNull(final QueryContext queryContext) - { - return QueryContexts.getAsEnum( - CTX_SELECT_DESTINATION, - queryContext.getString(CTX_SELECT_DESTINATION), - MSQSelectDestination.class - ); - } - public static int getRowsInMemory(final QueryContext queryContext) { return queryContext.getInt(CTX_ROWS_IN_MEMORY, DEFAULT_ROWS_IN_MEMORY); @@ -380,6 +378,14 @@ public class MultiStageQueryContext return queryContext.getEnum(CTX_ARRAY_INGEST_MODE, ArrayIngestMode.class, DEFAULT_ARRAY_INGEST_MODE); } + public static int getTargetPartitionsPerWorkerWithDefault( + final QueryContext queryContext, + final int defaultValue + ) + { + return queryContext.getInt(CTX_TARGET_PARTITIONS_PER_WORKER, defaultValue); + } + /** * See {@link #CTX_INCLUDE_ALL_COUNTERS}. */ diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerMemoryParametersTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerMemoryParametersTest.java index 9d27dcca666..d6ae0d7e190 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerMemoryParametersTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/ControllerMemoryParametersTest.java @@ -39,7 +39,7 @@ public class ControllerMemoryParametersTest 1 ); - Assert.assertEquals(100_400_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes()); + Assert.assertEquals(101_400_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes()); } @Test @@ -50,7 +50,7 @@ public class ControllerMemoryParametersTest 100 ); - Assert.assertEquals(103_800_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes()); + Assert.assertEquals(104_800_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes()); } @Test @@ -61,7 +61,7 @@ public class ControllerMemoryParametersTest 1 ); - Assert.assertEquals(49_200_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes()); + Assert.assertEquals(50_200_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes()); } @Test @@ -91,7 +91,6 @@ public class ControllerMemoryParametersTest Assert.assertEquals(1, fault.getServerWorkers()); Assert.assertEquals(NUM_PROCESSORS_IN_JVM, fault.getServerThreads()); Assert.assertEquals(24_000_000, fault.getUsableMemory()); - Assert.assertEquals(33_750_000, fault.getSuggestedServerMemory()); } @Test @@ -102,7 +101,7 @@ public class ControllerMemoryParametersTest 1 ); - Assert.assertEquals(25_000_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes()); + Assert.assertEquals(26_000_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes()); } private MemoryIntrospector makeMemoryIntrospector( @@ -111,11 +110,11 @@ public class ControllerMemoryParametersTest ) { return new MemoryIntrospectorImpl( - new TestLookupProvider(ImmutableMap.of()), totalMemoryInJvm, USABLE_MEMORY_FRACTION, numQueriesInJvm, - NUM_PROCESSORS_IN_JVM + NUM_PROCESSORS_IN_JVM, + new TestLookupProvider(ImmutableMap.of()) ); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java index 1cc756d7c57..1d253f376fc 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java @@ -80,6 +80,12 @@ public class MSQDrillWindowQueryTest extends DrillWindowQueryTest { return injector.getInstance(MSQTaskSqlEngine.class); } + + @Override + public Boolean isExplainSupported() + { + return false; + } } @Override 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 4449090fb69..af51c5dd4a7 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 @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; +import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.common.config.NullHandling; import org.apache.druid.error.DruidException; import org.apache.druid.error.DruidExceptionMatcher; @@ -34,6 +35,11 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.indexing.MSQSpec; +import org.apache.druid.msq.indexing.MSQTuningConfig; +import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; import org.apache.druid.msq.indexing.error.ColumnNameRestrictedFault; import org.apache.druid.msq.indexing.error.RowTooLargeFault; import org.apache.druid.msq.indexing.error.TooManySegmentsInTimeChunkFault; @@ -43,11 +49,20 @@ import org.apache.druid.msq.test.CounterSnapshotMatcher; import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; +import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.server.lookup.cache.LookupLoadingSpec; +import org.apache.druid.sql.calcite.planner.ColumnMapping; +import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.hamcrest.CoreMatchers; @@ -95,6 +110,7 @@ public class MSQInsertTest extends MSQTestBase }; return Arrays.asList(data); } + @MethodSource("data") @ParameterizedTest(name = "{index}:with context {0}") public void testInsertOnFoo1(String contextName, Map context) @@ -155,6 +171,105 @@ public class MSQInsertTest extends MSQTestBase } + @MethodSource("data") + @ParameterizedTest(name = "{index}:with context {0}") + public void testInsertOnFoo1WithSpec(String contextName, Map context) + { + List expectedRows = expectedFooRows(); + int expectedCounterRows = expectedRows.size(); + long[] expectedArray = createExpectedFrameArray(expectedCounterRows, 1); + + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("cnt", ColumnType.LONG).build(); + + MSQSpec msqSpec = new MSQSpec( + GroupByQuery.builder() + .setDataSource("foo") + .setInterval(Intervals.ONLY_ETERNITY) + .setDimFilter(notNull("dim1")) + .setGranularity(Granularities.ALL) + .setDimensions( + new DefaultDimensionSpec("__time", "d0", ColumnType.LONG), + new DefaultDimensionSpec("dim1", "d1", ColumnType.STRING) + ) + .setContext(ImmutableMap.builder() + .put("__user", "allowAll") + .put("enableWindowing", true) + .put("finalize", true) + .put("maxNumTasks", 2) + .put("maxParseExceptions", 0) + .put("sqlInsertSegmentGranularity", "\"DAY\"") + .put("sqlQueryId", "test-query") + .put("sqlStringifyArrays", false) + .build() + ) + .setLimitSpec(DefaultLimitSpec.builder() + .orderBy(OrderByColumnSpec.asc("d1")) + .build() + ) + .setAggregatorSpecs(new CountAggregatorFactory("a0")) + .setQuerySegmentSpec(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)) + .build(), + new ColumnMappings( + ImmutableList.of( + new ColumnMapping("d0", "__time"), + new ColumnMapping("d1", "dim1"), + new ColumnMapping("a0", "cnt")) + ), + new DataSourceMSQDestination( + "foo1", + Granularity.fromString("DAY"), + null, + null, + null, + null + ), + WorkerAssignmentStrategy.MAX, + MSQTuningConfig.defaultConfig() + ); + + ImmutableMap sqlContext = + ImmutableMap.builder() + .putAll(context) + .put("sqlInsertSegmentGranularity", "\"DAY\"") + .put("forceTimeChunkLock", true) + .build(); + + MSQControllerTask controllerTask = new MSQControllerTask( + TEST_CONTROLLER_TASK_ID, + msqSpec, + null, + sqlContext, + null, + ImmutableList.of(SqlTypeName.TIMESTAMP, SqlTypeName.VARCHAR, SqlTypeName.BIGINT), + ImmutableList.of(ColumnType.LONG, ColumnType.STRING, ColumnType.LONG), + null + ); + + testIngestQuery().setTaskSpec(controllerTask) + .setExpectedDataSource("foo1") + .setQueryContext(context) + .setExpectedRowSignature(rowSignature) + .setExpectedSegments(expectedFooSegments()) + .setExpectedResultRows(expectedRows) + .setExpectedMSQSegmentReport( + new MSQSegmentReport( + NumberedShardSpec.class.getSimpleName(), + "Using NumberedShardSpec to generate segments since the query is inserting rows." + ) + ) + .setExpectedSegmentGenerationProgressCountersForStageWorker( + CounterSnapshotMatcher + .with().segmentRowsProcessed(Arrays.stream(expectedArray).sum()), + 2, 0 + ) + .setExpectedLookupLoadingSpec(LookupLoadingSpec.ALL) + .verifyResults(); + + } + @MethodSource("data") @ParameterizedTest(name = "{index}:with context {0}") public void testInsertWithExistingTimeColumn(String contextName, Map context) throws IOException @@ -1346,7 +1461,7 @@ public class MSQInsertTest extends MSQTestBase final File toRead = getResourceAsTemporaryFile("/wikipedia-sampled.json"); final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(toRead.getAbsolutePath()); - Mockito.doReturn(500).when(workerMemoryParameters).getStandardFrameSize(); + Mockito.doReturn(500).when(workerMemoryParameters).getFrameSize(); testIngestQuery().setSql(" insert into foo1 SELECT\n" + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java index 879da23977e..8577dcd18d4 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQParseExceptionsTest.java @@ -157,7 +157,7 @@ public class MSQParseExceptionsTest extends MSQTestBase .dataSource( new ExternalDataSource( new LocalInputSource(null, null, ImmutableList.of(toRead), SystemFields.none()), - new CsvInputFormat(null, null, null, true, 0), + new CsvInputFormat(null, null, null, true, 0, null), RowSignature.builder() .add("timestamp", ColumnType.STRING) .add("agent_category", ColumnType.STRING) @@ -255,7 +255,7 @@ public class MSQParseExceptionsTest extends MSQTestBase .dataSource( new ExternalDataSource( new LocalInputSource(null, null, ImmutableList.of(toRead), SystemFields.none()), - new CsvInputFormat(null, null, null, true, 0), + new CsvInputFormat(null, null, null, true, 0, null), RowSignature.builder() .add("timestamp", ColumnType.STRING) .add("agent_category", ColumnType.STRING) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java index ff2b15121da..2ae3eaf0383 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQReplaceTest.java @@ -2643,8 +2643,8 @@ public class MSQReplaceTest extends MSQTestBase ); } else { partitionsSpec = new DimensionRangePartitionsSpec( - MultiStageQueryContext.getRowsPerSegment(QueryContext.of(context)), null, + MultiStageQueryContext.getRowsPerSegment(QueryContext.of(context)), partitionDimensions, false ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index 305cdebf691..a5822f3a0b7 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -1779,7 +1779,7 @@ public class MSQSelectTest extends MSQTestBase .setDataSource( new ExternalDataSource( new InlineInputSource("dim1\nabc\nxyz\ndef\nxyz\nabc\nxyz\nabc\nxyz\ndef\nbbb\naaa"), - new CsvInputFormat(null, null, null, true, 0), + new CsvInputFormat(null, null, null, true, 0, null), RowSignature.builder().add("dim1", ColumnType.STRING).build() ) ) @@ -2324,7 +2324,7 @@ public class MSQSelectTest extends MSQTestBase if (DURABLE_STORAGE.equals(contextName) || FAULT_TOLERANCE.equals(contextName)) { new File( localFileStorageDir, - DurableStorageUtils.getWorkerOutputSuccessFilePath("query-test-query", 0, 0) + DurableStorageUtils.getWorkerOutputSuccessFilePath(TEST_CONTROLLER_TASK_ID, 0, 0) ); Mockito.verify(localFileStorageConnector, Mockito.times(2)) @@ -2376,7 +2376,7 @@ public class MSQSelectTest extends MSQTestBase Collections.nCopies(numFiles, toRead), SystemFields.none() ), - new CsvInputFormat(null, null, null, true, 0), + new CsvInputFormat(null, null, null, true, 0, null), RowSignature.builder().add("timestamp", ColumnType.STRING).build() )) .intervals(querySegmentSpec(Filtration.eternity())) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java index 3a1e3d95f80..f694fc9f39f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java @@ -1848,7 +1848,7 @@ public class MSQWindowTest extends MSQTestBase .setSql( "select cityName, added, SUM(added) OVER () cc from wikipedia") .setQueryContext(customContext) - .setExpectedMSQFault(new TooManyRowsInAWindowFault(15921, 200)) + .setExpectedMSQFault(new TooManyRowsInAWindowFault(15922, 200)) .verifyResults(); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/QueryValidatorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/QueryValidatorTest.java index d7364124483..c1d1030fb08 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/QueryValidatorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/QueryValidatorTest.java @@ -108,6 +108,7 @@ public class QueryValidatorTest Collections.singletonList(() -> inputFiles), // Slice with a large number of inputFiles null, null, + null, null ); @@ -125,7 +126,7 @@ public class QueryValidatorTest QueryValidator.validateWorkOrder(workOrder); } - private static QueryDefinition createQueryDefinition(int numColumns, int numWorkers) + public static QueryDefinition createQueryDefinition(int numColumns, int numWorkers) { QueryDefinitionBuilder builder = QueryDefinition.builder(UUID.randomUUID().toString()); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/RunWorkOrderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/RunWorkOrderTest.java new file mode 100644 index 00000000000..dbd6857b272 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/RunWorkOrderTest.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.msq.exec; + +import org.apache.druid.frame.processor.FrameProcessorExecutor; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.msq.indexing.error.MSQException; +import org.apache.druid.msq.kernel.FrameContext; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.ArgumentMatchers; +import org.mockito.Mockito; + +public class RunWorkOrderTest +{ + private static final String CANCELLATION_ID = "my-cancellation-id"; + + @Test + public void test_stopUnchecked() throws InterruptedException + { + final FrameProcessorExecutor exec = Mockito.mock(FrameProcessorExecutor.class); + final WorkerContext workerContext = Mockito.mock(WorkerContext.class); + final FrameContext frameContext = Mockito.mock(FrameContext.class); + final WorkerStorageParameters storageParameters = Mockito.mock(WorkerStorageParameters.class); + final RunWorkOrderListener listener = Mockito.mock(RunWorkOrderListener.class); + + Mockito.when(frameContext.storageParameters()).thenReturn(storageParameters); + + final RunWorkOrder runWorkOrder = + new RunWorkOrder(null, null, null, exec, CANCELLATION_ID, workerContext, frameContext, listener, false, false); + + runWorkOrder.stopUnchecked(null); + + // Calling a second time doesn't do anything special. + runWorkOrder.stopUnchecked(null); + + Mockito.verify(exec).cancel(CANCELLATION_ID); + Mockito.verify(frameContext).close(); + Mockito.verify(listener).onFailure(ArgumentMatchers.any(MSQException.class)); + } + + @Test + public void test_stopUnchecked_error() throws InterruptedException + { + final FrameProcessorExecutor exec = Mockito.mock(FrameProcessorExecutor.class); + final WorkerContext workerContext = Mockito.mock(WorkerContext.class); + final FrameContext frameContext = Mockito.mock(FrameContext.class); + final WorkerStorageParameters storageParameters = Mockito.mock(WorkerStorageParameters.class); + final RunWorkOrderListener listener = Mockito.mock(RunWorkOrderListener.class); + + Mockito.when(frameContext.storageParameters()).thenReturn(storageParameters); + + final RunWorkOrder runWorkOrder = + new RunWorkOrder(null, null, null, exec, CANCELLATION_ID, workerContext, frameContext, listener, false, false); + + final ISE exception = new ISE("oops"); + + Assert.assertThrows( + IllegalStateException.class, + () -> runWorkOrder.stopUnchecked(exception) + ); + + // Calling a second time doesn't do anything special. We already tried our best. + runWorkOrder.stopUnchecked(null); + + Mockito.verify(exec).cancel(CANCELLATION_ID); + Mockito.verify(frameContext).close(); + Mockito.verify(listener).onFailure(ArgumentMatchers.eq(exception)); + } + + @Test + public void test_stopUnchecked_errorDuringExecCancel() throws InterruptedException + { + final FrameProcessorExecutor exec = Mockito.mock(FrameProcessorExecutor.class); + final WorkerContext workerContext = Mockito.mock(WorkerContext.class); + final FrameContext frameContext = Mockito.mock(FrameContext.class); + final WorkerStorageParameters storageParameters = Mockito.mock(WorkerStorageParameters.class); + final RunWorkOrderListener listener = Mockito.mock(RunWorkOrderListener.class); + + final ISE exception = new ISE("oops"); + Mockito.when(frameContext.storageParameters()).thenReturn(storageParameters); + Mockito.doThrow(exception).when(exec).cancel(CANCELLATION_ID); + + final RunWorkOrder runWorkOrder = + new RunWorkOrder(null, null, null, exec, CANCELLATION_ID, workerContext, frameContext, listener, false, false); + + Assert.assertThrows( + IllegalStateException.class, + () -> runWorkOrder.stopUnchecked(null) + ); + + Mockito.verify(exec).cancel(CANCELLATION_ID); + Mockito.verify(frameContext).close(); + Mockito.verify(listener).onFailure(ArgumentMatchers.eq(exception)); + } + + @Test + public void test_stopUnchecked_errorDuringFrameContextClose() throws InterruptedException + { + final FrameProcessorExecutor exec = Mockito.mock(FrameProcessorExecutor.class); + final WorkerContext workerContext = Mockito.mock(WorkerContext.class); + final FrameContext frameContext = Mockito.mock(FrameContext.class); + final WorkerStorageParameters storageParameters = Mockito.mock(WorkerStorageParameters.class); + final RunWorkOrderListener listener = Mockito.mock(RunWorkOrderListener.class); + + final ISE exception = new ISE("oops"); + Mockito.when(frameContext.storageParameters()).thenReturn(storageParameters); + Mockito.doThrow(exception).when(frameContext).close(); + + final RunWorkOrder runWorkOrder = + new RunWorkOrder(null, null, null, exec, CANCELLATION_ID, workerContext, frameContext, listener, false, false); + + Assert.assertThrows( + IllegalStateException.class, + () -> runWorkOrder.stopUnchecked(null) + ); + + Mockito.verify(exec).cancel(CANCELLATION_ID); + Mockito.verify(frameContext).close(); + Mockito.verify(listener).onFailure(ArgumentMatchers.eq(exception)); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java index b5141e12dc8..a9bf5e91bb4 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TaskDataSegmentProviderTest.java @@ -43,11 +43,11 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.msq.counters.ChannelCounters; import org.apache.druid.query.OrderBy; +import org.apache.druid.segment.CompleteSegment; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnHolder; @@ -187,7 +187,7 @@ public class TaskDataSegmentProviderTest for (int i = 0; i < iterations; i++) { final int expectedSegmentNumber = i % NUM_SEGMENTS; final DataSegment segment = segments.get(expectedSegmentNumber); - final ListenableFuture>> f = + final ListenableFuture>> f = exec.submit(() -> provider.fetchSegment(segment.getId(), new ChannelCounters(), false)); testFutures.add( @@ -195,8 +195,8 @@ public class TaskDataSegmentProviderTest f, holderSupplier -> { try { - final ResourceHolder holder = holderSupplier.get(); - Assert.assertEquals(segment.getId(), holder.get().getId()); + final ResourceHolder holder = holderSupplier.get(); + Assert.assertEquals(segment.getId(), holder.get().getSegment().getId()); final String expectedStorageDir = DataSegmentPusher.getDefaultStorageDir(segment, false); final File expectedFile = new File( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TestMSQSqlModule.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TestMSQSqlModule.java index 5bf4bbd44dc..0b48d2904dd 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TestMSQSqlModule.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/TestMSQSqlModule.java @@ -26,7 +26,7 @@ import com.google.inject.Provides; import org.apache.druid.guice.LazySingleton; import org.apache.druid.initialization.ServerInjectorBuilderTest.TestDruidModule; import org.apache.druid.msq.guice.MultiStageQuery; -import org.apache.druid.msq.guice.SegmentGenerationTerminalStageSpecFactory; +import org.apache.druid.msq.indexing.destination.SegmentGenerationTerminalStageSpecFactory; import org.apache.druid.msq.sql.MSQTaskSqlEngine; import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.test.MSQTestOverlordServiceClient; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerImplTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerImplTest.java new file mode 100644 index 00000000000..32cd36d0998 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerImplTest.java @@ -0,0 +1,88 @@ +/* + * 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.exec; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.msq.kernel.WorkOrder; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.QueryContext; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.Map; + +public class WorkerImplTest +{ + @Test + public void test_makeWorkOrderToUse_nothingMissing() + { + final WorkOrder workOrder = new WorkOrder( + QueryValidatorTest.createQueryDefinition(10, 2), + 0, + 0, + Collections.singletonList(() -> 1), + null, + null, + OutputChannelMode.MEMORY, + ImmutableMap.of("foo", "bar") + ); + + Assert.assertSame( + workOrder, + WorkerImpl.makeWorkOrderToUse( + workOrder, + QueryContext.of(ImmutableMap.of("foo", "baz")) /* Conflicts with workOrder context; should be ignored */ + ) + ); + } + + @Test + public void test_makeWorkOrderToUse_missingOutputChannelModeAndWorkerContext() + { + final Map taskContext = + ImmutableMap.of("foo", "bar", MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, true); + + final WorkOrder workOrder = new WorkOrder( + QueryValidatorTest.createQueryDefinition(10, 2), + 1, + 2, + Collections.singletonList(() -> 1), + null, + null, + null, + null + ); + + Assert.assertEquals( + new WorkOrder( + workOrder.getQueryDefinition(), + workOrder.getStageNumber(), + workOrder.getWorkerNumber(), + workOrder.getInputs(), + null, + null, + OutputChannelMode.DURABLE_STORAGE_INTERMEDIATE, + taskContext + ), + WorkerImpl.makeWorkOrderToUse(workOrder, QueryContext.of(taskContext)) + ); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java index 1ead2a181fd..8e467f0ed69 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/WorkerMemoryParametersTest.java @@ -19,185 +19,436 @@ package org.apache.druid.msq.exec; +import com.google.common.collect.ImmutableList; +import it.unimi.dsi.fastutil.ints.IntSet; +import it.unimi.dsi.fastutil.ints.IntSets; import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.frame.key.ClusterBy; +import org.apache.druid.frame.key.KeyColumn; +import org.apache.druid.frame.key.KeyOrder; import org.apache.druid.msq.indexing.error.MSQException; -import org.apache.druid.msq.indexing.error.MSQFault; import org.apache.druid.msq.indexing.error.NotEnoughMemoryFault; -import org.apache.druid.msq.indexing.error.TooManyWorkersFault; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.stage.ReadablePartitions; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.kernel.GlobalSortTargetSizeShuffleSpec; +import org.apache.druid.msq.kernel.ShuffleSpec; import org.junit.Assert; import org.junit.Test; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + public class WorkerMemoryParametersTest { @Test - public void test_oneWorkerInJvm_alone() + public void test_1WorkerInJvm_alone_1Thread() { - Assert.assertEquals(params(335_500_000, 1, 41, 75_000_000), create(1_000_000_000, 1, 1, 1, 1, 0, 0)); - Assert.assertEquals(params(223_000_000, 2, 13, 75_000_000), create(1_000_000_000, 1, 2, 1, 1, 0, 0)); - Assert.assertEquals(params(133_000_000, 4, 3, 75_000_000), create(1_000_000_000, 1, 4, 1, 1, 0, 0)); - Assert.assertEquals(params(73_000_000, 3, 2, 75_000_000), create(1_000_000_000, 1, 8, 1, 1, 0, 0)); - Assert.assertEquals(params(49_923_076, 2, 2, 75_000_000), create(1_000_000_000, 1, 12, 1, 1, 0, 0)); + final int numThreads = 1; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; - final MSQException e = Assert.assertThrows( - MSQException.class, - () -> create(1_000_000_000, 1, 32, 1, 1, 0, 0) + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(973_000_000, frameSize, 1, 874, 97_300_000, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1) ); - Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32, 1), e.getFault()); - - final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 1, 0, 0)) - .getFault(); - - Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32, 1), fault); } @Test - public void test_oneWorkerInJvm_alone_twoConcurrentStages() + public void test_1WorkerInJvm_alone_withBroadcast_1Thread() { - Assert.assertEquals(params(166_750_000, 1, 20, 37_500_000), create(1_000_000_000, 1, 1, 2, 1, 0, 0)); - Assert.assertEquals(params(110_500_000, 2, 6, 37_500_000), create(1_000_000_000, 1, 2, 2, 1, 0, 0)); - Assert.assertEquals(params(65_500_000, 2, 3, 37_500_000), create(1_000_000_000, 1, 4, 2, 1, 0, 0)); - Assert.assertEquals(params(35_500_000, 1, 3, 37_500_000), create(1_000_000_000, 1, 8, 2, 1, 0, 0)); + final int numThreads = 1; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; - final MSQException e = Assert.assertThrows( - MSQException.class, - () -> create(1_000_000_000, 1, 12, 2, 1, 0, 0) + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads); + final List slices = makeInputSlices( + ReadablePartitions.striped(0, 1, numThreads), + ReadablePartitions.striped(0, 1, 1) ); + final IntSet broadcastInputs = IntSets.singleton(1); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); - Assert.assertEquals(new NotEnoughMemoryFault(1_736_034_666, 1_000_000_000, 750_000_000, 1, 12, 2), e.getFault()); - - final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 2, 1, 0, 0)) - .getFault(); - - Assert.assertEquals(new NotEnoughMemoryFault(4_048_090_666L, 1_000_000_000, 750_000_000, 2, 32, 2), fault); + Assert.assertEquals( + new WorkerMemoryParameters(673_000_000, frameSize, 1, 604, 67_300_000, 200_000_000), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1) + ); } @Test - public void test_oneWorkerInJvm_twoHundredWorkersInCluster() + public void test_1WorkerInJvm_alone_4Threads() { - Assert.assertEquals(params(474_000_000, 1, 83, 150_000_000), create(2_000_000_000, 1, 1, 1, 200, 0, 0)); - Assert.assertEquals(params(249_000_000, 2, 27, 150_000_000), create(2_000_000_000, 1, 2, 1, 200, 0, 0)); + final int numThreads = 4; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; - final MSQException e = Assert.assertThrows( - MSQException.class, - () -> create(1_000_000_000, 1, 4, 1, 200, 0, 0) + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(892_000_000, frameSize, 4, 199, 22_300_000, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1) ); - - Assert.assertEquals(new TooManyWorkersFault(200, 109), e.getFault()); } @Test - public void test_fourWorkersInJvm_twoHundredWorkersInCluster() + public void test_1WorkerInJvm_alone_withBroadcast_4Threads() { - Assert.assertEquals(params(1_014_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 1, 200, 0, 0)); - Assert.assertEquals(params(811_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 1, 200, 0, 0)); - Assert.assertEquals(params(558_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 1, 200, 0, 0)); - Assert.assertEquals(params(305_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 1, 200, 0, 0)); - Assert.assertEquals(params(102_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 1, 200, 0, 0)); + final int numThreads = 4; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; - final MSQException e = Assert.assertThrows( - MSQException.class, - () -> create(8_000_000_000L, 4, 32, 1, 200, 0, 0) + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads); + final List slices = makeInputSlices( + ReadablePartitions.striped(0, 1, numThreads), + ReadablePartitions.striped(0, 1, 1) ); + final IntSet broadcastInputs = IntSets.singleton(1); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); - Assert.assertEquals(new TooManyWorkersFault(200, 124), e.getFault()); - - // Make sure 124 actually works, and 125 doesn't. (Verify the error message above.) - Assert.assertEquals(params(25_000_000, 4, 3, 150_000_000), create(8_000_000_000L, 4, 32, 1, 124, 0, 0)); - - final MSQException e2 = Assert.assertThrows( - MSQException.class, - () -> create(8_000_000_000L, 4, 32, 1, 125, 0, 0) + Assert.assertEquals( + new WorkerMemoryParameters(592_000_000, frameSize, 4, 132, 14_800_000, 200_000_000), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1) ); - - Assert.assertEquals(new TooManyWorkersFault(125, 124), e2.getFault()); } @Test - public void test_fourWorkersInJvm_twoHundredWorkersInCluster_twoConcurrentStages() + public void test_1WorkerInJvm_alone_noStats_4Threads() { - Assert.assertEquals(params(406_500_000, 1, 74, 84_375_000), create(9_000_000_000L, 4, 1, 2, 200, 0, 0)); - Assert.assertEquals(params(305_250_000, 2, 30, 84_375_000), create(9_000_000_000L, 4, 2, 2, 200, 0, 0)); - Assert.assertEquals(params(178_687_500, 4, 10, 84_375_000), create(9_000_000_000L, 4, 4, 2, 200, 0, 0)); - Assert.assertEquals(params(52_125_000, 4, 6, 84_375_000), create(9_000_000_000L, 4, 8, 2, 200, 0, 0)); + final int numThreads = 4; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; - final MSQException e = Assert.assertThrows( - MSQException.class, - () -> create(8_000_000_000L, 4, 16, 2, 200, 0, 0) + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, 4); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = null; + + Assert.assertEquals( + new WorkerMemoryParameters(892_000_000, frameSize, 4, 222, 0, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1) ); - - Assert.assertEquals(new TooManyWorkersFault(200, 109), e.getFault()); - - // Make sure 109 actually works, and 110 doesn't. (Verify the error message above.) - Assert.assertEquals(params(25_000_000, 4, 3, 75_000_000), create(8_000_000_000L, 4, 16, 2, 109, 0, 0)); - - final MSQException e2 = Assert.assertThrows( - MSQException.class, - () -> create(8_000_000_000L, 4, 16, 2, 110, 0, 0) - ); - - Assert.assertEquals(new TooManyWorkersFault(110, 109), e2.getFault()); } @Test - public void test_oneWorkerInJvm_smallWorkerCapacity() + public void test_1WorkerInJvm_alone_2ConcurrentStages_4Threads() { - // Supersorter max channels per processer are one less than they are usually to account for extra frames that are required while creating composing output channels - Assert.assertEquals(params(41_200_000, 1, 3, 9_600_000), create(128_000_000, 1, 1, 1, 1, 0, 0)); - Assert.assertEquals(params(26_800_000, 1, 1, 9_600_000), create(128_000_000, 1, 2, 1, 1, 0, 0)); + final int numThreads = 4; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; - final MSQException e = Assert.assertThrows( - MSQException.class, - () -> create(1_000_000_000, 1, 32, 1, 1, 0, 0) + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(392_000_000, frameSize, 4, 87, 9_800_000, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 2, 1) ); - Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32, 1), e.getFault()); - - final MSQException e2 = Assert.assertThrows( - MSQException.class, - () -> create(128_000_000, 1, 4, 1, 1, 0, 0) - ); - Assert.assertEquals(new NotEnoughMemoryFault(580_006_666, 12_8000_000, 96_000_000, 1, 4, 1), e2.getFault()); - - final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 1, 0, 0)) - .getFault(); - - Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32, 1), fault); } @Test - public void test_fourWorkersInJvm_twoHundredWorkersInCluster_hashPartitions() + public void test_1WorkerInJvm_alone_2ConcurrentStages_4Threads_highHeap() { - Assert.assertEquals(params(814_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 1, 200, 200, 0)); - Assert.assertEquals(params(611_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 1, 200, 200, 0)); - Assert.assertEquals(params(358_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 1, 200, 200, 0)); - Assert.assertEquals(params(105_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 1, 200, 200, 0)); + final int numThreads = 4; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; - final MSQException e = Assert.assertThrows( - MSQException.class, - () -> create(9_000_000_000L, 4, 16, 1, 200, 200, 0) + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(6_250_000_000L, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(2_392_000_000L, frameSize, 4, 537, 59_800_000, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 2, 1) ); - - Assert.assertEquals(new TooManyWorkersFault(200, 138), e.getFault()); - - // Make sure 138 actually works, and 139 doesn't. (Verify the error message above.) - Assert.assertEquals(params(26_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 1, 138, 138, 0)); - - final MSQException e2 = Assert.assertThrows( - MSQException.class, - () -> create(9_000_000_000L, 4, 16, 1, 139, 139, 0) - ); - - Assert.assertEquals(new TooManyWorkersFault(139, 138), e2.getFault()); } @Test - public void test_oneWorkerInJvm_oneByteUsableMemory() + public void test_1WorkerInJvm_alone_32Threads() { + final int numThreads = 32; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(136_000_000, frameSize, 32, 2, 425_000, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1) + ); + } + + @Test + public void test_1WorkerInJvm_alone_33Threads() + { + final int numThreads = 33; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(109_000_000, frameSize, 32, 2, 330_303, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1) + ); + } + + @Test + public void test_1WorkerInJvm_alone_40Threads() + { + final int numThreads = 40; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + final MSQException e = Assert.assertThrows( MSQException.class, - () -> WorkerMemoryParameters.createInstance(1, 1, 1, 1, 32, 1, 1) + () -> WorkerMemoryParameters.createInstance( + memoryIntrospector, + frameSize, + slices, + broadcastInputs, + shuffleSpec, + 1, + 1 + ) ); - Assert.assertEquals(new NotEnoughMemoryFault(554669334, 1, 1, 1, 1, 1), e.getFault()); + Assert.assertEquals( + new NotEnoughMemoryFault(1_366_250_000, 1_250_000_000, 1_000_000_000, 1, 40, 1, 1), + e.getFault() + ); + } + + @Test + public void test_1WorkerInJvm_alone_40Threads_slightlyLessMemoryThanError() + { + // Test with one byte less than the amount of memory recommended in the error message + // for test_1WorkerInJvm_alone_40Threads. + final int numThreads = 40; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_366_250_000 - 1, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + final MSQException e = Assert.assertThrows( + MSQException.class, + () -> WorkerMemoryParameters.createInstance( + memoryIntrospector, + frameSize, + slices, + broadcastInputs, + shuffleSpec, + 1, + 1 + ) + ); + + Assert.assertEquals( + new NotEnoughMemoryFault(1_366_250_000, 1_366_249_999, 1_092_999_999, 1, 40, 1, 1), + e.getFault() + ); + } + + @Test + public void test_1WorkerInJvm_alone_40Threads_memoryFromError() + { + // Test with the amount of memory recommended in the error message for test_1WorkerInJvm_alone_40Threads. + final int numThreads = 40; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_366_250_000, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(13_000_000, frameSize, 1, 2, 250_000, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1) + ); + } + + @Test + public void test_1WorkerInJvm_alone_40Threads_2ConcurrentStages() + { + final int numThreads = 40; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + final MSQException e = Assert.assertThrows( + MSQException.class, + () -> WorkerMemoryParameters.createInstance( + memoryIntrospector, + frameSize, + slices, + broadcastInputs, + shuffleSpec, + 2, + 1 + ) + ); + + Assert.assertEquals( + new NotEnoughMemoryFault(2_732_500_000L, 1_250_000_000, 1_000_000_000, 1, 40, 1, 2), + e.getFault() + ); + } + + @Test + public void test_1WorkerInJvm_alone_40Threads_2ConcurrentStages_memoryFromError() + { + // Test with the amount of memory recommended in the error message from + // test_1WorkerInJvm_alone_40Threads_2ConcurrentStages. + final int numThreads = 40; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(2_732_500_000L, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(13_000_000, frameSize, 1, 2, 250_000, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 2, 1) + ); + } + + @Test + public void test_1WorkerInJvm_200WorkersInCluster_4Threads() + { + final int numThreads = 4; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(2_500_000_000L, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 200, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(1_096_000_000, frameSize, 4, 245, 27_400_000, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1) + ); + } + + @Test + public void test_1WorkerInJvm_200WorkersInCluster_4Threads_2OutputPartitions() + { + final int numThreads = 4; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(2_500_000_000L, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 200, 2)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(1_548_000_000, frameSize, 4, 347, 38_700_000, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1) + ); + } + + @Test + public void test_1WorkerInJvm_200WorkersInCluster_2ConcurrentStages_4Threads() + { + final int numThreads = 4; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(2_500_000_000L, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 200, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(96_000_000, frameSize, 4, 20, 2_500_000, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 2, 1) + ); + } + + @Test + public void test_12WorkersInJvm_200WorkersInCluster_64Threads_4OutputPartitions() + { + final int numThreads = 64; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(40_000_000_000L, 12, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 200, 4)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(1_762_666_666, frameSize, 64, 23, 2_754_166, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1) + ); + } + + @Test + public void test_12WorkersInJvm_200WorkersInCluster_2ConcurrentStages_64Threads_4OutputPartitions() + { + final int numThreads = 64; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(40_000_000_000L, 12, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, 200, 4)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(429_333_333, frameSize, 64, 5, 670_833, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 2, 1) + ); + } + + @Test + public void test_1WorkerInJvm_MaxWorkersInCluster_2ConcurrentStages_2Threads() + { + final int numWorkers = Limits.MAX_WORKERS; + final int numThreads = 2; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(6_250_000_000L, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, numWorkers, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(448_000_000, frameSize, 2, 200, 22_400_000, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 2, 1) + ); + } + + @Test + public void test_1WorkerInJvm_MaxWorkersInCluster_1Thread() + { + final int numWorkers = Limits.MAX_WORKERS; + final int numThreads = 1; + final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE; + + final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(2_500_000_000L, 1, numThreads); + final List slices = makeInputSlices(ReadablePartitions.striped(0, numWorkers, numThreads)); + final IntSet broadcastInputs = IntSets.emptySet(); + final ShuffleSpec shuffleSpec = makeSortShuffleSpec(); + + Assert.assertEquals( + new WorkerMemoryParameters(974_000_000, frameSize, 1, 875, 97_400_000, 0), + WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1) + ); } @Test @@ -206,39 +457,28 @@ public class WorkerMemoryParametersTest EqualsVerifier.forClass(WorkerMemoryParameters.class).usingGetClass().verify(); } - private static WorkerMemoryParameters params( - final long processorBundleMemory, - final int superSorterMaxActiveProcessors, - final int superSorterMaxChannelsPerProcessor, - final int partitionStatisticsMaxRetainedBytes + private static MemoryIntrospectorImpl createMemoryIntrospector( + final long totalMemory, + final int numTasksInJvm, + final int numProcessingThreads ) { - return new WorkerMemoryParameters( - processorBundleMemory, - superSorterMaxActiveProcessors, - superSorterMaxChannelsPerProcessor, - partitionStatisticsMaxRetainedBytes - ); + return new MemoryIntrospectorImpl(totalMemory, 0.8, numTasksInJvm, numProcessingThreads, null); } - private static WorkerMemoryParameters create( - final long maxMemoryInJvm, - final int numWorkersInJvm, - final int numProcessingThreadsInJvm, - final int maxConcurrentStages, - final int numInputWorkers, - final int numHashOutputPartitions, - final int totalLookUpFootprint - ) + private static List makeInputSlices(final ReadablePartitions... partitionss) { - return WorkerMemoryParameters.createInstance( - maxMemoryInJvm, - numWorkersInJvm, - numProcessingThreadsInJvm, - maxConcurrentStages, - numInputWorkers, - numHashOutputPartitions, - totalLookUpFootprint + return Arrays.stream(partitionss) + .map(partitions -> new StageInputSlice(0, partitions, OutputChannelMode.LOCAL_STORAGE)) + .collect(Collectors.toList()); + } + + private static ShuffleSpec makeSortShuffleSpec() + { + return new GlobalSortTargetSizeShuffleSpec( + new ClusterBy(ImmutableList.of(new KeyColumn("foo", KeyOrder.ASCENDING)), 0), + 1_000_000, + false ); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java index dfb88d17b21..8de80cf109f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/IndexerWorkerContextTest.java @@ -59,6 +59,7 @@ public class IndexerWorkerContextTest null, null, null, + null, null ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 34092d061b2..15b12be1575 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -93,6 +93,7 @@ public class MSQCompactionRunnerTest private static final String TIMESTAMP_COLUMN = ColumnHolder.TIME_COLUMN_NAME; private static final int TARGET_ROWS_PER_SEGMENT = 100000; + private static final int MAX_ROWS_PER_SEGMENT = 150000; private static final GranularityType SEGMENT_GRANULARITY = GranularityType.HOUR; private static final GranularityType QUERY_GRANULARITY = GranularityType.HOUR; private static List PARTITION_DIMENSIONS; @@ -258,19 +259,21 @@ public class MSQCompactionRunnerTest null ); - DataSchema dataSchema = new DataSchema( - DATA_SOURCE, - new TimestampSpec(TIMESTAMP_COLUMN, null, null), - new DimensionsSpec(DIMENSIONS), - new AggregatorFactory[]{}, - new UniformGranularitySpec( - SEGMENT_GRANULARITY.getDefaultGranularity(), - null, - false, - Collections.singletonList(COMPACTION_INTERVAL) - ), - new TransformSpec(dimFilter, Collections.emptyList()) - ); + DataSchema dataSchema = + DataSchema.builder() + .withDataSource(DATA_SOURCE) + .withTimestamp(new TimestampSpec(TIMESTAMP_COLUMN, null, null)) + .withDimensions(DIMENSIONS) + .withGranularity( + new UniformGranularitySpec( + SEGMENT_GRANULARITY.getDefaultGranularity(), + null, + false, + Collections.singletonList(COMPACTION_INTERVAL) + ) + ) + .withTransform(new TransformSpec(dimFilter, Collections.emptyList())) + .build(); List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( @@ -286,7 +289,7 @@ public class MSQCompactionRunnerTest new MSQTuningConfig( 1, MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY, - TARGET_ROWS_PER_SEGMENT, + MAX_ROWS_PER_SEGMENT, null, createIndexSpec() ), @@ -326,7 +329,7 @@ public class MSQCompactionRunnerTest DimFilter dimFilter = new SelectorDimFilter("dim1", "foo", null); CompactionTask taskCreatedWithTransformSpec = createCompactionTask( - new DimensionRangePartitionsSpec(TARGET_ROWS_PER_SEGMENT, null, PARTITION_DIMENSIONS, false), + new DimensionRangePartitionsSpec(null, MAX_ROWS_PER_SEGMENT, PARTITION_DIMENSIONS, false), dimFilter, Collections.emptyMap(), null, @@ -364,7 +367,7 @@ public class MSQCompactionRunnerTest new MSQTuningConfig( 1, MultiStageQueryContext.DEFAULT_ROWS_IN_MEMORY, - TARGET_ROWS_PER_SEGMENT, + MAX_ROWS_PER_SEGMENT, null, createIndexSpec() ), diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java index 76586c1e108..8d974285fb5 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQControllerTaskTest.java @@ -35,6 +35,7 @@ import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; import org.apache.druid.query.Druids; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.server.coordination.BroadcastDatasourceLoadingSpec; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; @@ -104,6 +105,22 @@ public class MSQControllerTaskTest Assert.assertEquals(LookupLoadingSpec.NONE, controllerTask.getLookupLoadingSpec()); } + @Test + public void testGetDefaultBroadcastDatasourceLoadingSpec() + { + MSQControllerTask controllerTask = new MSQControllerTask( + null, + MSQ_SPEC, + null, + null, + null, + null, + null, + null + ); + Assert.assertEquals(BroadcastDatasourceLoadingSpec.NONE, controllerTask.getBroadcastDatasourceLoadingSpec()); + } + @Test public void testGetLookupLoadingSpecUsingLookupLoadingInfoInContext() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/client/IndexerWorkerManagerClientTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/client/IndexerWorkerManagerClientTest.java deleted file mode 100644 index 4b53420cbb9..00000000000 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/client/IndexerWorkerManagerClientTest.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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.indexing.client; - -import com.google.common.util.concurrent.Futures; -import org.apache.druid.client.indexing.TaskStatusResponse; -import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexer.TaskState; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.TaskStatusPlus; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.rpc.indexing.OverlordClient; -import org.junit.Assert; -import org.junit.Test; -import org.mockito.ArgumentMatchers; -import org.mockito.Mockito; - -import java.util.Collections; - -public class IndexerWorkerManagerClientTest -{ - - @Test - public void testGetLocationCallsMultiStatusApiByDefault() - { - final OverlordClient overlordClient = Mockito.mock(OverlordClient.class); - - final String taskId = "worker1"; - final TaskLocation expectedLocation = new TaskLocation("localhost", 1000, 1100, null); - Mockito.when(overlordClient.taskStatuses(Collections.singleton(taskId))).thenReturn( - Futures.immediateFuture( - Collections.singletonMap( - taskId, - new TaskStatus(taskId, TaskState.RUNNING, 100L, null, expectedLocation) - ) - ) - ); - - final IndexerWorkerManagerClient managerClient = new IndexerWorkerManagerClient(overlordClient); - Assert.assertEquals(managerClient.location(taskId), expectedLocation); - - Mockito.verify(overlordClient, Mockito.times(1)).taskStatuses(ArgumentMatchers.anySet()); - Mockito.verify(overlordClient, Mockito.never()).taskStatus(ArgumentMatchers.anyString()); - } - - @Test - public void testGetLocationFallsBackToSingleTaskApiIfLocationIsUnknown() - { - final OverlordClient overlordClient = Mockito.mock(OverlordClient.class); - - final String taskId = "worker1"; - Mockito.when(overlordClient.taskStatuses(Collections.singleton(taskId))).thenReturn( - Futures.immediateFuture( - Collections.singletonMap( - taskId, - new TaskStatus(taskId, TaskState.RUNNING, 100L, null, TaskLocation.unknown()) - ) - ) - ); - - final TaskLocation expectedLocation = new TaskLocation("localhost", 1000, 1100, null); - final TaskStatusPlus taskStatus = new TaskStatusPlus( - taskId, - null, - null, - DateTimes.nowUtc(), - DateTimes.nowUtc(), - TaskState.RUNNING, - null, - 100L, - expectedLocation, - "wiki", - null - ); - - Mockito.when(overlordClient.taskStatus(taskId)).thenReturn( - Futures.immediateFuture(new TaskStatusResponse(taskId, taskStatus)) - ); - - final IndexerWorkerManagerClient managerClient = new IndexerWorkerManagerClient(overlordClient); - Assert.assertEquals(managerClient.location(taskId), expectedLocation); - - Mockito.verify(overlordClient, Mockito.times(1)).taskStatuses(ArgumentMatchers.anySet()); - Mockito.verify(overlordClient, Mockito.times(1)).taskStatus(ArgumentMatchers.anyString()); - } - -} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java index 0c16ae08af5..55c6c48c1af 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/error/MSQFaultSerdeTest.java @@ -74,7 +74,7 @@ public class MSQFaultSerdeTest )); assertFaultSerde(new InvalidNullByteFault("the source", 1, "the column", "the value", 2)); assertFaultSerde(new InvalidFieldFault("the source", "the column", 1, "the error", "the log msg")); - assertFaultSerde(new NotEnoughMemoryFault(1000, 1000, 900, 1, 2, 2)); + assertFaultSerde(new NotEnoughMemoryFault(1234, 1000, 1000, 900, 1, 2, 2)); assertFaultSerde(QueryNotSupportedFault.INSTANCE); assertFaultSerde(new QueryRuntimeFault("new error", "base error")); assertFaultSerde(new QueryRuntimeFault("new error", null)); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/IndexerTableInputSpecSlicerTest.java similarity index 98% rename from extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java rename to extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/IndexerTableInputSpecSlicerTest.java index a27ae7d9780..ac864419abe 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/TableInputSpecSlicerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/input/table/IndexerTableInputSpecSlicerTest.java @@ -27,6 +27,7 @@ import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.msq.exec.SegmentSource; +import org.apache.druid.msq.indexing.IndexerTableInputSpecSlicer; import org.apache.druid.msq.input.NilInputSlice; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -41,7 +42,7 @@ import org.junit.Test; import java.util.Collections; -public class TableInputSpecSlicerTest extends InitializedNullHandlingTest +public class IndexerTableInputSpecSlicerTest extends InitializedNullHandlingTest { private static final String DATASOURCE = "test-ds"; private static final long BYTES_PER_SEGMENT = 1000; @@ -97,7 +98,7 @@ public class TableInputSpecSlicerTest extends InitializedNullHandlingTest BYTES_PER_SEGMENT ); private SegmentTimeline timeline; - private TableInputSpecSlicer slicer; + private IndexerTableInputSpecSlicer slicer; private TaskActionClient taskActionClient; @Before @@ -131,7 +132,7 @@ public class TableInputSpecSlicerTest extends InitializedNullHandlingTest } }; - slicer = new TableInputSpecSlicer( + slicer = new IndexerTableInputSpecSlicer( null /* not used for SegmentSource.NONE */, taskActionClient, SegmentSource.NONE diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelConfigTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelConfigTest.java new file mode 100644 index 00000000000..765101359f6 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/kernel/controller/ControllerQueryKernelConfigTest.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.msq.kernel.controller; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.msq.indexing.destination.DurableStorageMSQDestination; +import org.apache.druid.msq.indexing.destination.MSQDestination; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; +import java.util.Map; + +public class ControllerQueryKernelConfigTest +{ + @Test + public void testBuilder() + { + int maxRetainedPartitionSketchBytes = 1; + int maxConcurrentStages = 2; + boolean pipeline = false; + boolean durableStorage = true; + boolean faultTolerance = true; + MSQDestination destination = DurableStorageMSQDestination.instance(); + String controllerHost = "controllerHost"; + List workerIds = ImmutableList.of("worker1", "worker2"); + Map workerContextMap = ImmutableMap.of("foo", "bar"); + + final ControllerQueryKernelConfig config1 = new ControllerQueryKernelConfig( + maxRetainedPartitionSketchBytes, + maxConcurrentStages, + pipeline, + durableStorage, + faultTolerance, + destination, + controllerHost, + workerIds, + workerContextMap + ); + + final ControllerQueryKernelConfig config2 = ControllerQueryKernelConfig + .builder() + .maxRetainedPartitionSketchBytes(maxRetainedPartitionSketchBytes) + .maxConcurrentStages(maxConcurrentStages) + .pipeline(pipeline) + .durableStorage(durableStorage) + .faultTolerance(faultTolerance) + .destination(destination) + .controllerHost(controllerHost) + .workerIds(workerIds) + .workerContextMap(workerContextMap) + .build(); + + Assert.assertEquals(config1, config2); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(ControllerQueryKernelConfig.class) + .usingGetClass() + .verify(); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessorTest.java index b0135ccdd4d..4270fe8bdcc 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinSegmentMapFnProcessorTest.java @@ -45,8 +45,8 @@ import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.JoinDataSource; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContext; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.JoinType; @@ -74,7 +74,7 @@ public class BroadcastJoinSegmentMapFnProcessorTest extends InitializedNullHandl @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private StorageAdapter adapter; + private CursorFactory cursorFactory; private File testDataFile1; private File testDataFile2; private FrameReader frameReader1; @@ -84,11 +84,11 @@ public class BroadcastJoinSegmentMapFnProcessorTest extends InitializedNullHandl public void setUp() throws IOException { final ArenaMemoryAllocator allocator = ArenaMemoryAllocator.createOnHeap(10_000); - adapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + cursorFactory = new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); // File 1: the entire test dataset. testDataFile1 = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) // No particular reason to test with both frame types .allocator(allocator) .frames(), @@ -97,7 +97,7 @@ public class BroadcastJoinSegmentMapFnProcessorTest extends InitializedNullHandl // File 2: just two rows. testDataFile2 = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) // No particular reason to test with both frame types .allocator(allocator) .maxRowsPerFrame(1) @@ -106,8 +106,8 @@ public class BroadcastJoinSegmentMapFnProcessorTest extends InitializedNullHandl temporaryFolder.newFile() ); - frameReader1 = FrameReader.create(adapter.getRowSignature()); - frameReader2 = FrameReader.create(adapter.getRowSignature()); + frameReader1 = FrameReader.create(cursorFactory.getRowSignature()); + frameReader2 = FrameReader.create(cursorFactory.getRowSignature()); } @Test @@ -169,7 +169,7 @@ public class BroadcastJoinSegmentMapFnProcessorTest extends InitializedNullHandl Assert.assertEquals(1209, rowsFromStage3.size()); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), Sequences.simple(rowsFromStage3.stream().map(Arrays::asList).collect(Collectors.toList())) ); @@ -178,7 +178,7 @@ public class BroadcastJoinSegmentMapFnProcessorTest extends InitializedNullHandl Assert.assertEquals(2, rowsFromStage4.size()); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false).limit(2), + FrameTestUtil.readRowsFromCursorFactory(cursorFactory).limit(2), Sequences.simple(rowsFromStage4.stream().map(Arrays::asList).collect(Collectors.toList())) ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/ChainedProcessorManagerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/ChainedProcessorManagerTest.java index a7df04963c3..4fadb7700f7 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/ChainedProcessorManagerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/ChainedProcessorManagerTest.java @@ -32,9 +32,9 @@ import org.apache.druid.frame.channel.WritableFrameChannel; import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.frame.processor.FrameProcessorExecutorTest; import org.apache.druid.frame.processor.FrameProcessors; +import org.apache.druid.frame.processor.manager.NilFrameProcessor; import org.apache.druid.frame.processor.manager.ProcessorManager; import org.apache.druid.frame.processor.manager.ProcessorManagers; -import org.apache.druid.frame.processor.manager.SequenceProcessorManagerTest; import org.apache.druid.frame.processor.test.SimpleReturningFrameProcessor; import org.apache.druid.frame.processor.test.SingleChannelFrameProcessor; import org.apache.druid.frame.processor.test.SingleRowWritingFrameProcessor; @@ -184,7 +184,7 @@ public class ChainedProcessorManagerTest extends FrameProcessorExecutorTest.Base ProcessorManagers.of( ImmutableList.of( new SimpleReturningFrameProcessor<>(ImmutableList.of(4L, 5L, 6L)), - new SequenceProcessorManagerTest.NilFrameProcessor<>() + new NilFrameProcessor<>() ) ), (values) -> createNextProcessors( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/FrameProcessorTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/FrameProcessorTestBase.java index 75168e07ad8..cde2b0ea4e9 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/FrameProcessorTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/FrameProcessorTestBase.java @@ -20,6 +20,7 @@ package org.apache.druid.msq.querykit; import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; @@ -27,15 +28,13 @@ import org.apache.druid.frame.channel.BlockingQueueFrameChannel; import org.apache.druid.frame.key.KeyColumn; import org.apache.druid.frame.processor.FrameProcessorExecutor; import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.testutil.FrameSequenceBuilder; -import org.apache.druid.java.util.common.Intervals; 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; import org.apache.druid.msq.kernel.StageId; import org.apache.druid.msq.kernel.StagePartition; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; @@ -49,30 +48,32 @@ public class FrameProcessorTestBase extends InitializedNullHandlingTest { protected static final StagePartition STAGE_PARTITION = new StagePartition(new StageId("q", 0), 0); + private ListeningExecutorService innerExec; protected FrameProcessorExecutor exec; @Before public void setUp() { - exec = new FrameProcessorExecutor(MoreExecutors.listeningDecorator(Execs.singleThreaded("test-exec"))); + innerExec = MoreExecutors.listeningDecorator(Execs.singleThreaded("test-exec")); + exec = new FrameProcessorExecutor(innerExec); } @After public void tearDown() throws Exception { - exec.getExecutorService().shutdownNow(); - exec.getExecutorService().awaitTermination(10, TimeUnit.MINUTES); + innerExec.shutdownNow(); + innerExec.awaitTermination(10, TimeUnit.MINUTES); } - protected ReadableInput makeChannelFromAdapter( - final StorageAdapter adapter, + protected ReadableInput makeChannelFromCursorFactory( + final CursorFactory cursorFactory, final List keyColumns, int rowsPerInputFrame ) throws IOException { // Create a single, sorted frame. final FrameSequenceBuilder singleFrameBuilder = - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) .maxRowsPerFrame(Integer.MAX_VALUE) .sortBy(keyColumns); @@ -86,7 +87,7 @@ public class FrameProcessorTestBase extends InitializedNullHandlingTest final FrameReader frameReader = FrameReader.create(signature); final FrameSequenceBuilder frameSequenceBuilder = - FrameSequenceBuilder.fromAdapter(new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY)) + FrameSequenceBuilder.fromCursorFactory(frameReader.makeCursorFactory(frame)) .frameType(FrameType.ROW_BASED) .maxRowsPerFrame(rowsPerInputFrame); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java index 0b9b73facde..9d64fffe23e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessorTest.java @@ -47,8 +47,8 @@ import org.apache.druid.query.operator.window.WindowOperatorFactory; import org.apache.druid.query.operator.window.ranking.WindowRowNumberProcessor; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.LegacySegmentSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.RowBasedSegment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.timeline.SegmentId; @@ -222,14 +222,14 @@ public class WindowOperatorQueryFrameProcessorTest extends FrameProcessorTestBas signature ); - return makeChannelFromAdapter(segment.asStorageAdapter(), keyColumns); + return makeChannelFromCursorFactory(segment.asCursorFactory(), keyColumns); } - private ReadableInput makeChannelFromAdapter( - final StorageAdapter adapter, + private ReadableInput makeChannelFromCursorFactory( + final CursorFactory cursorFactory, final List keyColumns ) throws IOException { - return makeChannelFromAdapter(adapter, keyColumns, 1000); + return makeChannelFromCursorFactory(cursorFactory, keyColumns, 1000); } } 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 4a8ec435873..dcf5ffe8a87 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,8 +45,8 @@ import org.apache.druid.msq.indexing.error.TooManyRowsWithSameKeyFault; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.msq.querykit.FrameProcessorTestBase; import org.apache.druid.msq.test.LimitedFrameWriterFactory; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.RowBasedSegment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.join.JoinTestHelper; @@ -1491,17 +1491,17 @@ public class SortMergeJoinFrameProcessorTest extends FrameProcessorTestBase signature ) )) { - final StorageAdapter adapter = segment.asStorageAdapter(); - return makeChannelFromAdapter(adapter, keyColumns); + final CursorFactory cursorFactory = segment.asCursorFactory(); + return makeChannelFromCursorFactory(cursorFactory, keyColumns); } } - private ReadableInput makeChannelFromAdapter( - final StorageAdapter adapter, + private ReadableInput makeChannelFromCursorFactory( + final CursorFactory cursorFactory, final List keyColumns ) throws IOException { - return makeChannelFromAdapter(adapter, keyColumns, rowsPerInputFrame); + return makeChannelFromCursorFactory(cursorFactory, keyColumns, rowsPerInputFrame); } private FrameWriterFactory makeFrameWriterFactory(final RowSignature signature) 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 e300a416705..632cd66b61a 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 @@ -36,7 +36,7 @@ import org.apache.druid.msq.kernel.StagePartition; import org.apache.druid.msq.querykit.FrameProcessorTestBase; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.junit.Assert; import org.junit.Test; @@ -50,11 +50,11 @@ public class QueryResultsFrameProcessorTest extends FrameProcessorTestBase public void sanityTest() throws ExecutionException, InterruptedException, IOException { - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final FrameSequenceBuilder frameSequenceBuilder = - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .maxRowsPerFrame(5) .frameType(FrameType.ROW_BASED) .allocator(ArenaMemoryAllocator.createOnHeap(100_000)); @@ -85,7 +85,7 @@ public class QueryResultsFrameProcessorTest extends FrameProcessorTestBase FrameReader.create(signature) ); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, signature, false), + FrameTestUtil.readRowsFromCursorFactory(cursorFactory, signature, false), rowsFromProcessor ); 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 205f3495807..bfb511f949f 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 @@ -46,9 +46,10 @@ import org.apache.druid.msq.test.LimitedFrameWriterFactory; import org.apache.druid.query.Druids; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.junit.Assert; import org.junit.Test; @@ -62,11 +63,11 @@ public class ScanQueryFrameProcessorTest extends FrameProcessorTestBase @Test public void test_runWithInputChannel() throws Exception { - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final CursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final FrameSequenceBuilder frameSequenceBuilder = - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .maxRowsPerFrame(5) .frameType(FrameType.ROW_BASED) .allocator(ArenaMemoryAllocator.createOnHeap(100_000)); @@ -86,7 +87,7 @@ public class ScanQueryFrameProcessorTest extends FrameProcessorTestBase Druids.newScanQueryBuilder() .dataSource("test") .intervals(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)) - .columns(adapter.getRowSignature().getColumnNames()) + .columns(cursorFactory.getRowSignature().getColumnNames()) .build(); final StagePartition stagePartition = new StagePartition(new StageId("query", 0), 0); @@ -138,7 +139,7 @@ public class ScanQueryFrameProcessorTest extends FrameProcessorTestBase ); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, signature, false), + FrameTestUtil.readRowsFromCursorFactory(cursorFactory, signature, false), rowsFromProcessor ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/quidem/MSQQuidemTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/quidem/MSQQuidemTest.java index e5c3a28e5cc..30deed4b626 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/quidem/MSQQuidemTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/quidem/MSQQuidemTest.java @@ -25,7 +25,6 @@ import java.io.File; public class MSQQuidemTest extends DruidQuidemTestBase { - public MSQQuidemTest() { super(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/rpc/BaseWorkerClientImplTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/rpc/BaseWorkerClientImplTest.java new file mode 100644 index 00000000000..dd8633c886f --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/rpc/BaseWorkerClientImplTest.java @@ -0,0 +1,383 @@ +/* + * 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.rpc; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import it.unimi.dsi.fastutil.bytes.ByteArrays; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.channel.ByteTracker; +import org.apache.druid.frame.channel.ChannelClosedForWritesException; +import org.apache.druid.frame.channel.ReadableByteChunksFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.file.FrameFile; +import org.apache.druid.frame.file.FrameFileHttpResponseHandler; +import org.apache.druid.frame.file.FrameFileWriter; +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.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.ISE; +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.guava.Sequences; +import org.apache.druid.msq.exec.WorkerClient; +import org.apache.druid.msq.kernel.StageId; +import org.apache.druid.rpc.MockServiceClient; +import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.rpc.ServiceClient; +import org.apache.druid.segment.QueryableIndexCursorFactory; +import org.apache.druid.segment.TestIndex; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.utils.CloseableUtils; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; + +import javax.ws.rs.core.HttpHeaders; +import javax.ws.rs.core.MediaType; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.channels.Channels; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +public class BaseWorkerClientImplTest extends InitializedNullHandlingTest +{ + private static final String WORKER_ID = "w0"; + /** + * Bytes for a {@link FrameFile} with no frames. (Not an empty array.) + */ + private static byte[] NIL_FILE_BYTES; + /** + * Bytes for a {@link FrameFile} holding {@link TestIndex#getMMappedTestIndex()}. + */ + private static byte[] FILE_BYTES; + private static FrameReader FRAME_READER; + + private ObjectMapper jsonMapper; + private MockServiceClient workerServiceClient; + private WorkerClient workerClient; + private ExecutorService exec; + + @BeforeClass + public static void setupClass() + { + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex()); + + NIL_FILE_BYTES = toFileBytes(Sequences.empty()); + FILE_BYTES = toFileBytes( + FrameSequenceBuilder.fromCursorFactory(cursorFactory) + .frameType(FrameType.COLUMNAR) + .maxRowsPerFrame(10) + .frames() + ); + FRAME_READER = FrameReader.create(cursorFactory.getRowSignature()); + } + + @AfterClass + public static void afterClass() + { + NIL_FILE_BYTES = null; + FILE_BYTES = null; + FRAME_READER = null; + } + + @Before + public void setup() + { + jsonMapper = new DefaultObjectMapper(); + workerServiceClient = new MockServiceClient(); + workerClient = new TestWorkerClient(jsonMapper, workerServiceClient); + exec = Execs.singleThreaded(StringUtils.encodeForFormat("exec-for-" + getClass().getName()) + "-%s"); + } + + @After + public void tearDown() throws InterruptedException + { + workerServiceClient.verify(); + exec.shutdownNow(); + if (!exec.awaitTermination(1, TimeUnit.MINUTES)) { + throw new ISE("Timed out waiting for exec to finish"); + } + } + + @Test + public void test_fetchChannelData_empty() throws Exception + { + workerServiceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/channels/xyz/1/2?offset=0") + .header(HttpHeaders.ACCEPT_ENCODING, "identity"), + HttpResponseStatus.OK, + fetchChannelDataResponseHeaders(false), + NIL_FILE_BYTES + ).expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/channels/xyz/1/2?offset=" + NIL_FILE_BYTES.length) + .header(HttpHeaders.ACCEPT_ENCODING, "identity"), + HttpResponseStatus.OK, + fetchChannelDataResponseHeaders(true), + ByteArrays.EMPTY_ARRAY + ); + + // Perform the test. + final StageId stageId = new StageId("xyz", 1); + final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("testChannel", false); + final Future>> framesFuture = readChannelAsync(channel); + + Assert.assertFalse(workerClient.fetchChannelData(WORKER_ID, stageId, 2, 0, channel).get()); + Assert.assertTrue(workerClient.fetchChannelData(WORKER_ID, stageId, 2, NIL_FILE_BYTES.length, channel).get()); + channel.doneWriting(); // Caller is expected to call doneWriting after fetchChannelData returns true. + + Assert.assertEquals( + 0, + framesFuture.get().size() + ); + } + + @Test + public void test_fetchChannelData_empty_intoClosedChannel() + { + workerServiceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/channels/xyz/1/2?offset=0") + .header(HttpHeaders.ACCEPT_ENCODING, "identity"), + HttpResponseStatus.OK, + fetchChannelDataResponseHeaders(false), + NIL_FILE_BYTES + ); + + // Perform the test. + final StageId stageId = new StageId("xyz", 1); + final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("testChannel", false); + channel.close(); // ReadableFrameChannel's close() method. + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> workerClient.fetchChannelData(WORKER_ID, stageId, 2, 0, channel).get() + ); + + MatcherAssert.assertThat( + e.getCause(), + CoreMatchers.instanceOf(ChannelClosedForWritesException.class) + ); + } + + @Test + public void test_fetchChannelData_empty_retry500() throws Exception + { + workerServiceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/channels/xyz/1/2?offset=0") + .header(HttpHeaders.ACCEPT_ENCODING, "identity"), + HttpResponseStatus.INTERNAL_SERVER_ERROR, + ImmutableMap.of(), + ByteArrays.EMPTY_ARRAY + ).expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/channels/xyz/1/2?offset=0") + .header(HttpHeaders.ACCEPT_ENCODING, "identity"), + HttpResponseStatus.OK, + fetchChannelDataResponseHeaders(false), + NIL_FILE_BYTES + ).expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/channels/xyz/1/2?offset=" + NIL_FILE_BYTES.length) + .header(HttpHeaders.ACCEPT_ENCODING, "identity"), + HttpResponseStatus.OK, + fetchChannelDataResponseHeaders(true), + ByteArrays.EMPTY_ARRAY + ); + + // Perform the test. + final StageId stageId = new StageId("xyz", 1); + final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("testChannel", false); + final Future>> framesFuture = readChannelAsync(channel); + + Assert.assertFalse(workerClient.fetchChannelData(WORKER_ID, stageId, 2, 0, channel).get()); + Assert.assertFalse(workerClient.fetchChannelData(WORKER_ID, stageId, 2, 0, channel).get()); + Assert.assertTrue(workerClient.fetchChannelData(WORKER_ID, stageId, 2, NIL_FILE_BYTES.length, channel).get()); + channel.doneWriting(); // Caller is expected to call doneWriting after fetchChannelData returns true. + + Assert.assertEquals( + 0, + framesFuture.get().size() + ); + } + + @Test + public void test_fetchChannelData_empty_serviceClientError() + { + workerServiceClient.expectAndThrow( + new RequestBuilder(HttpMethod.GET, "/channels/xyz/1/2?offset=0") + .header(HttpHeaders.ACCEPT_ENCODING, "identity"), + new IOException("Some error") + ); + + // Perform the test. + final StageId stageId = new StageId("xyz", 1); + final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("testChannel", false); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> workerClient.fetchChannelData(WORKER_ID, stageId, 2, 0, channel).get() + ); + + MatcherAssert.assertThat( + e.getCause(), + CoreMatchers.allOf( + CoreMatchers.instanceOf(IOException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.equalTo("Some error")) + ) + ); + + channel.close(); + } + + @Test + public void test_fetchChannelData_nonEmpty() throws Exception + { + workerServiceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/channels/xyz/1/2?offset=0") + .header(HttpHeaders.ACCEPT_ENCODING, "identity"), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_OCTET_STREAM), + FILE_BYTES + ).expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/channels/xyz/1/2?offset=" + FILE_BYTES.length) + .header(HttpHeaders.ACCEPT_ENCODING, "identity"), + HttpResponseStatus.OK, + fetchChannelDataResponseHeaders(true), + ByteArrays.EMPTY_ARRAY + ); + + // Perform the test. + final StageId stageId = new StageId("xyz", 1); + final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("testChannel", false); + final Future>> framesFuture = readChannelAsync(channel); + + Assert.assertFalse(workerClient.fetchChannelData(WORKER_ID, stageId, 2, 0, channel).get()); + Assert.assertTrue(workerClient.fetchChannelData(WORKER_ID, stageId, 2, FILE_BYTES.length, channel).get()); + channel.doneWriting(); // Caller is expected to call doneWriting after fetchChannelData returns true. + + FrameTestUtil.assertRowsEqual( + FrameTestUtil.readRowsFromCursorFactory(new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex())), + Sequences.simple(framesFuture.get()) + ); + } + + private Future>> readChannelAsync(final ReadableFrameChannel channel) + { + return exec.submit(() -> { + final List> retVal = new ArrayList<>(); + while (!channel.isFinished()) { + FutureUtils.getUnchecked(channel.readabilityFuture(), false); + + if (channel.canRead()) { + final Frame frame = channel.read(); + retVal.addAll(FrameTestUtil.readRowsFromCursorFactory(FRAME_READER.makeCursorFactory(frame)).toList()); + } + } + channel.close(); + return retVal; + }); + } + + /** + * Returns a frame file (as bytes) from a sequence of frames. + */ + private static byte[] toFileBytes(final Sequence frames) + { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final FrameFileWriter writer = + FrameFileWriter.open(Channels.newChannel(baos), null, ByteTracker.unboundedTracker()); + frames.forEach(frame -> { + try { + writer.writeFrame(frame, FrameFileWriter.NO_PARTITION); + } + catch (IOException e) { + throw new RuntimeException(e); + } + }); + CloseableUtils.closeAndWrapExceptions(writer); + return baos.toByteArray(); + } + + + /** + * Expected response headers for the "fetch channel data" API. + */ + private static Map fetchChannelDataResponseHeaders(final boolean lastResponse) + { + final ImmutableMap.Builder builder = + ImmutableMap.builder() + .put(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_OCTET_STREAM); + + if (lastResponse) { + builder.put( + FrameFileHttpResponseHandler.HEADER_LAST_FETCH_NAME, + FrameFileHttpResponseHandler.HEADER_LAST_FETCH_VALUE + ); + } + + return builder.build(); + } + + /** + * Worker client that communicates with a single worker named {@link #WORKER_ID}. + */ + private static class TestWorkerClient extends BaseWorkerClientImpl + { + private final ServiceClient workerServiceClient; + + public TestWorkerClient(ObjectMapper objectMapper, ServiceClient workerServiceClient) + { + super(objectMapper, MediaType.APPLICATION_JSON); + this.workerServiceClient = workerServiceClient; + } + + @Override + protected ServiceClient getClient(String workerId) + { + if (WORKER_ID.equals(workerId)) { + return workerServiceClient; + } else { + throw new ISE("Expected workerId[%s], got[%s]", WORKER_ID, workerId); + } + } + + @Override + public void close() + { + // Nothing to close. + } + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java index 927372a3a6a..d0410897a07 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java @@ -19,8 +19,10 @@ package org.apache.druid.msq.shuffle.output; +import com.google.common.collect.Iterables; import com.google.common.io.ByteStreams; import com.google.common.math.IntMath; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; @@ -30,10 +32,12 @@ import org.apache.druid.frame.file.FrameFile; 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.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.utils.CloseableUtils; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; import org.hamcrest.Matchers; @@ -50,206 +54,424 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.math.RoundingMode; +import java.nio.file.Files; import java.util.List; public class ChannelStageOutputReaderTest extends InitializedNullHandlingTest { - private static final int MAX_FRAMES = 10; - private static final int EXPECTED_NUM_ROWS = 1209; - - private final BlockingQueueFrameChannel channel = new BlockingQueueFrameChannel(MAX_FRAMES); - private final ChannelStageOutputReader reader = new ChannelStageOutputReader(channel.readable()); - - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); - - private FrameReader frameReader; - private List frameList; - - @Before - public void setUp() + /** + * Tests that use {@link BlockingQueueFrameChannel#minimal()}. + */ + public static class WithMinimalBuffering extends InitializedNullHandlingTest { - final IncrementalIndex index = TestIndex.getIncrementalTestIndex(); - final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - frameReader = FrameReader.create(adapter.getRowSignature()); - frameList = FrameSequenceBuilder.fromAdapter(adapter) - .frameType(FrameType.ROW_BASED) - .maxRowsPerFrame(IntMath.divide(index.size(), MAX_FRAMES, RoundingMode.CEILING)) - .frames() - .toList(); - } - - @After - public void tearDown() - { - reader.close(); - } - - @Test - public void test_readLocally() throws IOException - { - writeAllFramesToChannel(); - - Assert.assertSame(channel.readable(), reader.readLocally()); - reader.close(); // Won't close the channel, because it's already been returned by readLocally - - final int numRows = FrameTestUtil.readRowsFromFrameChannel(channel.readable(), frameReader).toList().size(); - Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); - } - - @Test - public void test_readLocally_closePriorToRead() throws IOException - { - writeAllFramesToChannel(); - - reader.close(); - - // Can't read the channel after closing the reader - Assert.assertThrows( - IllegalStateException.class, - reader::readLocally - ); - } - - @Test - public void test_readLocally_thenReadRemotely() throws IOException - { - writeAllFramesToChannel(); - - Assert.assertSame(channel.readable(), reader.readLocally()); - - // Can't read remotely after reading locally - Assert.assertThrows( - IllegalStateException.class, - () -> reader.readRemotelyFrom(0) + private final Frame frame = Iterables.getOnlyElement( + FrameSequenceBuilder + .fromCursorFactory(new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex())) + .frameType(FrameType.ROW_BASED) + .frames() + .toList() ); - // Can still read locally after this error - final int numRows = FrameTestUtil.readRowsFromFrameChannel(channel.readable(), frameReader).toList().size(); - Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); - } + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); - @Test - public void test_readRemotely_strideBasedOnReturnedChunk() throws IOException - { - // Test that reads entire chunks from readRemotelyFrom. This is a typical usage pattern. + private BlockingQueueFrameChannel channel; + private ChannelStageOutputReader channelReader; + private File tmpFile; + private OutputStream tmpOut; + private FrameFile tmpFrameFile; - writeAllFramesToChannel(); + // Variables used by doRead() + private long offset; + private ListenableFuture nextRead; - final File tmpFile = temporaryFolder.newFile(); - - try (final FileOutputStream tmpOut = new FileOutputStream(tmpFile)) { - int numReads = 0; - long offset = 0; - - while (true) { - try (final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(offset), true)) { - numReads++; - final long bytesWritten = ByteStreams.copy(in, tmpOut); - offset += bytesWritten; - - if (bytesWritten == 0) { - break; - } - } - } - - MatcherAssert.assertThat(numReads, Matchers.greaterThan(1)); + @Before + public void setUp() throws Exception + { + channel = BlockingQueueFrameChannel.minimal(); + channelReader = new ChannelStageOutputReader(channel.readable()); + tmpFile = temporaryFolder.newFile(); + tmpOut = Files.newOutputStream(tmpFile.toPath()); } - final FrameFile frameFile = FrameFile.open(tmpFile, null); - final int numRows = - FrameTestUtil.readRowsFromFrameChannel(new ReadableFileFrameChannel(frameFile), frameReader).toList().size(); - - Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); - } - - @Test - public void test_readRemotely_strideOneByte() throws IOException - { - // Test that reads one byte at a time from readRemotelyFrom. This helps ensure that there are no edge cases - // in the chunk-reading logic. - - writeAllFramesToChannel(); - - final File tmpFile = temporaryFolder.newFile(); - - try (final FileOutputStream tmpOut = new FileOutputStream(tmpFile)) { - int numReads = 0; - long offset = 0; - - while (true) { - try (final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(offset), true)) { - numReads++; - final int nextByte = in.read(); - - if (nextByte < 0) { - break; - } - - tmpOut.write(nextByte); - offset++; - } - } - - Assert.assertEquals(numReads, offset + 1); + @After + public void tearDown() throws Exception + { + CloseableUtils.closeAll(tmpOut, tmpFrameFile); } - final FrameFile frameFile = FrameFile.open(tmpFile, null); - final int numRows = - FrameTestUtil.readRowsFromFrameChannel(new ReadableFileFrameChannel(frameFile), frameReader).toList().size(); + @Test + public void test_remote_empty() throws Exception + { + // Close without writing anything. + channel.writable().close(); - Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); - } + while (doRead(-1)) { + // Do nothing, just keep reading. + } - @Test - public void test_readRemotely_thenLocally() throws IOException - { - writeAllFramesToChannel(); + Assert.assertEquals(0, tmpFrameFile.numFrames()); + } - // Read remotely - FutureUtils.getUnchecked(reader.readRemotelyFrom(0), true); - - // Then read locally - Assert.assertThrows( - IllegalStateException.class, - reader::readLocally - ); - } - - @Test - public void test_readRemotely_cannotReverse() throws IOException - { - writeAllFramesToChannel(); - - // Read remotely from offset = 1. - final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(1), true); - final int offset = ByteStreams.toByteArray(in).length; - MatcherAssert.assertThat(offset, Matchers.greaterThan(0)); - - // Then read again from offset = 0; should get an error. - final RuntimeException e = Assert.assertThrows( - RuntimeException.class, - () -> FutureUtils.getUnchecked(reader.readRemotelyFrom(0), true) - ); - - MatcherAssert.assertThat( - e, - ThrowableCauseMatcher.hasCause( - Matchers.allOf( - CoreMatchers.instanceOf(IllegalStateException.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Offset[0] no longer available")) - ) - ) - ); - } - - private void writeAllFramesToChannel() throws IOException - { - for (Frame frame : frameList) { + @Test + public void test_remote_oneFrame() throws Exception + { + // Close after writing one frame. channel.writable().write(frame); + channel.writable().close(); + + while (doRead(-1)) { + // Do nothing, just keep reading. + } + + Assert.assertEquals(1, tmpFrameFile.numFrames()); + Assert.assertEquals(frame.numBytes(), tmpFrameFile.frame(0).numBytes()); + } + + @Test + public void test_remote_oneFrame_writeAfterFirstRead() throws Exception + { + Assert.assertTrue(doRead(-1)); + + // Close after writing one frame. + channel.writable().write(frame); + channel.writable().close(); + + while (doRead(-1)) { + // Do nothing, just keep reading. + } + + Assert.assertEquals(1, tmpFrameFile.numFrames()); + Assert.assertEquals(frame.numBytes(), tmpFrameFile.frame(0).numBytes()); + } + + @Test + public void test_remote_oneFrame_readOneByteAtATime() throws Exception + { + // Close after writing one frame. + channel.writable().write(frame); + channel.writable().close(); + + while (doRead(1)) { + // Do nothing, just keep reading. + } + + Assert.assertEquals(1, tmpFrameFile.numFrames()); + Assert.assertEquals(frame.numBytes(), tmpFrameFile.frame(0).numBytes()); + } + + @Test + public void test_remote_threeFrames_readOneByteAtATime() throws Exception + { + // Write one frame. + channel.writable().write(frame); + + // See that we can't write another frame. + final IllegalStateException e = Assert.assertThrows( + IllegalStateException.class, + () -> channel.writable().write(frame) + ); + + MatcherAssert.assertThat( + e, + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Channel has no capacity")) + ); + + // Read the first frame until we start blocking. + while (nextRead == null) { + Assert.assertTrue(doRead(1)); + } + + // Write the next frame. + Assert.assertFalse(nextRead.isDone()); + channel.writable().write(frame); + + // This write would have unblocked nextRead, which will now be done. + Assert.assertTrue(nextRead.isDone()); + + // Write a third frame. + channel.writable().write(frame); + + // See that we can't write a fourth frame. + final IllegalStateException e2 = Assert.assertThrows( + IllegalStateException.class, + () -> channel.writable().write(frame) + ); + + MatcherAssert.assertThat( + e2, + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Channel has no capacity")) + ); + + // And read until we start blocking. + while (nextRead == null) { + Assert.assertTrue(doRead(1)); + } + + // Close. + channel.writable().close(); + + // Read until end of stream. + while (doRead(1)) { + // Just keep looping. + } + + Assert.assertEquals(3, tmpFrameFile.numFrames()); + Assert.assertEquals(frame.numBytes(), tmpFrameFile.frame(0).numBytes()); + Assert.assertEquals(frame.numBytes(), tmpFrameFile.frame(1).numBytes()); + Assert.assertEquals(frame.numBytes(), tmpFrameFile.frame(2).numBytes()); + } + + /** + * Do the next read operation. + * + * @return false if done reading, true if there's more to read + */ + private boolean doRead(final long limit) throws IOException + { + if (nextRead == null) { + nextRead = channelReader.readRemotelyFrom(offset); + } + + if (nextRead.isDone()) { + try (final InputStream in = FutureUtils.getUncheckedImmediately(nextRead)) { + nextRead = null; + long readSize = 0; + + if (limit == -1) { + // Unlimited + readSize = ByteStreams.copy(in, tmpOut); + } else { + // Limited + while (readSize < limit) { + final int r = in.read(); + if (r != -1) { + readSize++; + tmpOut.write(r); + } else { + break; + } + } + } + + offset += readSize; + + if (readSize == 0) { + channel.readable().close(); + tmpOut.close(); + tmpFrameFile = FrameFile.open(tmpFile, null); + return false; + } + } + } + + return true; + } + } + + /** + * Tests that use {@link BlockingQueueFrameChannel} that is fully buffered. + */ + public static class WithMaximalBuffering extends InitializedNullHandlingTest + { + private static final int MAX_FRAMES = 10; + private static final int EXPECTED_NUM_ROWS = 1209; + + private final BlockingQueueFrameChannel channel = new BlockingQueueFrameChannel(MAX_FRAMES); + private final ChannelStageOutputReader reader = new ChannelStageOutputReader(channel.readable()); + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private FrameReader frameReader; + private List frameList; + + @Before + public void setUp() + { + final IncrementalIndex index = TestIndex.getIncrementalTestIndex(); + final IncrementalIndexCursorFactory adapter = new IncrementalIndexCursorFactory(index); + frameReader = FrameReader.create(adapter.getRowSignature()); + frameList = FrameSequenceBuilder.fromCursorFactory(adapter) + .frameType(FrameType.ROW_BASED) + .maxRowsPerFrame(IntMath.divide(index.size(), MAX_FRAMES, RoundingMode.CEILING)) + .frames() + .toList(); + } + + @After + public void tearDown() + { + reader.close(); + } + + @Test + public void test_readLocally() throws IOException + { + writeAllFramesToChannel(); + + Assert.assertSame(channel.readable(), reader.readLocally()); + reader.close(); // Won't close the channel, because it's already been returned by readLocally + + final int numRows = FrameTestUtil.readRowsFromFrameChannel(channel.readable(), frameReader).toList().size(); + Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); + } + + @Test + public void test_readLocally_closePriorToRead() throws IOException + { + writeAllFramesToChannel(); + + reader.close(); + + // Can't read the channel after closing the reader + Assert.assertThrows( + IllegalStateException.class, + reader::readLocally + ); + } + + @Test + public void test_readLocally_thenReadRemotely() throws IOException + { + writeAllFramesToChannel(); + + Assert.assertSame(channel.readable(), reader.readLocally()); + + // Can't read remotely after reading locally + Assert.assertThrows( + IllegalStateException.class, + () -> reader.readRemotelyFrom(0) + ); + + // Can still read locally after this error + final int numRows = FrameTestUtil.readRowsFromFrameChannel(channel.readable(), frameReader).toList().size(); + Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); + } + + @Test + public void test_readRemotely_strideBasedOnReturnedChunk() throws IOException + { + // Test that reads entire chunks from readRemotelyFrom. This is a typical usage pattern. + + writeAllFramesToChannel(); + + final File tmpFile = temporaryFolder.newFile(); + + try (final FileOutputStream tmpOut = new FileOutputStream(tmpFile)) { + int numReads = 0; + long offset = 0; + + while (true) { + try (final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(offset), true)) { + numReads++; + final long bytesWritten = ByteStreams.copy(in, tmpOut); + offset += bytesWritten; + + if (bytesWritten == 0) { + break; + } + } + } + + MatcherAssert.assertThat(numReads, Matchers.greaterThan(1)); + } + + final FrameFile frameFile = FrameFile.open(tmpFile, null); + final int numRows = + FrameTestUtil.readRowsFromFrameChannel(new ReadableFileFrameChannel(frameFile), frameReader).toList().size(); + + Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); + } + + @Test + public void test_readRemotely_strideOneByte() throws IOException + { + // Test that reads one byte at a time from readRemotelyFrom. This helps ensure that there are no edge cases + // in the chunk-reading logic. + + writeAllFramesToChannel(); + + final File tmpFile = temporaryFolder.newFile(); + + try (final OutputStream tmpOut = Files.newOutputStream(tmpFile.toPath())) { + int numReads = 0; + long offset = 0; + + while (true) { + try (final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(offset), true)) { + numReads++; + final int nextByte = in.read(); + + if (nextByte < 0) { + break; + } + + tmpOut.write(nextByte); + offset++; + } + } + + Assert.assertEquals(numReads, offset + 1); + } + + final FrameFile frameFile = FrameFile.open(tmpFile, null); + final int numRows = + FrameTestUtil.readRowsFromFrameChannel(new ReadableFileFrameChannel(frameFile), frameReader).toList().size(); + + Assert.assertEquals(EXPECTED_NUM_ROWS, numRows); + } + + @Test + public void test_readRemotely_thenLocally() throws IOException + { + writeAllFramesToChannel(); + + // Read remotely + FutureUtils.getUnchecked(reader.readRemotelyFrom(0), true); + + // Then read locally + Assert.assertThrows( + IllegalStateException.class, + reader::readLocally + ); + } + + @Test + public void test_readRemotely_cannotReverse() throws IOException + { + writeAllFramesToChannel(); + + // Read remotely from offset = 1. + final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(1), true); + final int offset = ByteStreams.toByteArray(in).length; + MatcherAssert.assertThat(offset, Matchers.greaterThan(0)); + + // Then read again from offset = 0; should get an error. + final RuntimeException e = Assert.assertThrows( + RuntimeException.class, + () -> FutureUtils.getUnchecked(reader.readRemotelyFrom(0), true) + ); + + MatcherAssert.assertThat( + e, + ThrowableCauseMatcher.hasCause( + Matchers.allOf( + CoreMatchers.instanceOf(IllegalStateException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Offset[0] no longer available")) + ) + ) + ); + } + + private void writeAllFramesToChannel() throws IOException + { + for (Frame frame : frameList) { + channel.writable().write(frame); + } + channel.writable().close(); } - channel.writable().close(); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java index eaa2a9efe5a..7d65cea9872 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteMSQTestsHelper.java @@ -61,15 +61,14 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.TestGroupByBuffers; +import org.apache.druid.segment.CompleteSegment; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.loading.DataSegmentPusher; @@ -86,6 +85,7 @@ import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.LinearShardSpec; import org.easymock.EasyMock; import org.joda.time.Interval; import org.mockito.Mockito; @@ -161,11 +161,10 @@ public class CalciteMSQTestsHelper ) ); ObjectMapper testMapper = MSQTestBase.setupObjectMapper(dummyInjector); - IndexIO indexIO = new IndexIO(testMapper, ColumnConfig.DEFAULT); SegmentCacheManager segmentCacheManager = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, testMapper) .manufacturate(cacheManagerDir); LocalDataSegmentPusherConfig config = new LocalDataSegmentPusherConfig(); - MSQTestSegmentManager segmentManager = new MSQTestSegmentManager(segmentCacheManager, indexIO); + MSQTestSegmentManager segmentManager = new MSQTestSegmentManager(segmentCacheManager); config.storageDirectory = storageDir; binder.bind(DataSegmentPusher.class).toProvider(() -> new MSQTestDelegateDataSegmentPusher( new LocalDataSegmentPusher(config), @@ -206,7 +205,10 @@ public class CalciteMSQTestsHelper return mockFactory; } - private static Supplier> getSupplierForSegment(Function tempFolderProducer, SegmentId segmentId) + protected static Supplier> getSupplierForSegment( + Function tempFolderProducer, + SegmentId segmentId + ) { final QueryableIndex index; switch (segmentId.getDataSource()) { @@ -440,9 +442,9 @@ public class CalciteMSQTestsHelper } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return new QueryableIndexStorageAdapter(index); + return new QueryableIndexCursorFactory(index); } @Override @@ -450,6 +452,13 @@ public class CalciteMSQTestsHelper { } }; - return () -> new ReferenceCountingResourceHolder<>(segment, Closer.create()); + DataSegment dataSegment = DataSegment.builder() + .dataSource(segmentId.getDataSource()) + .interval(segmentId.getInterval()) + .version(segmentId.getVersion()) + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(); + return () -> new ReferenceCountingResourceHolder<>(new CompleteSegment(dataSegment, segment), Closer.create()); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index bb8691b9322..f0ef697cc38 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -92,12 +92,12 @@ import org.apache.druid.msq.guice.MSQExternalDataSourceModule; import org.apache.druid.msq.guice.MSQIndexingModule; import org.apache.druid.msq.guice.MSQSqlModule; import org.apache.druid.msq.guice.MultiStageQuery; -import org.apache.druid.msq.guice.SegmentGenerationTerminalStageSpecFactory; import org.apache.druid.msq.indexing.InputChannelFactory; import org.apache.druid.msq.indexing.MSQControllerTask; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQTuningConfig; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; +import org.apache.druid.msq.indexing.destination.SegmentGenerationTerminalStageSpecFactory; import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.indexing.error.InsertLockPreemptedFaultTest; import org.apache.druid.msq.indexing.error.MSQErrorReport; @@ -134,12 +134,13 @@ import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.TestGroupByBuffers; import org.apache.druid.rpc.ServiceClientFactory; +import org.apache.druid.segment.CompleteSegment; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; @@ -198,6 +199,7 @@ import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.PruneLoadSpec; import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.TombstoneShardSpec; @@ -336,6 +338,7 @@ public class MSQTestBase extends BaseCalciteQueryTest private TestGroupByBuffers groupByBuffers; protected final WorkerMemoryParameters workerMemoryParameters = Mockito.spy(makeTestWorkerMemoryParameters()); + protected static final String TEST_CONTROLLER_TASK_ID = "query-test-query"; protected static class MSQBaseComponentSupplier extends StandardComponentSupplier { @@ -380,6 +383,7 @@ public class MSQTestBase extends BaseCalciteQueryTest @AfterEach public void tearDown2() { + Mockito.framework().clearInlineMocks(); groupByBuffers.close(); } @@ -421,13 +425,25 @@ public class MSQTestBase extends BaseCalciteQueryTest MSQSqlModule sqlModule = new MSQSqlModule(); - segmentManager = new MSQTestSegmentManager(segmentCacheManager, indexIO); + segmentManager = new MSQTestSegmentManager(segmentCacheManager); BrokerClient brokerClient = mock(BrokerClient.class); List modules = ImmutableList.of( binder -> { DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() { + @Override + public int getNumThreads() + { + return 1; + } + + @Override + public int intermediateComputeSizeBytes() + { + return 10_000_000; + } + @Override public String getFormatString() { @@ -619,7 +635,10 @@ public class MSQTestBase extends BaseCalciteQueryTest } @Nonnull - private Supplier> getSupplierForSegment(Function tempFolderProducer, SegmentId segmentId) + protected Supplier> getSupplierForSegment( + Function tempFolderProducer, + SegmentId segmentId + ) { if (segmentManager.getSegment(segmentId) == null) { final QueryableIndex index; @@ -698,9 +717,9 @@ public class MSQTestBase extends BaseCalciteQueryTest } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return new QueryableIndexStorageAdapter(index); + return new QueryableIndexCursorFactory(index); } @Override @@ -710,7 +729,14 @@ public class MSQTestBase extends BaseCalciteQueryTest }; segmentManager.addSegment(segment); } - return () -> ReferenceCountingResourceHolder.fromCloseable(segmentManager.getSegment(segmentId)); + DataSegment dataSegment = DataSegment.builder() + .dataSource(segmentId.getDataSource()) + .interval(segmentId.getInterval()) + .version(segmentId.getVersion()) + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(); + return () -> ReferenceCountingResourceHolder.fromCloseable(new CompleteSegment(dataSegment, segmentManager.getSegment(segmentId))); } public SelectTester testSelectQuery() @@ -750,14 +776,13 @@ public class MSQTestBase extends BaseCalciteQueryTest public static WorkerMemoryParameters makeTestWorkerMemoryParameters() { - return WorkerMemoryParameters.createInstance( - WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50, - 2, - 10, + return new WorkerMemoryParameters( + 100_000_000, + WorkerMemoryParameters.DEFAULT_FRAME_SIZE, 1, - 2, - 1, - 0 + 50, + 10_000_000, + 10_000_000 ); } @@ -861,6 +886,7 @@ public class MSQTestBase extends BaseCalciteQueryTest public abstract class MSQTester> { protected String sql = null; + protected MSQControllerTask taskSpec = null; protected Map queryContext = DEFAULT_MSQ_CONTEXT; protected List expectedRowSignature = null; protected MSQSpec expectedMSQSpec = null; @@ -888,6 +914,12 @@ public class MSQTestBase extends BaseCalciteQueryTest return asBuilder(); } + public Builder setTaskSpec(MSQControllerTask taskSpec) + { + this.taskSpec = taskSpec; + return asBuilder(); + } + public Builder setQueryContext(Map queryContext) { this.queryContext = queryContext; @@ -1152,8 +1184,15 @@ public class MSQTestBase extends BaseCalciteQueryTest public void verifyResults() { - Preconditions.checkArgument(sql != null, "sql cannot be null"); - Preconditions.checkArgument(queryContext != null, "queryContext cannot be null"); + Preconditions.checkArgument( + sql != null || taskSpec != null, + "sql and taskSpec both cannot be null" + ); + Preconditions.checkArgument( + sql == null || taskSpec == null, + "sql and taskSpec both cannot be provided in the same test" + ); + Preconditions.checkArgument(sql == null || queryContext != null, "queryContext cannot be null"); Preconditions.checkArgument( (expectedResultRows != null && expectedResultRows.isEmpty()) || expectedDataSource != null, "dataSource cannot be null when expectedResultRows is non-empty" @@ -1169,7 +1208,15 @@ public class MSQTestBase extends BaseCalciteQueryTest Preconditions.checkArgument(expectedShardSpec != null, "shardSpecClass cannot be null"); readyToRun(); try { - String controllerId = runMultiStageQuery(sql, queryContext); + String controllerId; + if (sql != null) { + // Run the sql command. + controllerId = runMultiStageQuery(sql, queryContext); + } else { + // Run the task spec directly instead. + controllerId = TEST_CONTROLLER_TASK_ID; + indexingServiceClient.runTask(controllerId, taskSpec); + } if (expectedMSQFault != null || expectedMSQFaultClass != null) { MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId); if (expectedMSQFault != null) { @@ -1199,17 +1246,17 @@ public class MSQTestBase extends BaseCalciteQueryTest verifyLookupLoadingInfoInTaskContext(msqControllerTask.getContext()); log.info( "found generated segments: %s", - segmentManager.getAllDataSegments().stream().map(s -> s.toString()).collect( + segmentManager.getAllTestGeneratedDataSegments().stream().map(s -> s.toString()).collect( Collectors.joining("\n")) ); // check if segments are created if (!expectedResultRows.isEmpty()) { - Assert.assertNotEquals(0, segmentManager.getAllDataSegments().size()); + Assert.assertNotEquals(0, segmentManager.getAllTestGeneratedDataSegments().size()); } String foundDataSource = null; SortedMap>> segmentIdVsOutputRowsMap = new TreeMap<>(); - for (DataSegment dataSegment : segmentManager.getAllDataSegments()) { + for (DataSegment dataSegment : segmentManager.getAllTestGeneratedDataSegments()) { //Assert shard spec class Assert.assertEquals(expectedShardSpec, dataSegment.getShardSpec().getClass()); @@ -1223,12 +1270,11 @@ public class MSQTestBase extends BaseCalciteQueryTest dataSegment.getDataSource() ); } - final QueryableIndex queryableIndex = indexIO.loadIndex(segmentCacheManager.getSegmentFiles( - dataSegment)); - final StorageAdapter storageAdapter = new QueryableIndexStorageAdapter(queryableIndex); + final QueryableIndex queryableIndex = indexIO.loadIndex(segmentCacheManager.getSegmentFiles(dataSegment)); + final CursorFactory cursorFactory = new QueryableIndexCursorFactory(queryableIndex); // assert rowSignature - Assert.assertEquals(expectedRowSignature, resultSignatureFromRowSignature(storageAdapter.getRowSignature())); + Assert.assertEquals(expectedRowSignature, resultSignatureFromRowSignature(cursorFactory.getRowSignature())); // assert rollup Assert.assertEquals(expectedRollUp, queryableIndex.getMetadata().isRollup()); @@ -1242,7 +1288,7 @@ public class MSQTestBase extends BaseCalciteQueryTest queryableIndex.getMetadata().getAggregators() ); - for (List row : FrameTestUtil.readRowsFromAdapter(storageAdapter, null, false).toList()) { + for (List row : FrameTestUtil.readRowsFromCursorFactory(cursorFactory).toList()) { // transforming rows for sketch assertions List transformedRow = row.stream() .map(r -> { @@ -1362,18 +1408,32 @@ public class MSQTestBase extends BaseCalciteQueryTest assertResultsEquals(sql, expectedResultRows, transformedOutputRows); } catch (Exception e) { - throw new ISE(e, "Query %s failed", sql); + throw new ISE(e, "Query %s failed", sql != null ? sql : taskSpec); } } public void verifyExecutionError() { - Preconditions.checkArgument(sql != null, "sql cannot be null"); - Preconditions.checkArgument(queryContext != null, "queryContext cannot be null"); + Preconditions.checkArgument( + sql != null || taskSpec != null, + "sql and taskSpec both cannot be null" + ); + Preconditions.checkArgument( + sql == null || taskSpec == null, + "sql and taskSpec both cannot be provided in the same test" + ); + Preconditions.checkArgument(sql == null || queryContext != null, "queryContext cannot be null"); Preconditions.checkArgument(expectedExecutionErrorMatcher != null, "Execution error matcher cannot be null"); readyToRun(); try { - String controllerId = runMultiStageQuery(sql, queryContext); + String controllerId; + if (sql != null) { + controllerId = runMultiStageQuery(sql, queryContext); + } else { + // Run the task spec directly instead. + controllerId = TEST_CONTROLLER_TASK_ID; + indexingServiceClient.runTask(controllerId, taskSpec); + } getPayloadOrThrow(controllerId); Assert.fail(StringUtils.format("Query did not throw an exception (sql = [%s])", sql)); } @@ -1399,8 +1459,15 @@ public class MSQTestBase extends BaseCalciteQueryTest public Pair, List>> runQueryWithResult() { readyToRun(); - Preconditions.checkArgument(sql != null, "sql cannot be null"); - Preconditions.checkArgument(queryContext != null, "queryContext cannot be null"); + Preconditions.checkArgument( + sql != null || taskSpec != null, + "sql and taskSpec both cannot be null" + ); + Preconditions.checkArgument( + sql == null || taskSpec == null, + "sql and taskSpec both cannot be provided in the same test" + ); + Preconditions.checkArgument(sql == null || queryContext != null, "queryContext cannot be null"); try { String controllerId = runMultiStageQuery(sql, queryContext); @@ -1496,7 +1563,7 @@ public class MSQTestBase extends BaseCalciteQueryTest } catch (Exception e) { if (expectedExecutionErrorMatcher == null) { - throw new ISE(e, "Query %s failed", sql); + throw new ISE(e, "Query %s failed", sql != null ? sql : taskSpec); } assertThat(e, expectedExecutionErrorMatcher); return null; @@ -1517,7 +1584,7 @@ public class MSQTestBase extends BaseCalciteQueryTest } Assert.assertEquals(expectedRowSignature, specAndResults.rhs.lhs); - assertResultsEquals(sql, expectedResultRows, specAndResults.rhs.rhs); + assertResultsEquals(sql != null ? sql : taskSpec.toString(), expectedResultRows, specAndResults.rhs.rhs); assertMSQSpec(expectedMSQSpec, specAndResults.lhs); } @@ -1525,7 +1592,7 @@ public class MSQTestBase extends BaseCalciteQueryTest { Preconditions.checkArgument(expectedExecutionErrorMatcher != null, "Execution error matcher cannot be null"); if (runQueryWithResult() != null) { - throw new ISE("Query %s did not throw an exception", sql); + throw new ISE("Query %s did not throw an exception", sql != null ? sql : taskSpec); } } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java index 4c7ca61be02..3791be4f309 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerClient.java @@ -75,7 +75,7 @@ public class MSQTestControllerClient implements ControllerClient } @Override - public void postWorkerError(String workerId, MSQErrorReport errorWrapper) + public void postWorkerError(MSQErrorReport errorWrapper) { controller.workerError(errorWrapper); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java index e6510430203..a7ec6054b56 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestControllerContext.java @@ -54,14 +54,16 @@ import org.apache.druid.msq.exec.WorkerManager; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.indexing.IndexerControllerContext; +import org.apache.druid.msq.indexing.IndexerTableInputSpecSlicer; import org.apache.druid.msq.indexing.MSQSpec; import org.apache.druid.msq.indexing.MSQWorkerTask; import org.apache.druid.msq.indexing.MSQWorkerTaskLauncher; import org.apache.druid.msq.input.InputSpecSlicer; -import org.apache.druid.msq.input.table.TableInputSpecSlicer; -import org.apache.druid.msq.kernel.QueryDefinition; import org.apache.druid.msq.kernel.controller.ControllerQueryKernelConfig; +import org.apache.druid.msq.querykit.QueryKit; +import org.apache.druid.msq.querykit.QueryKitSpec; import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.query.Query; import org.apache.druid.query.QueryContext; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.DruidNode; @@ -269,11 +271,28 @@ public class MSQTestControllerContext implements ControllerContext }; @Override - public ControllerQueryKernelConfig queryKernelConfig(MSQSpec querySpec, QueryDefinition queryDef) + public ControllerQueryKernelConfig queryKernelConfig(String queryId, MSQSpec querySpec) { return IndexerControllerContext.makeQueryKernelConfig(querySpec, new ControllerMemoryParameters(100_000_000)); } + @Override + public QueryKitSpec makeQueryKitSpec( + final QueryKit> queryKit, + final String queryId, + final MSQSpec querySpec, + final ControllerQueryKernelConfig queryKernelConfig + ) + { + return new QueryKitSpec( + queryKit, + queryId, + querySpec.getTuningConfig().getMaxNumWorkers(), + querySpec.getTuningConfig().getMaxNumWorkers(), + 1 + ); + } + @Override public void emitMetric(String metric, Number value) { @@ -304,9 +323,9 @@ public class MSQTestControllerContext implements ControllerContext } @Override - public InputSpecSlicer newTableInputSpecSlicer() + public InputSpecSlicer newTableInputSpecSlicer(WorkerManager workerManager) { - return new TableInputSpecSlicer( + return new IndexerTableInputSpecSlicer( coordinatorClient, taskActionClient, MultiStageQueryContext.getSegmentSources(queryContext) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestDelegateDataSegmentPusher.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestDelegateDataSegmentPusher.java index 73fca53682c..5d22c08e602 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestDelegateDataSegmentPusher.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestDelegateDataSegmentPusher.java @@ -60,7 +60,7 @@ public class MSQTestDelegateDataSegmentPusher implements DataSegmentPusher public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException { final DataSegment dataSegment = delegate.push(file, segment, useUniquePath); - segmentManager.addDataSegment(dataSegment); + segmentManager.addTestGeneratedDataSegment(dataSegment); return dataSegment; } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestSegmentManager.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestSegmentManager.java index 6151cb37cc2..2c0fdac3a69 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestSegmentManager.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestSegmentManager.java @@ -20,7 +20,6 @@ package org.apache.druid.msq.test; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.Segment; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.loading.SegmentLoadingException; @@ -37,24 +36,26 @@ import java.util.concurrent.ConcurrentMap; */ public class MSQTestSegmentManager { - private final ConcurrentMap dataSegments = new ConcurrentHashMap<>(); + private final ConcurrentMap testGeneratedDataSegments = new ConcurrentHashMap<>(); private final ConcurrentMap segments = new ConcurrentHashMap<>(); private final SegmentCacheManager segmentCacheManager; - private final IndexIO indexIO; final Object lock = new Object(); - public MSQTestSegmentManager(SegmentCacheManager segmentCacheManager, IndexIO indexIO) + public MSQTestSegmentManager(SegmentCacheManager segmentCacheManager) { this.segmentCacheManager = segmentCacheManager; - this.indexIO = indexIO; } - public void addDataSegment(DataSegment dataSegment) + /** + * Registers a data segment which was generated during the test run (as opposed to during setup). This is used to + * validate which segments are generated by the test. + */ + public void addTestGeneratedDataSegment(DataSegment dataSegment) { synchronized (lock) { - dataSegments.put(dataSegment.getId(), dataSegment); + testGeneratedDataSegments.put(dataSegment.getId(), dataSegment); try { segmentCacheManager.getSegmentFiles(dataSegment); @@ -65,9 +66,9 @@ public class MSQTestSegmentManager } } - public Collection getAllDataSegments() + public Collection getAllTestGeneratedDataSegments() { - return dataSegments.values(); + return testGeneratedDataSegments.values(); } public void addSegment(Segment segment) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java index 082429a9d7b..0902e978641 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestWorkerContext.java @@ -21,20 +21,21 @@ package org.apache.druid.msq.test; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Injector; +import org.apache.druid.collections.StupidPool; import org.apache.druid.frame.processor.Bouncer; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.msq.exec.Controller; import org.apache.druid.msq.exec.ControllerClient; import org.apache.druid.msq.exec.DataServerQueryHandlerFactory; -import org.apache.druid.msq.exec.OutputChannelMode; +import org.apache.druid.msq.exec.ProcessingBuffers; import org.apache.druid.msq.exec.Worker; import org.apache.druid.msq.exec.WorkerClient; import org.apache.druid.msq.exec.WorkerContext; import org.apache.druid.msq.exec.WorkerMemoryParameters; import org.apache.druid.msq.exec.WorkerStorageParameters; import org.apache.druid.msq.kernel.FrameContext; -import org.apache.druid.msq.kernel.QueryDefinition; +import org.apache.druid.msq.kernel.WorkOrder; import org.apache.druid.msq.querykit.DataSegmentProvider; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.segment.IndexIO; @@ -48,6 +49,7 @@ import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFacto import org.apache.druid.server.DruidNode; import java.io.File; +import java.nio.ByteBuffer; import java.util.Map; public class MSQTestWorkerContext implements WorkerContext @@ -58,7 +60,6 @@ public class MSQTestWorkerContext implements WorkerContext private final Injector injector; private final Map inMemoryWorkers; private final File file = FileUtils.createTempDir(); - private final Bouncer bouncer = new Bouncer(1); private final WorkerMemoryParameters workerMemoryParameters; private final WorkerStorageParameters workerStorageParameters; @@ -130,9 +131,9 @@ public class MSQTestWorkerContext implements WorkerContext } @Override - public FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode) + public FrameContext frameContext(WorkOrder workOrder) { - return new FrameContextImpl(new File(tempDir(), queryDef.getStageDefinition(stageNumber).getId().toString())); + return new FrameContextImpl(new File(tempDir(), workOrder.getStageDefinition().getId().toString())); } @Override @@ -159,6 +160,12 @@ public class MSQTestWorkerContext implements WorkerContext return injector.getInstance(DataServerQueryHandlerFactory.class); } + @Override + public boolean includeAllCounters() + { + return true; + } + class FrameContextImpl implements FrameContext { private final File tempDir; @@ -240,9 +247,12 @@ public class MSQTestWorkerContext implements WorkerContext } @Override - public Bouncer processorBouncer() + public ProcessingBuffers processingBuffers() { - return bouncer; + return new ProcessingBuffers( + new StupidPool<>("testProcessing", () -> ByteBuffer.allocate(1_000_000)), + new Bouncer(1) + ); } @Override diff --git a/extensions-core/mysql-metadata-storage/pom.xml b/extensions-core/mysql-metadata-storage/pom.xml index 7c2bad187f5..57f5ab4f12a 100644 --- a/extensions-core/mysql-metadata-storage/pom.xml +++ b/extensions-core/mysql-metadata-storage/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/orc-extensions/pom.xml b/extensions-core/orc-extensions/pom.xml index 13981c6675b..ae67d8f4dd4 100644 --- a/extensions-core/orc-extensions/pom.xml +++ b/extensions-core/orc-extensions/pom.xml @@ -26,7 +26,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-core/parquet-extensions/pom.xml b/extensions-core/parquet-extensions/pom.xml index 3a658ce3c57..e4abb30fe9e 100644 --- a/extensions-core/parquet-extensions/pom.xml +++ b/extensions-core/parquet-extensions/pom.xml @@ -27,7 +27,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-core/postgresql-metadata-storage/pom.xml b/extensions-core/postgresql-metadata-storage/pom.xml index 27c432b8c51..9c4159e36e8 100644 --- a/extensions-core/postgresql-metadata-storage/pom.xml +++ b/extensions-core/postgresql-metadata-storage/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/protobuf-extensions/pom.xml b/extensions-core/protobuf-extensions/pom.xml index aaf525070be..2c7c82cd34a 100644 --- a/extensions-core/protobuf-extensions/pom.xml +++ b/extensions-core/protobuf-extensions/pom.xml @@ -30,7 +30,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/s3-extensions/pom.xml b/extensions-core/s3-extensions/pom.xml index 8a9b75a049d..26ce68e4931 100644 --- a/extensions-core/s3-extensions/pom.xml +++ b/extensions-core/s3-extensions/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java index 986627f80bf..bbb1b294abf 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java @@ -1015,7 +1015,7 @@ public class S3InputSourceTest extends InitializedNullHandlingTest InputSourceReader reader = inputSource.reader( someSchema, - new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0), + new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null), temporaryFolder.newFolder() ); @@ -1063,7 +1063,7 @@ public class S3InputSourceTest extends InitializedNullHandlingTest InputSourceReader reader = inputSource.reader( someSchema, - new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0), + new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null), temporaryFolder.newFolder() ); try (CloseableIterator readerIterator = reader.read()) { @@ -1111,7 +1111,7 @@ public class S3InputSourceTest extends InitializedNullHandlingTest InputSourceReader reader = inputSource.reader( someSchema, - new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0), + new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null), temporaryFolder.newFolder() ); diff --git a/extensions-core/simple-client-sslcontext/pom.xml b/extensions-core/simple-client-sslcontext/pom.xml index 3a5bb913fb4..c75b75c90f5 100644 --- a/extensions-core/simple-client-sslcontext/pom.xml +++ b/extensions-core/simple-client-sslcontext/pom.xml @@ -22,7 +22,7 @@ druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml 4.0.0 diff --git a/extensions-core/stats/pom.xml b/extensions-core/stats/pom.xml index 048ad964bf8..4cfa00ad491 100644 --- a/extensions-core/stats/pom.xml +++ b/extensions-core/stats/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/extensions-core/testing-tools/pom.xml b/extensions-core/testing-tools/pom.xml index fd0eed880c2..902fc9655ad 100644 --- a/extensions-core/testing-tools/pom.xml +++ b/extensions-core/testing-tools/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index edadd6ce9a8..a0f0beac97a 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java index d1ceccc96a3..5ef56583f7c 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/hadoop/DatasourceRecordReader.java @@ -47,11 +47,11 @@ import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.realtime.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.transform.Transformer; import org.apache.hadoop.fs.Path; @@ -98,12 +98,12 @@ public class DatasourceRecordReader extends RecordReader spec = DatasourceInputFormat.getIngestionSpec(context.getConfiguration(), dataSource); logger.info("load schema [%s]", spec); - List adapters = Lists.transform( + List adapters = Lists.transform( segments, - new Function() + new Function() { @Override - public WindowedStorageAdapter apply(WindowedDataSegment segment) + public WindowedCursorFactory apply(WindowedDataSegment segment) { try { logger.info("Getting storage path for segment [%s]", segment.getSegment().getId()); @@ -122,10 +122,7 @@ public class DatasourceRecordReader extends RecordReader indexes.add(index); numRows += index.getNumRows(); - return new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(index), - segment.getInterval() - ); + return new WindowedCursorFactory(new QueryableIndexCursorFactory(index), segment.getInterval()); } catch (IOException ex) { throw new RuntimeException(ex); @@ -196,7 +193,7 @@ public class DatasourceRecordReader extends RecordReader private Yielder rowYielder; public SegmentReader( - final List adapters, + final List cursorFactories, final TransformSpec transformSpec, final List dims, final List metrics, @@ -207,18 +204,18 @@ public class DatasourceRecordReader extends RecordReader Sequence rows = Sequences.concat( Iterables.transform( - adapters, - new Function>() + cursorFactories, + new Function>() { @Nullable @Override - public Sequence apply(WindowedStorageAdapter adapter) + public Sequence apply(WindowedCursorFactory windowed) { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(Filters.toFilter(dimFilter)) - .setInterval(adapter.getInterval()) + .setInterval(windowed.getInterval()) .build(); - final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(buildSpec); + final CursorHolder cursorHolder = windowed.getCursorFactory().makeCursorHolder(buildSpec); final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return Sequences.empty(); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java index 6516b0a0e00..41dae9c9e65 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java @@ -44,12 +44,11 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.LocalDataSegmentPuller; -import org.apache.druid.segment.realtime.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; @@ -401,10 +400,9 @@ public class BatchDeltaIngestionTest new LocalDataSegmentPuller().getSegmentFiles(indexZip, tmpUnzippedSegmentDir); QueryableIndex index = INDEX_IO.loadIndex(tmpUnzippedSegmentDir); - StorageAdapter adapter = new QueryableIndexStorageAdapter(index); DatasourceRecordReader.SegmentReader segmentReader = new DatasourceRecordReader.SegmentReader( - ImmutableList.of(new WindowedStorageAdapter(adapter, windowedDataSegment.getInterval())), + ImmutableList.of(new WindowedCursorFactory(new QueryableIndexCursorFactory(index), windowedDataSegment.getInterval())), TransformSpec.NONE, expectedDimensions, expectedMetrics, @@ -435,30 +433,33 @@ public class BatchDeltaIngestionTest { HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - "website", - MAPPER.convertValue( - new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))), - null, - ImmutableList.of("timestamp", "host", "host2", "visited_num"), - false, - 0 - ), - null - ), - Map.class - ), - aggregators != null ? aggregators : new AggregatorFactory[]{ - new LongSumAggregatorFactory("visited_sum", "visited_num"), - new HyperUniquesAggregatorFactory("unique_hosts", "host2") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(INTERVAL_FULL)), - null, - MAPPER - ), + DataSchema.builder() + .withDataSource("website") + .withParserMap(MAPPER.convertValue( + new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec("timestamp", "yyyyMMddHH", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))), + null, + ImmutableList.of("timestamp", "host", "host2", "visited_num"), + false, + 0 + ), + null + ), + Map.class + )) + .withAggregators(aggregators != null ? aggregators : new AggregatorFactory[]{ + new LongSumAggregatorFactory("visited_sum", "visited_num"), + new HyperUniquesAggregatorFactory("unique_hosts", "host2") + }) + .withGranularity(new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(INTERVAL_FULL) + )) + .withObjectMapper(MAPPER) + .build(), new HadoopIOConfig( inputSpec, null, diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java index 24a8ee0ef7e..dd22a95083c 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java @@ -32,7 +32,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -158,46 +157,45 @@ public class DetermineHashedPartitionsJobTest } HadoopIngestionSpec ingestionSpec = new HadoopIngestionSpec( - new DataSchema( - "test_schema", - HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( - new StringInputRowParser( - new DelimitedParseSpec( - new TimestampSpec("ts", null, null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of( - "market", - "quality", - "placement", - "placementish" - )) - ), - "\t", - null, - Arrays.asList( - "ts", - "market", - "quality", - "placement", - "placementish", - "index" - ), - false, - 0 - ), - null - ), - Map.class - ), - new AggregatorFactory[]{new DoubleSumAggregatorFactory("index", "index")}, - new UniformGranularitySpec( - segmentGranularity, - Granularities.NONE, - intervals - ), - null, - HadoopDruidIndexerConfig.JSON_MAPPER - ), + DataSchema.builder() + .withDataSource("test_schema") + .withParserMap(HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( + new StringInputRowParser( + new DelimitedParseSpec( + new TimestampSpec("ts", null, null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of( + "market", + "quality", + "placement", + "placementish" + )) + ), + "\t", + null, + Arrays.asList( + "ts", + "market", + "quality", + "placement", + "placementish", + "index" + ), + false, + 0 + ), + null + ), + Map.class + )) + .withAggregators(new DoubleSumAggregatorFactory("index", "index")) + .withGranularity(new UniformGranularitySpec( + segmentGranularity, + Granularities.NONE, + intervals + )) + .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER) + .build(), new HadoopIOConfig( ImmutableMap.of( "paths", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java index a3c98f29565..bfd28d2cfca 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java @@ -29,7 +29,6 @@ import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -280,33 +279,36 @@ public class DeterminePartitionsJobTest config = new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - "website", - HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( - new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country")) - ), - null, - ImmutableList.of("timestamp", "host", "country", "visited_num"), - false, - 0 - ), - null - ), - Map.class - ), - new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.NONE, - ImmutableList.of(Intervals.of(interval)) - ), - null, - HadoopDruidIndexerConfig.JSON_MAPPER - ), + DataSchema.builder() + .withDataSource("website") + .withParserMap( + HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( + new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec("timestamp", "yyyyMMddHH", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country")) + ), + null, + ImmutableList.of("timestamp", "host", "country", "visited_num"), + false, + 0 + ), + null + ), + Map.class + ) + ) + .withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(Intervals.of(interval)) + ) + ) + .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER) + .build(), new HadoopIOConfig( ImmutableMap.of( "paths", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineRangePartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineRangePartitionsJobTest.java index e79d066ab55..3ff525c8b43 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineRangePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineRangePartitionsJobTest.java @@ -29,7 +29,6 @@ import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -328,35 +327,36 @@ public class DetermineRangePartitionsJobTest config = new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - "website", - null, - null, - new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.NONE, - ImmutableList.of(Intervals.of(interval)) - ), - null, - HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( - new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country")) - ), - null, - ImmutableList.of("timestamp", "host", "country", "visited_num"), - false, - 0 - ), - null - ), - Map.class - ), - HadoopDruidIndexerConfig.JSON_MAPPER - ), + DataSchema.builder() + .withDataSource("website") + .withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(Intervals.of(interval)) + ) + ) + .withParserMap( + HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( + new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec("timestamp", "yyyyMMddHH", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country")) + ), + null, + ImmutableList.of("timestamp", "host", "country", "visited_num"), + false, + 0 + ), + null + ), + Map.class + ) + ) + .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER) + .build(), new HadoopIOConfig( ImmutableMap.of( "paths", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java index 8aead05d625..ed47d180b43 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -37,7 +37,6 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; @@ -217,18 +216,18 @@ public class HadoopDruidIndexerConfigTest private static class HadoopIngestionSpecBuilder { - private static final DataSchema DATA_SCHEMA = new DataSchema( - "foo", - null, - new AggregatorFactory[0], - new UniformGranularitySpec( - Granularities.MINUTE, - Granularities.MINUTE, - ImmutableList.of(Intervals.of("2010-01-01/P1D")) - ), - null, - HadoopDruidIndexerConfigTest.JSON_MAPPER - ); + private static final DataSchema DATA_SCHEMA = + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.MINUTE, + Granularities.MINUTE, + ImmutableList.of(Intervals.of("2010-01-01/P1D")) + ) + ) + .withObjectMapper(HadoopDruidIndexerConfigTest.JSON_MAPPER) + .build(); private static final HadoopIOConfig HADOOP_IO_CONFIG = new HadoopIOConfig( ImmutableMap.of("paths", "bar", "type", "static"), diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java index db20ed8a184..da57b8ccf4a 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java @@ -31,7 +31,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.TestHelper; @@ -58,27 +57,29 @@ import java.util.stream.Collectors; public class HadoopDruidIndexerMapperTest { private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); - private static final DataSchema DATA_SCHEMA = new DataSchema( - "test_ds", - JSON_MAPPER.convertValue( - new HadoopyStringInputRowParser( - new JSONParseSpec( - new TimestampSpec("t", "auto", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")) - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of(), - null - ) - ), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - JSON_MAPPER - ); + private static final DataSchema DATA_SCHEMA = + DataSchema.builder() + .withDataSource("test_ds") + .withParserMap( + JSON_MAPPER.convertValue( + new HadoopyStringInputRowParser( + new JSONParseSpec( + new TimestampSpec("t", "auto", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")) + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of(), + null + ) + ), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ) + ) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .withObjectMapper(JSON_MAPPER) + .build(); private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig( JSON_MAPPER.convertValue( diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java index 6402721e73c..afcfb402359 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java @@ -34,7 +34,6 @@ import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.DataSegment; @@ -274,18 +273,17 @@ public class HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest throws Exception { HadoopIngestionSpec spec = new HadoopIngestionSpec( - new DataSchema( - "foo", - null, - new AggregatorFactory[0], - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P1D")) - ), - null, - jsonMapper - ), + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P1D")) + ) + ) + .withObjectMapper(jsonMapper) + .build(), new HadoopIOConfig( jsonMapper.convertValue(datasourcePathSpec, Map.class), null, diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java index 385c28ff0fb..37cdbb7300d 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java @@ -64,30 +64,33 @@ public class IndexGeneratorCombinerTest { HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - "website", - HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( - new StringInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "keywords"))) - ), - null - ), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("visited_sum", "visited"), - new HyperUniquesAggregatorFactory("unique_hosts", "host") - }, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.NONE, - ImmutableList.of(Intervals.of("2010/2011")) - ), - null, - HadoopDruidIndexerConfig.JSON_MAPPER - ), + DataSchema.builder() + .withDataSource("website") + .withParserMap( + HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( + new StringInputRowParser( + new TimeAndDimsParseSpec( + new TimestampSpec("timestamp", "yyyyMMddHH", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "keywords"))) + ), + null + ), + Map.class + ) + ) + .withAggregators( + new LongSumAggregatorFactory("visited_sum", "visited"), + new HyperUniquesAggregatorFactory("unique_hosts", "host") + ) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(Intervals.of("2010/2011")) + ) + ) + .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER) + .build(), new HadoopIOConfig( ImmutableMap.of( "paths", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java index e14ade454f4..241746ca58d 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java @@ -506,17 +506,19 @@ public class IndexGeneratorJobTest config = new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - datasourceName, - mapper.convertValue( - inputRowParser, - Map.class - ), - aggs, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)), - null, - mapper - ), + DataSchema.builder() + .withDataSource(datasourceName) + .withParserMap(mapper.convertValue(inputRowParser, Map.class)) + .withAggregators(aggs) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(interval) + ) + ) + .withObjectMapper(mapper) + .build(), new HadoopIOConfig( ImmutableMap.copyOf(inputSpec), null, diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java index 7069e9a78de..530c0f657a5 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java @@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.TestHelper; @@ -68,27 +67,30 @@ import java.util.Map; public class JobHelperTest { private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); - private static final DataSchema DATA_SCHEMA = new DataSchema( - "test_ds", - JSON_MAPPER.convertValue( - new HadoopyStringInputRowParser( - new JSONParseSpec( - new TimestampSpec("t", "auto", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")) - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of(), - null - ) - ), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - JSON_MAPPER - ); + + private static final DataSchema DATA_SCHEMA = + DataSchema.builder() + .withDataSource("test_ds") + .withParserMap( + JSON_MAPPER.convertValue( + new HadoopyStringInputRowParser( + new JSONParseSpec( + new TimestampSpec("t", "auto", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")) + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of(), + null + ) + ), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ) + ) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .withObjectMapper(JSON_MAPPER) + .build(); private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig( JSON_MAPPER.convertValue( @@ -123,27 +125,34 @@ public class JobHelperTest dataFile = temporaryFolder.newFile(); config = new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - "website", - HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( - new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))), - null, - ImmutableList.of("timestamp", "host", "visited_num"), - false, - 0 - ), - null - ), - Map.class - ), - new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")}, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)), - null, - HadoopDruidIndexerConfig.JSON_MAPPER - ), + DataSchema.builder() + .withDataSource("website") + .withParserMap( + HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( + new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec("timestamp", "yyyyMMddHH", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))), + null, + ImmutableList.of("timestamp", "host", "visited_num"), + false, + 0 + ), + null + ), + Map.class + ) + ) + .withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(this.interval) + ) + ) + .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER) + .build(), new HadoopIOConfig( ImmutableMap.of( "paths", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java index 042aa5e9550..e7f1402606f 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java @@ -36,22 +36,22 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.query.filter.SpatialDimFilter; +import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; -import org.apache.druid.segment.realtime.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; +import org.apache.druid.timeline.SegmentId; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -61,8 +61,10 @@ import org.junit.runners.Parameterized; import java.io.File; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Collection; import java.util.List; +import java.util.stream.Collectors; /** */ @@ -134,10 +136,12 @@ public class DatasourceRecordReaderSegmentReaderTest .setMaxRowCount(5000) .build() ) { - final StorageAdapter sa = new QueryableIndexStorageAdapter(qi); - final WindowedStorageAdapter wsa = new WindowedStorageAdapter(sa, sa.getInterval()); + final WindowedCursorFactory ws = new WindowedCursorFactory( + new QueryableIndexCursorFactory(qi), + qi.getDataInterval() + ); final DatasourceRecordReader.SegmentReader segmentReader = new DatasourceRecordReader.SegmentReader( - ImmutableList.of(wsa, wsa), + ImmutableList.of(ws, ws), TransformSpec.NONE, ImmutableList.of("host", "spatial"), ImmutableList.of("visited_sum", "unique_hosts"), @@ -162,15 +166,19 @@ public class DatasourceRecordReaderSegmentReaderTest // Check the index Assert.assertEquals(9, index.size()); - final IncrementalIndexStorageAdapter queryable = new IncrementalIndexStorageAdapter(index); - Assert.assertEquals(2, queryable.getAvailableDimensions().size()); - Assert.assertEquals("host", queryable.getAvailableDimensions().get(0)); - Assert.assertEquals("spatial", queryable.getAvailableDimensions().get(1)); - Assert.assertEquals(ImmutableList.of("visited_sum", "unique_hosts"), queryable.getAvailableMetrics()); + final IncrementalIndexSegment queryable = new IncrementalIndexSegment(index, SegmentId.dummy("test")); + final List dimensions = index.getDimensionNames(false); + Assert.assertEquals(2, dimensions.size()); + Assert.assertEquals("host", dimensions.get(0)); + Assert.assertEquals("spatial", dimensions.get(1)); + Assert.assertEquals( + ImmutableList.of("visited_sum", "unique_hosts"), + Arrays.stream(index.getMetricAggs()).map(AggregatorFactory::getName).collect(Collectors.toList()) + ); // Do a spatial filter final DatasourceRecordReader.SegmentReader segmentReader2 = new DatasourceRecordReader.SegmentReader( - ImmutableList.of(new WindowedStorageAdapter(queryable, Intervals.of("2000/3000"))), + ImmutableList.of(new WindowedCursorFactory(queryable.asCursorFactory(), Intervals.of("2000/3000"))), TransformSpec.NONE, ImmutableList.of("host", "spatial"), ImmutableList.of("visited_sum", "unique_hosts"), diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java index e8caea0256e..75a4fe45eee 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java @@ -44,7 +44,6 @@ import org.apache.druid.initialization.Initialization; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -308,33 +307,34 @@ public class DatasourcePathSpecTest { return new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - ingestionSpec1.getDataSource(), - HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( - new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), - DimensionsSpec.EMPTY, - null, - ImmutableList.of("timestamp", "host", "visited"), - false, - 0 - ), - null - ), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("visited_sum", "visited") - }, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.NONE, - ImmutableList.of(Intervals.of("2000/3000")) - ), - null, - HadoopDruidIndexerConfig.JSON_MAPPER - ), + DataSchema.builder() + .withDataSource(ingestionSpec1.getDataSource()) + .withParserMap( + HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( + new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec("timestamp", "yyyyMMddHH", null), + DimensionsSpec.EMPTY, + null, + ImmutableList.of("timestamp", "host", "visited"), + false, + 0 + ), + null + ), + Map.class + ) + ) + .withAggregators(new LongSumAggregatorFactory("visited_sum", "visited")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(Intervals.of("2000/3000")) + ) + ) + .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER) + .build(), new HadoopIOConfig( ImmutableMap.of( "paths", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java index 8af77ca0e4f..92bd8595560 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java @@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.hadoop.mapreduce.Job; @@ -152,18 +151,17 @@ public class GranularityPathSpecTest { UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"})); HadoopIngestionSpec spec = new HadoopIngestionSpec( - new DataSchema( - "foo", - null, - new AggregatorFactory[0], - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - ImmutableList.of(Intervals.of("2015-11-06T00:00Z/2015-11-07T00:00Z")) - ), - null, - jsonMapper - ), + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + ImmutableList.of(Intervals.of("2015-11-06T00:00Z/2015-11-07T00:00Z")) + ) + ) + .withObjectMapper(jsonMapper) + .build(), new HadoopIOConfig(null, null, null), DEFAULT_TUNING_CONFIG ); @@ -204,18 +202,17 @@ public class GranularityPathSpecTest { UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"})); HadoopIngestionSpec spec = new HadoopIngestionSpec( - new DataSchema( - "foo", - null, - new AggregatorFactory[0], - new UniformGranularitySpec( - Granularities.DAY, - Granularities.ALL, - ImmutableList.of(Intervals.of("2015-01-01T11Z/2015-01-02T05Z")) - ), - null, - jsonMapper - ), + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.ALL, + ImmutableList.of(Intervals.of("2015-01-01T11Z/2015-01-02T05Z")) + ) + ) + .withObjectMapper(jsonMapper) + .build(), new HadoopIOConfig(null, null, null), DEFAULT_TUNING_CONFIG ); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/StaticPathSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/StaticPathSpecTest.java index 06a1416ad83..fa1b2f59f48 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/StaticPathSpecTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/StaticPathSpecTest.java @@ -24,7 +24,6 @@ import org.apache.druid.indexer.HadoopDruidIndexerConfig; import org.apache.druid.indexer.HadoopIOConfig; import org.apache.druid.indexer.HadoopIngestionSpec; import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.input.MultipleInputs; @@ -54,7 +53,7 @@ public class StaticPathSpecTest Job job = new Job(); StaticPathSpec pathSpec = new StaticPathSpec("/a/c,/a/b/{c,d}", null); - DataSchema schema = new DataSchema("ds", null, new AggregatorFactory[0], null, null, jsonMapper); + DataSchema schema = DataSchema.builder().withDataSource("ds").withObjectMapper(jsonMapper).build(); HadoopIOConfig io = new HadoopIOConfig(null, null, null); pathSpec.addInputPaths(new HadoopDruidIndexerConfig(new HadoopIngestionSpec(schema, io, null)), job); diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 29f1a238471..a9479d7db08 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT @@ -189,10 +189,6 @@ datasketches-memory provided - - net.thisptr - jackson-jq - org.codehaus.jackson jackson-core-asl @@ -203,10 +199,6 @@ commons-collections4 provided - - org.eclipse.aether - aether-api - junit @@ -267,6 +259,11 @@ mockito-core test + + org.apache.maven.resolver + maven-resolver-api + 1.3.1 + diff --git a/indexing-service/src/main/java/org/apache/druid/guice/PeonProcessingModule.java b/indexing-service/src/main/java/org/apache/druid/guice/PeonProcessingModule.java new file mode 100644 index 00000000000..8961da7a555 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/guice/PeonProcessingModule.java @@ -0,0 +1,143 @@ +/* + * 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.guice; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulator; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.collections.BlockingPool; +import org.apache.druid.collections.DummyBlockingPool; +import org.apache.druid.collections.DummyNonBlockingPool; +import org.apache.druid.collections.NonBlockingPool; +import org.apache.druid.guice.annotations.Global; +import org.apache.druid.guice.annotations.Merging; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.ExecutorServiceMonitor; +import org.apache.druid.query.NoopQueryProcessingPool; +import org.apache.druid.query.QueryProcessingPool; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; + +import java.nio.ByteBuffer; + +/** + * This module fulfills the dependency injection of query processing and caching resources: buffer pools and + * thread pools on Peon selectively. Only the peons for the tasks supporting queries need to allocate direct buffers + * and thread pools. Thus, this is separate from the {@link DruidProcessingModule} to separate the needs of the peons and + * the historicals + * + * @see DruidProcessingModule + */ +public class PeonProcessingModule implements Module +{ + private static final Logger log = new Logger(PeonProcessingModule.class); + + @Override + public void configure(Binder binder) + { + DruidProcessingModule.registerConfigsAndMonitor(binder); + } + + @Provides + @LazySingleton + public CachePopulator getCachePopulator( + @Smile ObjectMapper smileMapper, + CachePopulatorStats cachePopulatorStats, + CacheConfig cacheConfig + ) + { + return DruidProcessingModule.createCachePopulator(smileMapper, cachePopulatorStats, cacheConfig); + } + + @Provides + @ManageLifecycle + public QueryProcessingPool getProcessingExecutorPool( + Task task, + DruidProcessingConfig config, + ExecutorServiceMonitor executorServiceMonitor, + Lifecycle lifecycle + ) + { + if (task.supportsQueries()) { + return DruidProcessingModule.createProcessingExecutorPool(config, executorServiceMonitor, lifecycle); + } else { + if (config.isNumThreadsConfigured()) { + log.warn( + "Ignoring the configured numThreads[%d] because task[%s] of type[%s] does not support queries", + config.getNumThreads(), + task.getId(), + task.getType() + ); + } + return NoopQueryProcessingPool.instance(); + } + } + + @Provides + @LazySingleton + @Global + public NonBlockingPool getIntermediateResultsPool(Task task, DruidProcessingConfig config) + { + if (task.supportsQueries()) { + return DruidProcessingModule.createIntermediateResultsPool(config); + } else { + return DummyNonBlockingPool.instance(); + } + } + + @Provides + @LazySingleton + @Merging + public BlockingPool getMergeBufferPool(Task task, DruidProcessingConfig config) + { + if (task.supportsQueries()) { + return DruidProcessingModule.createMergeBufferPool(config); + } else { + if (config.isNumMergeBuffersConfigured()) { + log.warn( + "Ignoring the configured numMergeBuffers[%d] because task[%s] of type[%s] does not support queries", + config.getNumThreads(), + task.getId(), + task.getType() + ); + } + return DummyBlockingPool.instance(); + } + } + + @Provides + @LazySingleton + @Merging + public GroupByResourcesReservationPool getGroupByResourcesReservationPool( + @Merging BlockingPool mergeBufferPool, + GroupByQueryConfig groupByQueryConfig + ) + { + return new GroupByResourcesReservationPool(mergeBufferPool, groupByQueryConfig); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 73c8a35405c..b3c01d79f98 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -41,6 +41,7 @@ import org.apache.druid.collections.ResourceHolder; import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; @@ -989,9 +990,19 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg // column for it. final ColumnHolder columnHolder = index.getColumnHolder(dimension); if (columnHolder != null) { + DimensionSchema schema = columnHolder.getColumnFormat().getColumnSchema(dimension); + // rewrite string dimensions to always use MultiValueHandling.ARRAY since it preserves the exact order of + // the row regardless of the mode the initial ingest was using + if (schema instanceof StringDimensionSchema) { + schema = new StringDimensionSchema( + schema.getName(), + DimensionSchema.MultiValueHandling.ARRAY, + schema.hasBitmapIndex() + ); + } dimensionSchemaMap.put( dimension, - columnHolder.getColumnFormat().getColumnSchema(dimension) + schema ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java index e1f6d2915ee..06082a988d9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java @@ -47,6 +47,7 @@ import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.coordination.BroadcastDatasourceLoadingSpec; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.timeline.DataSegment; @@ -412,6 +413,12 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask return LookupLoadingSpec.NONE; } + @Override + public BroadcastDatasourceLoadingSpec getBroadcastDatasourceLoadingSpec() + { + return BroadcastDatasourceLoadingSpec.NONE; + } + @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 9b882e2e8d2..cacdc47f520 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -41,13 +41,13 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; +import org.apache.druid.server.coordination.BroadcastDatasourceLoadingSpec; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.server.security.ResourceType; import javax.annotation.Nonnull; -import javax.annotation.Nullable; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -176,7 +176,9 @@ public interface Task /** * True if this task type embeds a query stack, and therefore should preload resources (like broadcast tables) - * that may be needed by queries. + * that may be needed by queries. Tasks supporting queries are also allocated processing buffers, processing threads + * and merge buffers. Those which do not should not assume that these resources are present and must explicitly allocate + * any direct buffers or processing pools if required. * * If true, {@link #getQueryRunner(Query)} does not necessarily return nonnull query runners. For example, * MSQWorkerTask returns true from this method (because it embeds a query stack for running multi-stage queries) @@ -327,9 +329,18 @@ public interface Task * This behaviour can be overridden by passing parameters {@link LookupLoadingSpec#CTX_LOOKUP_LOADING_MODE} * and {@link LookupLoadingSpec#CTX_LOOKUPS_TO_LOAD} in the task context. */ - @Nullable default LookupLoadingSpec getLookupLoadingSpec() { return LookupLoadingSpec.createFromContext(getContext(), LookupLoadingSpec.ALL); } + + /** + * Specifies the list of broadcast datasources to load for this task. Tasks load ALL broadcast datasources by default. + * This behavior can be overridden by passing parameters {@link BroadcastDatasourceLoadingSpec#CTX_BROADCAST_DATASOURCE_LOADING_MODE} + * and {@link BroadcastDatasourceLoadingSpec#CTX_BROADCAST_DATASOURCES_TO_LOAD} in the task context. + */ + default BroadcastDatasourceLoadingSpec getBroadcastDatasourceLoadingSpec() + { + return BroadcastDatasourceLoadingSpec.createFromContext(getContext(), BroadcastDatasourceLoadingSpec.ALL); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java index a85fa73349d..d2da00f38c8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java @@ -49,15 +49,17 @@ import org.apache.druid.segment.ColumnProcessorFactory; import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.realtime.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.utils.CloseableUtils; import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; @@ -118,18 +120,18 @@ public class DruidSegmentReader extends IntermediateRowParsingReader> intermediateRowIterator() throws IOException { final CleanableFile segmentFile = source().fetch(temporaryDirectory, null); - final WindowedStorageAdapter storageAdapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter( - indexIO.loadIndex(segmentFile.file()) - ), + final QueryableIndex queryableIndex = indexIO.loadIndex(segmentFile.file()); + final WindowedCursorFactory windowedCursorFactory = new WindowedCursorFactory( + new QueryableIndexCursorFactory(queryableIndex), intervalFilter ); final CursorBuildSpec cursorBuildSpec = CursorBuildSpec.builder() .setFilter(Filters.toFilter(dimFilter)) - .setInterval(storageAdapter.getInterval()) + .setInterval(windowedCursorFactory.getInterval()) .build(); - final CursorHolder cursorHolder = storageAdapter.getAdapter().makeCursorHolder(cursorBuildSpec); + final CursorFactory cursorFactory = windowedCursorFactory.getCursorFactory(); + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(cursorBuildSpec); final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return CloseableIterators.wrap(Collections.emptyIterator(), cursorHolder); @@ -138,10 +140,7 @@ public class DruidSegmentReader extends IntermediateRowParsingReader columnsToRead = Sets.newLinkedHashSet( - Iterables.filter( - storageAdapter.getAdapter().getRowSignature().getColumnNames(), - columnsFilter::apply - ) + Iterables.filter(cursorFactory.getRowSignature().getColumnNames(), columnsFilter::apply) ); final Sequence> sequence = cursorToSequence(cursor, columnsToRead).withBaggage(cursorHolder); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java index c676877c110..7e4dd6c39c2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java @@ -376,12 +376,16 @@ public class ForkingTaskRunner } // If the task type is queryable, we need to load broadcast segments on the peon, used for - // join queries + // join queries. This is replaced by --loadBroadcastDatasourceMode option, but is preserved here + // for backwards compatibility and can be removed in a future release. if (task.supportsQueries()) { command.add("--loadBroadcastSegments"); command.add("true"); } + command.add("--loadBroadcastDatasourceMode"); + command.add(task.getBroadcastDatasourceLoadingSpec().getMode().toString()); + if (!taskFile.exists()) { jsonMapper.writeValue(taskFile, task); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java index f98287124ed..8b4795d75f5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java @@ -67,14 +67,13 @@ public class InputSourceSampler { private static final String SAMPLER_DATA_SOURCE = "sampler"; - private static final DataSchema DEFAULT_DATA_SCHEMA = new DataSchema( - SAMPLER_DATA_SOURCE, - new TimestampSpec(null, null, null), - new DimensionsSpec(null), - null, - null, - null - ); + + private static final DataSchema DEFAULT_DATA_SCHEMA = + DataSchema.builder() + .withDataSource(SAMPLER_DATA_SOURCE) + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.builder().build()) + .build(); // We want to be able to sort the list of processed results back into the same order that we read them from the input // source so that the rows in the data loader are not always changing. To do this, we add a temporary column to the diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java index 34e4c5f0f2d..309336e1d53 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java @@ -41,6 +41,7 @@ public abstract class SeekableStreamIndexTaskTuningConfig implements Appenderato private static final IndexSpec DEFAULT_INDEX_SPEC = IndexSpec.DEFAULT; private static final Boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = Boolean.FALSE; private static final long DEFAULT_HANDOFF_CONDITION_TIMEOUT = Duration.ofMinutes(15).toMillis(); + private static final int DEFAULT_MAX_COLUMNS_TO_MERGE = -1; private final AppendableIndexSpec appendableIndexSpec; private final int maxRowsInMemory; @@ -66,6 +67,7 @@ public abstract class SeekableStreamIndexTaskTuningConfig implements Appenderato private final int maxSavedParseExceptions; private final int numPersistThreads; + private final int maxColumnsToMerge; public SeekableStreamIndexTaskTuningConfig( @Nullable AppendableIndexSpec appendableIndexSpec, @@ -88,7 +90,8 @@ public abstract class SeekableStreamIndexTaskTuningConfig implements Appenderato @Nullable Boolean logParseExceptions, @Nullable Integer maxParseExceptions, @Nullable Integer maxSavedParseExceptions, - @Nullable Integer numPersistThreads + @Nullable Integer numPersistThreads, + @Nullable Integer maxColumnsToMerge ) { this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; @@ -139,6 +142,7 @@ public abstract class SeekableStreamIndexTaskTuningConfig implements Appenderato : logParseExceptions; this.numPersistThreads = numPersistThreads == null ? DEFAULT_NUM_PERSIST_THREADS : Math.max(numPersistThreads, DEFAULT_NUM_PERSIST_THREADS); + this.maxColumnsToMerge = maxColumnsToMerge == null ? DEFAULT_MAX_COLUMNS_TO_MERGE : maxColumnsToMerge; } @Override @@ -289,6 +293,13 @@ public abstract class SeekableStreamIndexTaskTuningConfig implements Appenderato return numPersistThreads; } + @Override + @JsonProperty + public int getMaxColumnsToMerge() + { + return maxColumnsToMerge; + } + @Override public abstract SeekableStreamIndexTaskTuningConfig withBasePersistDirectory(File dir); @@ -315,6 +326,7 @@ public abstract class SeekableStreamIndexTaskTuningConfig implements Appenderato maxParseExceptions == that.maxParseExceptions && maxSavedParseExceptions == that.maxSavedParseExceptions && numPersistThreads == that.numPersistThreads && + maxColumnsToMerge == that.maxColumnsToMerge && Objects.equals(partitionsSpec, that.partitionsSpec) && Objects.equals(intermediatePersistPeriod, that.intermediatePersistPeriod) && Objects.equals(basePersistDirectory, that.basePersistDirectory) && @@ -347,7 +359,8 @@ public abstract class SeekableStreamIndexTaskTuningConfig implements Appenderato logParseExceptions, maxParseExceptions, maxSavedParseExceptions, - numPersistThreads + numPersistThreads, + maxColumnsToMerge ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java index f6732f68a6c..ff0aa674ef3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java @@ -33,7 +33,6 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.indexing.overlord.SegmentPublishResult; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.indexing.DataSchema; @@ -62,7 +61,7 @@ public class TestIndexTask extends IndexTask id, taskResource, new IndexIngestionSpec( - new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, null, mapper), + DataSchema.builder().withDataSource(dataSource).withObjectMapper(mapper).build(), new IndexTask.IndexIOConfig( new LocalInputSource(new File("lol"), "rofl"), new JsonInputFormat(null, null, null, null, null), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index 1b742971eb9..a8684eba731 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -29,8 +29,10 @@ import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SegmentsSplitHintSpec; import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; @@ -180,7 +182,12 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis expectedLongSumMetric.put("fieldName", "val"); CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("ts", DimensionSchema.MultiValueHandling.ARRAY, null), + new StringDimensionSchema("dim", DimensionSchema.MultiValueHandling.ARRAY, null) + ) + ), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -230,7 +237,12 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); CompactionState expectedState = new CompactionState( new HashedPartitionsSpec(null, 3, null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("ts", DimensionSchema.MultiValueHandling.ARRAY, null), + new StringDimensionSchema("dim", DimensionSchema.MultiValueHandling.ARRAY, null) + ) + ), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -295,7 +307,12 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass()); CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("ts", DimensionSchema.MultiValueHandling.ARRAY, null), + new StringDimensionSchema("dim", DimensionSchema.MultiValueHandling.ARRAY, null) + ) + ), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -410,7 +427,12 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis Assert.assertSame(DimensionRangeShardSpec.class, segment.getShardSpec().getClass()); CompactionState expectedState = new CompactionState( new DimensionRangePartitionsSpec(7, null, Arrays.asList("dim1", "dim2"), false), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("ts", DimensionSchema.MultiValueHandling.ARRAY, null), + new StringDimensionSchema("dim", DimensionSchema.MultiValueHandling.ARRAY, null) + ) + ), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -460,7 +482,12 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass()); CompactionState expectedState = new CompactionState( new SingleDimensionPartitionsSpec(7, null, "dim", false), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("ts", DimensionSchema.MultiValueHandling.ARRAY, null), + new StringDimensionSchema("dim", DimensionSchema.MultiValueHandling.ARRAY, null) + ) + ), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -513,7 +540,12 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis Assert.assertSame(DimensionRangeShardSpec.class, segment.getShardSpec().getClass()); CompactionState expectedState = new CompactionState( new DimensionRangePartitionsSpec(7, null, Arrays.asList("dim1", "dim2"), false), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("ts", DimensionSchema.MultiValueHandling.ARRAY, null), + new StringDimensionSchema("dim", DimensionSchema.MultiValueHandling.ARRAY, null) + ) + ), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -596,7 +628,12 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis expectedLongSumMetric.put("fieldName", "val"); CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("ts", DimensionSchema.MultiValueHandling.ARRAY, null), + new StringDimensionSchema("dim", DimensionSchema.MultiValueHandling.ARRAY, null) + ) + ), ImmutableList.of(expectedLongSumMetric), getObjectMapper().readValue( getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), @@ -658,7 +695,12 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis expectedLongSumMetric.put("fieldName", "val"); CompactionState expectedState = new CompactionState( new DynamicPartitionsSpec(null, Long.MAX_VALUE), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("ts", DimensionSchema.MultiValueHandling.ARRAY, null), + new StringDimensionSchema("dim", DimensionSchema.MultiValueHandling.ARRAY, null) + ) + ), ImmutableList.of(expectedCountMetric, expectedLongSumMetric), getObjectMapper().readValue( getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), @@ -928,7 +970,8 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis "|", null, false, - 0 + 0, + null ), appendToExisting, null @@ -939,18 +982,19 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis null, null, new ParallelIndexIngestionSpec( - new DataSchema( - DATA_SOURCE, - new TimestampSpec("ts", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim"))), - new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")}, - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.MINUTE, - ImmutableList.of(INTERVAL_TO_INDEX) - ), - null - ), + DataSchema.builder() + .withDataSource(DATA_SOURCE) + .withTimestamp(new TimestampSpec("ts", "auto", null)) + .withDimensions(DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim"))) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + ImmutableList.of(INTERVAL_TO_INDEX) + ) + ) + .build(), ioConfig, tuningConfig ), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 307a44dbf7e..a3fb807604f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -34,6 +34,7 @@ import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.client.indexing.NoopOverlordClient; import org.apache.druid.data.input.impl.CSVParseSpec; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.NewSpatialDimensionSchema; @@ -80,7 +81,7 @@ import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -99,7 +100,7 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.loading.TombstoneLoadSpec; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; @@ -227,7 +228,12 @@ public class CompactionTaskRunTest extends IngestionTestBase segmentGranularity, queryGranularity, intervals, - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("ts", DimensionSchema.MultiValueHandling.ARRAY, null), + new StringDimensionSchema("dim", DimensionSchema.MultiValueHandling.ARRAY, null) + ) + ), expectedLongSumMetric ); } @@ -386,7 +392,12 @@ public class CompactionTaskRunTest extends IngestionTestBase expectedLongSumMetric.put("fieldName", "val"); CompactionState expectedState = new CompactionState( new HashedPartitionsSpec(null, 3, null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("ts", DimensionSchema.MultiValueHandling.ARRAY, null), + new StringDimensionSchema("dim", DimensionSchema.MultiValueHandling.ARRAY, null) + ) + ), ImmutableList.of(expectedLongSumMetric), null, compactionTask.getTuningConfig().getIndexSpec().asMap(getObjectMapper()), @@ -801,7 +812,12 @@ public class CompactionTaskRunTest extends IngestionTestBase expectedLongSumMetric.put("fieldName", "val"); CompactionState expectedCompactionState = new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("ts", DimensionSchema.MultiValueHandling.ARRAY, null), + new StringDimensionSchema("dim", DimensionSchema.MultiValueHandling.ARRAY, null) + ) + ), ImmutableList.of(expectedLongSumMetric), getObjectMapper().readValue(getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class), IndexSpec.DEFAULT.asMap(mapper), @@ -866,7 +882,12 @@ public class CompactionTaskRunTest extends IngestionTestBase expectedLongSumMetric.put("fieldName", "val"); CompactionState expectedCompactionState = new CompactionState( new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))), + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("ts", DimensionSchema.MultiValueHandling.ARRAY, null), + new StringDimensionSchema("dim", DimensionSchema.MultiValueHandling.ARRAY, null) + ) + ), ImmutableList.of(expectedCountMetric, expectedLongSumMetric), getObjectMapper().readValue(getObjectMapper().writeValueAsString(compactionTask.getTransformSpec()), Map.class), IndexSpec.DEFAULT.asMap(mapper), @@ -1676,8 +1697,8 @@ public class CompactionTaskRunTest extends IngestionTestBase ImmutableList.of(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1)), DimensionsSpec.builder() .setDimensions(Arrays.asList( - new StringDimensionSchema("ts"), - new StringDimensionSchema("dim"), + new StringDimensionSchema("ts", DimensionSchema.MultiValueHandling.ARRAY, null), + new StringDimensionSchema("dim", DimensionSchema.MultiValueHandling.ARRAY, null), new NewSpatialDimensionSchema("spatial", Collections.singletonList("spatial")) )) .build(), @@ -1702,11 +1723,11 @@ public class CompactionTaskRunTest extends IngestionTestBase for (DataSegment segment : segments) { final File segmentFile = segmentCacheManager.getSegmentFiles(segment); - final WindowedStorageAdapter adapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(testUtils.getTestIndexIO().loadIndex(segmentFile)), + final WindowedCursorFactory windowed = new WindowedCursorFactory( + new QueryableIndexCursorFactory(testUtils.getTestIndexIO().loadIndex(segmentFile)), segment.getInterval() ); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = windowed.getCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); Assert.assertNotNull(cursor); cursor.reset(); @@ -1830,11 +1851,11 @@ public class CompactionTaskRunTest extends IngestionTestBase for (DataSegment segment : segments) { final File segmentFile = segmentCacheManager.getSegmentFiles(segment); - final WindowedStorageAdapter adapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(testUtils.getTestIndexIO().loadIndex(segmentFile)), + final WindowedCursorFactory windowed = new WindowedCursorFactory( + new QueryableIndexCursorFactory(testUtils.getTestIndexIO().loadIndex(segmentFile)), segment.getInterval() ); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = windowed.getCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); Assert.assertNotNull(cursor); cursor.reset(); @@ -1964,8 +1985,8 @@ public class CompactionTaskRunTest extends IngestionTestBase final File segmentFile = segmentCacheManager.getSegmentFiles(compactSegment); final QueryableIndex queryableIndex = testUtils.getTestIndexIO().loadIndex(segmentFile); - final WindowedStorageAdapter adapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(queryableIndex), + final WindowedCursorFactory windowed = new WindowedCursorFactory( + new QueryableIndexCursorFactory(queryableIndex), compactSegment.getInterval() ); Assert.assertEquals( @@ -1980,7 +2001,7 @@ public class CompactionTaskRunTest extends IngestionTestBase ); try (final CursorHolder cursorHolder = - adapter.getAdapter() + windowed.getCursorFactory() .makeCursorHolder(CursorBuildSpec.builder().setInterval(compactSegment.getInterval()).build())) { final Cursor cursor = cursorHolder.asCursor(); cursor.reset(); @@ -2210,11 +2231,11 @@ public class CompactionTaskRunTest extends IngestionTestBase for (DataSegment segment : segments) { final File segmentFile = segmentCacheManager.getSegmentFiles(segment); - final WindowedStorageAdapter adapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(testUtils.getTestIndexIO().loadIndex(segmentFile)), + final WindowedCursorFactory windowed = new WindowedCursorFactory( + new QueryableIndexCursorFactory(testUtils.getTestIndexIO().loadIndex(segmentFile)), segment.getInterval() ); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = windowed.getCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); Assert.assertNotNull(cursor); cursor.reset(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index c1bf649980f..90383b2b8e3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -1619,10 +1619,10 @@ public class CompactionTaskTest private static List getExpectedDimensionsSpecForAutoGeneration() { return ImmutableList.of( - new DimensionsSpec(getDimensionSchema(new StringDimensionSchema("string_to_double"))), - new DimensionsSpec(getDimensionSchema(new StringDimensionSchema("string_to_double"))), - new DimensionsSpec(getDimensionSchema(new StringDimensionSchema("string_to_double"))), - new DimensionsSpec(getDimensionSchema(new StringDimensionSchema("string_to_double"))), + new DimensionsSpec(getDimensionSchema(new StringDimensionSchema("string_to_double", DimensionSchema.MultiValueHandling.ARRAY, null))), + new DimensionsSpec(getDimensionSchema(new StringDimensionSchema("string_to_double", DimensionSchema.MultiValueHandling.ARRAY, null))), + new DimensionsSpec(getDimensionSchema(new StringDimensionSchema("string_to_double", DimensionSchema.MultiValueHandling.ARRAY, null))), + new DimensionsSpec(getDimensionSchema(new StringDimensionSchema("string_to_double", DimensionSchema.MultiValueHandling.ARRAY, null))), new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double"))), new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double"))) ); @@ -1632,27 +1632,27 @@ public class CompactionTaskTest { return Lists.newArrayList( new LongDimensionSchema("timestamp"), - new StringDimensionSchema("string_dim_4"), + new StringDimensionSchema("string_dim_4", DimensionSchema.MultiValueHandling.ARRAY, null), new LongDimensionSchema("long_dim_4"), new FloatDimensionSchema("float_dim_4"), new DoubleDimensionSchema("double_dim_4"), - new StringDimensionSchema("string_dim_0"), + new StringDimensionSchema("string_dim_0", DimensionSchema.MultiValueHandling.ARRAY, null), new LongDimensionSchema("long_dim_0"), new FloatDimensionSchema("float_dim_0"), new DoubleDimensionSchema("double_dim_0"), - new StringDimensionSchema("string_dim_1"), + new StringDimensionSchema("string_dim_1", DimensionSchema.MultiValueHandling.ARRAY, null), new LongDimensionSchema("long_dim_1"), new FloatDimensionSchema("float_dim_1"), new DoubleDimensionSchema("double_dim_1"), - new StringDimensionSchema("string_dim_2"), + new StringDimensionSchema("string_dim_2", DimensionSchema.MultiValueHandling.ARRAY, null), new LongDimensionSchema("long_dim_2"), new FloatDimensionSchema("float_dim_2"), new DoubleDimensionSchema("double_dim_2"), - new StringDimensionSchema("string_dim_3"), + new StringDimensionSchema("string_dim_3", DimensionSchema.MultiValueHandling.ARRAY, null), new LongDimensionSchema("long_dim_3"), new FloatDimensionSchema("float_dim_3"), new DoubleDimensionSchema("double_dim_3"), - new StringDimensionSchema("string_dim_5"), + new StringDimensionSchema("string_dim_5", DimensionSchema.MultiValueHandling.ARRAY, null), new LongDimensionSchema("long_dim_5"), new FloatDimensionSchema("float_dim_5"), new DoubleDimensionSchema("double_dim_5"), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopIndexTaskTest.java index ff828f16789..0a72b77e1db 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopIndexTaskTest.java @@ -27,7 +27,6 @@ import org.apache.druid.indexer.HadoopIngestionSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.security.Action; @@ -50,15 +49,19 @@ public class HadoopIndexTaskTest final HadoopIndexTask task = new HadoopIndexTask( null, new HadoopIngestionSpec( - new DataSchema( - "foo", null, new AggregatorFactory[0], new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P1D")) - ), - null, - jsonMapper - ), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P1D")) + ) + ) + .withObjectMapper(jsonMapper) + .build(), + new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), + null ), null, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java index ab953ba954a..d84aa154fd2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java @@ -25,7 +25,6 @@ import org.apache.druid.data.input.impl.NoopInputSource; import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.junit.Rule; @@ -45,14 +44,11 @@ public class IndexIngestionSpecTest "Cannot use parser and inputSource together. Try using inputFormat instead of parser." ); final IndexIngestionSpec spec = new IndexIngestionSpec( - new DataSchema( - "dataSource", - ImmutableMap.of("fake", "parser map"), - new AggregatorFactory[0], - new ArbitraryGranularitySpec(Granularities.NONE, null), - null, - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withParserMap(ImmutableMap.of("fake", "parser map")) + .withGranularity(new ArbitraryGranularitySpec(Granularities.NONE, null)) + .build(), new IndexIOConfig( new NoopInputSource(), new NoopInputFormat(), @@ -69,14 +65,11 @@ public class IndexIngestionSpecTest expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot use parser and inputSource together."); final IndexIngestionSpec spec = new IndexIngestionSpec( - new DataSchema( - "dataSource", - ImmutableMap.of("fake", "parser map"), - new AggregatorFactory[0], - new ArbitraryGranularitySpec(Granularities.NONE, null), - null, - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withParserMap(ImmutableMap.of("fake", "parser map")) + .withGranularity(new ArbitraryGranularitySpec(Granularities.NONE, null)) + .build(), new IndexIOConfig( new NoopInputSource(), null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index c54446fc4df..be8fdb2defa 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -73,7 +73,7 @@ import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnType; @@ -93,7 +93,7 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLocalCacheManager; import org.apache.druid.segment.loading.StorageLocation; import org.apache.druid.segment.loading.StorageLocationConfig; -import org.apache.druid.segment.realtime.WindowedStorageAdapter; +import org.apache.druid.segment.realtime.WindowedCursorFactory; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -163,9 +163,29 @@ public class IndexTaskTest extends IngestionTestBase null, null, false, - 0 + 0, + null ); + private static final DataSchema DATA_SCHEMA = + DataSchema.builder() + .withDataSource("test-json") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions( + new StringDimensionSchema("ts"), + new StringDimensionSchema("dim"), + new LongDimensionSchema("valDim") + ) + .withAggregators(new LongSumAggregatorFactory("valMet", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(Intervals.of("2014/P1D")) + ) + ) + .build(); + @Parameterized.Parameters(name = "{0}, useInputFormatApi={1}") public static Iterable constructorFeeder() { @@ -225,24 +245,7 @@ public class IndexTaskTest extends IngestionTestBase { IndexTask indexTask = createIndexTask( new IndexIngestionSpec( - new DataSchema( - "test-json", - DEFAULT_TIMESTAMP_SPEC, - new DimensionsSpec( - ImmutableList.of( - new StringDimensionSchema("ts"), - new StringDimensionSchema("dim"), - new LongDimensionSchema("valDim") - ) - ), - new AggregatorFactory[]{new LongSumAggregatorFactory("valMet", "val")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2014/P1D")) - ), - null - ), + DATA_SCHEMA, new IndexIOConfig( new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, @@ -275,24 +278,7 @@ public class IndexTaskTest extends IngestionTestBase IndexTask indexTask = createIndexTask( new IndexIngestionSpec( - new DataSchema( - "test-json", - DEFAULT_TIMESTAMP_SPEC, - new DimensionsSpec( - ImmutableList.of( - new StringDimensionSchema("ts"), - new StringDimensionSchema("dim"), - new LongDimensionSchema("valDim") - ) - ), - new AggregatorFactory[]{new LongSumAggregatorFactory("valMet", "val")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2014/P1D")) - ), - null - ), + DATA_SCHEMA, new IndexIOConfig( new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, @@ -337,24 +323,7 @@ public class IndexTaskTest extends IngestionTestBase IndexTask indexTask = createIndexTask( new IndexIngestionSpec( - new DataSchema( - "test-json", - DEFAULT_TIMESTAMP_SPEC, - new DimensionsSpec( - ImmutableList.of( - new StringDimensionSchema("ts"), - new StringDimensionSchema("dim"), - new LongDimensionSchema("valDim") - ) - ), - new AggregatorFactory[]{new LongSumAggregatorFactory("valMet", "val")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2014/P1D")) - ), - null - ), + DATA_SCHEMA, new IndexIOConfig( new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, @@ -505,7 +474,7 @@ public class IndexTaskTest extends IngestionTestBase indexIngestionSpec = createIngestionSpec( DEFAULT_TIMESTAMP_SPEC, dimensionsSpec, - new CsvInputFormat(columns, listDelimiter, null, false, 0), + new CsvInputFormat(columns, listDelimiter, null, false, 0, null), transformSpec, null, tuningConfig, @@ -536,11 +505,11 @@ public class IndexTaskTest extends IngestionTestBase DataSegment segment = segments.get(0); final File segmentFile = segmentCacheManager.getSegmentFiles(segment); - final WindowedStorageAdapter adapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(indexIO.loadIndex(segmentFile)), + final WindowedCursorFactory windowed = new WindowedCursorFactory( + new QueryableIndexCursorFactory(indexIO.loadIndex(segmentFile)), segment.getInterval() ); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = windowed.getCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final List> transforms = new ArrayList<>(); @@ -765,12 +734,12 @@ public class IndexTaskTest extends IngestionTestBase final File segmentFile = segmentCacheManager.getSegmentFiles(segment); - final WindowedStorageAdapter adapter = new WindowedStorageAdapter( - new QueryableIndexStorageAdapter(indexIO.loadIndex(segmentFile)), + final WindowedCursorFactory windowed = new WindowedCursorFactory( + new QueryableIndexCursorFactory(indexIO.loadIndex(segmentFile)), segment.getInterval() ); - try (final CursorHolder cursorHolder = adapter.getAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = windowed.getCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); final List hashes = new ArrayList<>(); final DimensionSelector selector = cursor.getColumnSelectorFactory() @@ -933,7 +902,7 @@ public class IndexTaskTest extends IngestionTestBase ingestionSpec = createIngestionSpec( timestampSpec, DimensionsSpec.EMPTY, - new CsvInputFormat(null, null, null, true, 0), + new CsvInputFormat(null, null, null, true, 0, null), null, null, tuningConfig, @@ -973,7 +942,7 @@ public class IndexTaskTest extends IngestionTestBase ingestionSpec = createIngestionSpec( timestampSpec, DimensionsSpec.EMPTY, - new CsvInputFormat(columns, null, null, true, 0), + new CsvInputFormat(columns, null, null, true, 0, null), null, null, tuningConfig, @@ -1373,7 +1342,7 @@ public class IndexTaskTest extends IngestionTestBase parseExceptionIgnoreSpec = createIngestionSpec( timestampSpec, DimensionsSpec.EMPTY, - new CsvInputFormat(columns, null, null, true, 0), + new CsvInputFormat(columns, null, null, true, 0, null), null, null, tuningConfig, @@ -1423,7 +1392,7 @@ public class IndexTaskTest extends IngestionTestBase indexIngestionSpec = createIngestionSpec( timestampSpec, DimensionsSpec.EMPTY, - new CsvInputFormat(columns, null, null, true, 0), + new CsvInputFormat(columns, null, null, true, 0, null), null, null, tuningConfig, @@ -1664,7 +1633,7 @@ public class IndexTaskTest extends IngestionTestBase ingestionSpec = createIngestionSpec( timestampSpec, dimensionsSpec, - new CsvInputFormat(columns, null, null, true, 0), + new CsvInputFormat(columns, null, null, true, 0, null), null, null, tuningConfig, @@ -1783,7 +1752,7 @@ public class IndexTaskTest extends IngestionTestBase ingestionSpec = createIngestionSpec( timestampSpec, dimensionsSpec, - new CsvInputFormat(columns, null, null, true, 0), + new CsvInputFormat(columns, null, null, true, 0, null), null, null, tuningConfig, @@ -1877,7 +1846,7 @@ public class IndexTaskTest extends IngestionTestBase ingestionSpec = createIngestionSpec( DEFAULT_TIMESTAMP_SPEC, DimensionsSpec.EMPTY, - new CsvInputFormat(null, null, null, true, 0), + new CsvInputFormat(null, null, null, true, 0, null), null, null, tuningConfig, @@ -1947,7 +1916,7 @@ public class IndexTaskTest extends IngestionTestBase ingestionSpec = createIngestionSpec( DEFAULT_TIMESTAMP_SPEC, DimensionsSpec.EMPTY, - new CsvInputFormat(columns, null, null, true, 0), + new CsvInputFormat(columns, null, null, true, 0, null), null, null, tuningConfig, @@ -2698,20 +2667,20 @@ public class IndexTaskTest extends IngestionTestBase if (inputFormat != null) { Preconditions.checkArgument(parseSpec == null, "Can't use parseSpec"); return new IndexIngestionSpec( - new DataSchema( - DATASOURCE, - Preconditions.checkNotNull(timestampSpec, "timestampSpec"), - Preconditions.checkNotNull(dimensionsSpec, "dimensionsSpec"), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - granularitySpec != null ? granularitySpec : new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2014/2015")) - ), - transformSpec - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(Preconditions.checkNotNull(timestampSpec, "timestampSpec")) + .withDimensions(Preconditions.checkNotNull(dimensionsSpec, "dimensionsSpec")) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + granularitySpec != null ? granularitySpec : new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(Intervals.of("2014/2015")) + ) + ) + .withTransform(transformSpec) + .build(), new IndexIOConfig( new LocalInputSource(baseDir, "druid*"), inputFormat, @@ -2723,22 +2692,21 @@ public class IndexTaskTest extends IngestionTestBase } else { parseSpec = parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC; return new IndexIngestionSpec( - new DataSchema( - DATASOURCE, - parseSpec.getTimestampSpec(), - parseSpec.getDimensionsSpec(), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - granularitySpec != null ? granularitySpec : new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2014/2015")) - ), - transformSpec, - null, - objectMapper - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(parseSpec.getTimestampSpec()) + .withDimensions(parseSpec.getDimensionsSpec()) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + granularitySpec != null ? granularitySpec : new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(Intervals.of("2014/2015")) + ) + ) + .withTransform(transformSpec) + .withObjectMapper(objectMapper) + .build(), new IndexIOConfig( new LocalInputSource(baseDir, "druid*"), createInputFormatFromParseSpec(parseSpec), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 67a0c518f57..472b74cd412 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -313,7 +313,8 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest csvParseSpec.getListDelimiter(), getColumnsFromHeader ? null : true, getColumnsFromHeader ? true : null, - csvParseSpec.getSkipHeaderRows() + csvParseSpec.getSkipHeaderRows(), + null ); } else if (parseSpec instanceof DelimitedParseSpec) { DelimitedParseSpec delimitedParseSpec = (DelimitedParseSpec) parseSpec; @@ -324,7 +325,8 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest delimitedParseSpec.getDelimiter(), getColumnsFromHeader ? null : true, getColumnsFromHeader ? true : null, - delimitedParseSpec.getSkipHeaderRows() + delimitedParseSpec.getSkipHeaderRows(), + null ); } else if (parseSpec instanceof RegexParseSpec) { RegexParseSpec regexParseSpec = (RegexParseSpec) parseSpec; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java index fe2b5a51c86..7a4df9de36c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java @@ -37,6 +37,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.metadata.IndexerSqlMetadataStorageCoordinatorTestBase; +import org.apache.druid.server.coordination.BroadcastDatasourceLoadingSpec; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.timeline.DataSegment; import org.assertj.core.api.Assertions; @@ -601,6 +602,16 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase Assert.assertEquals(LookupLoadingSpec.Mode.NONE, task.getLookupLoadingSpec().getMode()); } + @Test + public void testGetBroadcastDatasourcesToLoad() + { + final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder() + .dataSource(DATA_SOURCE) + .interval(Intervals.of("2019-03-01/2019-04-01")) + .build(); + Assert.assertEquals(BroadcastDatasourceLoadingSpec.Mode.NONE, task.getBroadcastDatasourceLoadingSpec().getMode()); + } + @Test public void testKillBatchSizeOneAndLimit4() throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java index e6ea0e1329a..99b0f8e7a75 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java @@ -38,7 +38,6 @@ import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.DataSchema; @@ -220,18 +219,19 @@ public class TaskSerdeTest null, null, new IndexIngestionSpec( - new DataSchema( - "foo", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P2D")) - ), - null - ), + DataSchema.builder() + .withDataSource("foo") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P2D")) + ) + ) + .build(), new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), TuningConfigBuilder.forIndexTask() .withMaxRowsInMemory(10) @@ -288,18 +288,19 @@ public class TaskSerdeTest null, new TaskResource("rofl", 2), new IndexIngestionSpec( - new DataSchema( - "foo", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P2D")) - ), - null - ), + DataSchema.builder() + .withDataSource("foo") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P2D")) + ) + ) + .build(), new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), TuningConfigBuilder.forIndexTask() .withMaxRowsInMemory(10) @@ -412,15 +413,19 @@ public class TaskSerdeTest final HadoopIndexTask task = new HadoopIndexTask( null, new HadoopIngestionSpec( - new DataSchema( - "foo", null, new AggregatorFactory[0], new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P1D")) - ), - null, - jsonMapper - ), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P1D")) + ) + ) + .withObjectMapper(jsonMapper) + .build(), + new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), + null ), null, null, @@ -454,19 +459,18 @@ public class TaskSerdeTest final HadoopIndexTask task = new HadoopIndexTask( null, new HadoopIngestionSpec( - new DataSchema( - "foo", - null, - null, - new AggregatorFactory[0], - new UniformGranularitySpec( - Granularities.DAY, - null, ImmutableList.of(Intervals.of("2010-01-01/P1D")) - ), - null, - null, - jsonMapper - ), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, ImmutableList.of(Intervals.of("2010-01-01/P1D")) + ) + ) + .withObjectMapper(jsonMapper) + .build(), + new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), + null ), null, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java index 44b8284f407..6caab0a0652 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -221,14 +221,13 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn dropExisting ); ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - timestampSpec, - dimensionsSpec, - DEFAULT_METRICS_SPEC, - granularitySpec, - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(timestampSpec) + .withDimensions(dimensionsSpec) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity(granularitySpec) + .build(), ioConfig, tuningConfig ); @@ -241,14 +240,13 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn dropExisting ); ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - parseSpec.getTimestampSpec(), - parseSpec.getDimensionsSpec(), - DEFAULT_METRICS_SPEC, - granularitySpec, - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(parseSpec.getTimestampSpec()) + .withDimensions(parseSpec.getDimensionsSpec()) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity(granularitySpec) + .build(), ioConfig, tuningConfig ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 1cd6a29e10d..d96a4d2a37e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -163,7 +163,8 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase null, false, false, - 0 + 0, + null ); public static final ParallelIndexTuningConfig DEFAULT_TUNING_CONFIG_FOR_PARALLEL_INDEXING = TuningConfigBuilder.forParallelIndexTask() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java index 0839ed04405..075dae22f5c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java @@ -61,7 +61,8 @@ public class HashPartitionAdjustingCorePartitionSizeTest extends AbstractMultiPh null, false, false, - 0 + 0, + null ); private static final Interval INTERVAL_TO_INDEX = Intervals.of("2020-01-01/P1M"); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java index cb58d6f79df..53df996e805 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java @@ -84,7 +84,8 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh null, false, false, - 0 + 0, + null ); private static final Interval INTERVAL_TO_INDEX = Intervals.of("2017-12/P1M"); private static final String INPUT_FILTER = "test_*"; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java index b8c59d042a3..3af5952abea 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java @@ -41,7 +41,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Comparators; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; @@ -80,7 +79,8 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin null, false, false, - 0 + 0, + null ); private static final Interval INTERVAL_TO_INDEX = Intervals.of("2017-12/P1M"); @@ -250,14 +250,13 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin null ); ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - timestampSpec, - dimensionsSpec, - new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")}, - granularitySpec, - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(timestampSpec) + .withDimensions(dimensionsSpec) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity(granularitySpec) + .build(), ioConfig, tuningConfig ); @@ -271,16 +270,13 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin ); //noinspection unchecked ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - parseSpec.getTimestampSpec(), - parseSpec.getDimensionsSpec(), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - granularitySpec, - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(parseSpec.getTimestampSpec()) + .withDimensions(parseSpec.getDimensionsSpec()) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity(granularitySpec) + .build(), ioConfig, tuningConfig ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java index 0d19cd86e03..3adc154bb22 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java @@ -127,18 +127,19 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP null, null, new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - TIMESTAMP_SPEC, - DIMENSIONS_SPEC.withDimensions(dimensionSchemas), - DEFAULT_METRICS_SPEC, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - INTERVAL_TO_INDEX - ), - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas)) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + INTERVAL_TO_INDEX + ) + ) + .build(), new ParallelIndexIOConfig( getInputSource(), JSON_FORMAT, @@ -177,18 +178,21 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP null, null, new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - TIMESTAMP_SPEC, - new DimensionsSpec.Builder().setDimensions(dimensionSchemas).setIncludeAllDimensions(true).build(), - DEFAULT_METRICS_SPEC, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - INTERVAL_TO_INDEX - ), - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(TIMESTAMP_SPEC) + .withDimensions( + DimensionsSpec.builder().setDimensions(dimensionSchemas).setIncludeAllDimensions(true).build() + ) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + INTERVAL_TO_INDEX + ) + ) + .build(), new ParallelIndexIOConfig( getInputSource(), new JsonInputFormat( @@ -237,18 +241,21 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP null, null, new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - TIMESTAMP_SPEC, - new DimensionsSpec.Builder().setIncludeAllDimensions(true).build(), - DEFAULT_METRICS_SPEC, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - null - ), - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(TIMESTAMP_SPEC) + .withDimensions( + DimensionsSpec.builder().setIncludeAllDimensions(true).build() + ) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + null + ) + ) + .build(), new ParallelIndexIOConfig( getInputSource(), new JsonInputFormat( @@ -303,20 +310,23 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP null, null, new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - TIMESTAMP_SPEC, - DIMENSIONS_SPEC.withDimensions( - DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "unknownDim")) - ), - DEFAULT_METRICS_SPEC, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - INTERVAL_TO_INDEX - ), - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(TIMESTAMP_SPEC) + .withDimensions( + DIMENSIONS_SPEC.withDimensions( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "unknownDim")) + ) + ) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + INTERVAL_TO_INDEX + ) + ) + .build(), new ParallelIndexIOConfig( getInputSource(), JSON_FORMAT, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index cbf71146973..2ea9385a014 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -36,7 +36,6 @@ import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -147,20 +146,19 @@ public class ParallelIndexSupervisorTaskKillTest extends AbstractParallelIndexSu final int numTotalSubTasks = inputSource.estimateNumSplits(new NoopInputFormat(), null); // set up ingestion spec final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DEFAULT_DIMENSIONS_SPEC, - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ) + ) + .build(), ioConfig, TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(numTotalSubTasks).build() ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index c9858b80847..4587ef6ce7e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -45,7 +45,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -400,20 +399,19 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd ) { final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DEFAULT_DIMENSIONS_SPEC, - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ) + ) + .build(), ioConfig, TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(NUM_SUB_TASKS).build() ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index e11fb2ef001..85929291e8b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -34,7 +34,6 @@ import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -183,7 +182,7 @@ public class ParallelIndexSupervisorTaskSerdeTest private final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( new LocalInputSource(new File("tmp"), "test_*"), - new CsvInputFormat(Arrays.asList("ts", "dim", "val"), null, null, false, 0), + new CsvInputFormat(Arrays.asList("ts", "dim", "val"), null, null, false, 0, null), false, null ); @@ -220,16 +219,19 @@ public class ParallelIndexSupervisorTaskSerdeTest ParallelIndexIngestionSpec build() { - DataSchema dataSchema = new DataSchema( - "dataSource", - TIMESTAMP_SPEC, - DIMENSIONS_SPEC, - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.MINUTE, inputIntervals), - null - ); + DataSchema dataSchema = DataSchema.builder() + .withDataSource("datasource") + .withTimestamp(TIMESTAMP_SPEC) + .withDimensions(DIMENSIONS_SPEC) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + inputIntervals + ) + ) + .build(); ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder .forParallelIndexTask() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index 57dbafa173f..b908d274e6c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -263,14 +263,11 @@ public class ParallelIndexSupervisorTaskTest .withLogParseExceptions(false) .build(); final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "datasource", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - null, - null, - null - ), + DataSchema.builder() + .withDataSource("datasource") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .build(), ioConfig, tuningConfig ); @@ -325,25 +322,24 @@ public class ParallelIndexSupervisorTaskTest expectedException.expect(IAE.class); expectedException.expectMessage("Cannot use parser and inputSource together. Try using inputFormat instead of parser."); new ParallelIndexIngestionSpec( - new DataSchema( - "datasource", - mapper.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - null, - null, - null - ) - ), - Map.class - ), - null, - null, - null, - mapper - ), + DataSchema.builder() + .withDataSource("datasource") + .withParserMap( + mapper.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY, + null, + null, + null + ) + ), + Map.class + ) + ) + .withObjectMapper(mapper) + .build(), ioConfig, tuningConfig ); @@ -559,14 +555,11 @@ public class ParallelIndexSupervisorTaskTest .build(); final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "datasource", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - null, - null, - null - ), + DataSchema.builder() + .withDataSource("datasource") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .build(), ioConfig, tuningConfig ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java index 2455ce692b9..f93ea8d0a8b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java @@ -31,11 +31,9 @@ import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashPartitionFunction; import org.joda.time.Interval; @@ -97,16 +95,13 @@ class ParallelIndexTestingFactory DimensionsSpec.getDefaultSchemas(ImmutableList.of(SCHEMA_DIMENSION)) ); - return new DataSchema( - DATASOURCE, - timestampSpec, - dimensionsSpec, - new AggregatorFactory[]{}, - granularitySpec, - TransformSpec.NONE, - null, - NESTED_OBJECT_MAPPER - ); + return DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(timestampSpec) + .withDimensions(dimensionsSpec) + .withGranularity(granularitySpec) + .withObjectMapper(NESTED_OBJECT_MAPPER) + .build(); } static ParallelIndexIngestionSpec createIngestionSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java index 9962dfcd821..9ef316e8aaa 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java @@ -64,7 +64,8 @@ public class PartialCompactionTest extends AbstractMultiPhaseParallelIndexingTes null, false, false, - 0 + 0, + null ); private static final Interval INTERVAL_TO_INDEX = Intervals.of("2017-12/P1M"); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionAdjustingCorePartitionSizeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionAdjustingCorePartitionSizeTest.java index 65b58fb6fd5..efa8991b206 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionAdjustingCorePartitionSizeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionAdjustingCorePartitionSizeTest.java @@ -61,7 +61,8 @@ public class RangePartitionAdjustingCorePartitionSizeTest extends AbstractMultiP null, false, false, - 0 + 0, + null ); private static final Interval INTERVAL_TO_INDEX = Intervals.of("2020-01-01/P1M"); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionMultiPhaseParallelIndexingTest.java index 9c04ce6c6ee..3de358fdb5f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionMultiPhaseParallelIndexingTest.java @@ -110,7 +110,8 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP LIST_DELIMITER, false, false, - 0 + 0, + null ); @Parameterized.Parameters(name = "{0}, useInputFormatApi={1}, maxNumConcurrentSubTasks={2}, useMultiValueDim={3}, intervalToIndex={4}") diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java index 96494b8ac79..269ab6aa0be 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java @@ -39,7 +39,6 @@ import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; @@ -85,7 +84,8 @@ public class RangePartitionTaskKillTest extends AbstractMultiPhaseParallelIndexi LIST_DELIMITER, false, false, - 0 + 0, + null ); private File inputDir; @@ -329,14 +329,13 @@ public class RangePartitionTaskKillTest extends AbstractMultiPhaseParallelIndexi null ); ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - timestampSpec, - dimensionsSpec, - new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")}, - granularitySpec, - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(timestampSpec) + .withDimensions(dimensionsSpec) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity(granularitySpec) + .build(), ioConfig, tuningConfig ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index ac8371acaa0..b5122490864 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -43,7 +43,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.DataSegmentsWithSchemas; @@ -391,20 +390,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv null, null, new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2017-12/P1M")) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas)) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(Intervals.of("2017-12/P1M")) + ) + ) + .build(), new ParallelIndexIOConfig( new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true), DEFAULT_INPUT_FORMAT, @@ -444,20 +442,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv null, null, new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2017-12/P1M")) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas)) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(Intervals.of("2017-12/P1M")) + ) + ) + .build(), new ParallelIndexIOConfig( new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true), DEFAULT_INPUT_FORMAT, @@ -785,21 +782,24 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv null, null, new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DimensionsSpec.builder() - .setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim")) - .setIncludeAllDimensions(true) - .build(), - new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(interval) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions( + DimensionsSpec.builder() + .setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim")) + .setIncludeAllDimensions(true) + .build() + ) + .withAggregators(new CountAggregatorFactory("cnt")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(interval) + ) + ) + .build(), new ParallelIndexIOConfig( new SettableSplittableLocalInputSource(inputDir, "*.json", true), new JsonInputFormat( @@ -868,21 +868,24 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv null, null, new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DimensionsSpec.builder() - .setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim")) - .useSchemaDiscovery(true) - .build(), - new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(interval) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions( + DimensionsSpec.builder() + .setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim")) + .useSchemaDiscovery(true) + .build() + ) + .withAggregators(new CountAggregatorFactory("cnt")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(interval) + ) + ) + .build(), new ParallelIndexIOConfig( new SettableSplittableLocalInputSource(inputDir, "*.json", true), new JsonInputFormat( @@ -948,20 +951,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv final ParallelIndexIngestionSpec ingestionSpec; if (useInputFormatApi) { ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DEFAULT_DIMENSIONS_SPEC, - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - segmentGranularity, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + segmentGranularity, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ) + ) + .build(), new ParallelIndexIOConfig( new SettableSplittableLocalInputSource(inputDir, inputSourceFilter, splittableInputSource), DEFAULT_INPUT_FORMAT, @@ -972,18 +974,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv ); } else { ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DEFAULT_DIMENSIONS_SPEC, - DEFAULT_METRICS_SPEC, - new UniformGranularitySpec( - segmentGranularity, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity( + new UniformGranularitySpec( + segmentGranularity, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ) + ) + .build(), new ParallelIndexIOConfig( new LocalInputSource(inputDir, inputSourceFilter), createInputFormatFromParseSpec(DEFAULT_PARSE_SPEC), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java index 4acc3d3f591..577dce1255d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java @@ -26,7 +26,6 @@ import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; @@ -48,14 +47,11 @@ public class SinglePhaseSubTaskSpecTest "groupId", "supervisorTaskId", new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - new TimestampSpec(null, null, null), - new DimensionsSpec(null), - new AggregatorFactory[0], - null, - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.builder().build()) + .build(), new ParallelIndexIOConfig( new LocalInputSource(new File("baseDir"), "filter"), new JsonInputFormat(null, null, null, null, null), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelperTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelperTest.java index aea98e9e103..fcb92543914 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelperTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelperTest.java @@ -67,8 +67,7 @@ public class TombstoneHelperTest GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false, Collections.singletonList(interval) ); - DataSchema dataSchema = - new DataSchema("test", null, null, null, granularitySpec, null); + DataSchema dataSchema = DataSchema.builder().withDataSource("test").withGranularity(granularitySpec).build(); // no segments will be pushed when all rows are thrown away, assume that: List pushedSegments = Collections.emptyList(); @@ -93,8 +92,7 @@ public class TombstoneHelperTest GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false, Collections.singletonList(interval) ); - DataSchema dataSchema = - new DataSchema("test", null, null, null, granularitySpec, null); + DataSchema dataSchema = DataSchema.builder().withDataSource("test").withGranularity(granularitySpec).build(); // no segments will be pushed when all rows are thrown away, assume that: List pushedSegments = Collections.emptyList(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java index 44850ad0558..35aeef9715d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java @@ -122,17 +122,19 @@ public class InputRowSchemasTest extends InitializedNullHandlingTest new DoubleDimensionSchema("d5") ) ); - DataSchema schema = new DataSchema( - "dataSourceName", - new TimestampSpec(null, null, null), - dimensionsSpec, - new AggregatorFactory[]{ - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("met", "met") - }, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null - ); + DataSchema schema = + DataSchema.builder() + .withDataSource("dataSourceName") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(dimensionsSpec) + .withAggregators( + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("met", "met") + ) + .withGranularity( + new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null) + ) + .build(); InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema); Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec()); @@ -154,14 +156,12 @@ public class InputRowSchemasTest extends InitializedNullHandlingTest new DoubleDimensionSchema("d5") ) ); - DataSchema schema = new DataSchema( - "dataSourceName", - new TimestampSpec(null, null, null), - dimensionsSpec, - new AggregatorFactory[]{}, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null - ); + DataSchema schema = DataSchema.builder() + .withDataSource("dataSourceName") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(dimensionsSpec) + .withGranularity(new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null)) + .build(); InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema); Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index ddbed6be7c7..522134c4556 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -109,7 +109,6 @@ import org.apache.druid.query.DirectQueryProcessingPool; import org.apache.druid.query.ForwardingQueryProcessingPool; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9Factory; @@ -672,18 +671,19 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest null, null, new IndexIngestionSpec( - new DataSchema( - "foo", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P2D")) - ), - null - ), + DataSchema.builder() + .withDataSource("foo") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P2D")) + ) + ) + .build(), new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) @@ -735,18 +735,18 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest null, null, new IndexIngestionSpec( - new DataSchema( - "foo", - null, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P1D")) - ), - null, - mapper - ), + DataSchema.builder() + .withDataSource("foo") + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P1D")) + ) + ) + .withObjectMapper(mapper) + .build(), new IndexIOConfig(new MockExceptionInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) @@ -1165,18 +1165,19 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest null, null, new IndexIngestionSpec( - new DataSchema( - "foo", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P2D")) - ), - null - ), + DataSchema.builder() + .withDataSource("foo") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P2D")) + ) + ) + .build(), new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) @@ -1253,18 +1254,19 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest null, null, new IndexIngestionSpec( - new DataSchema( - "foo", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P2D")) - ), - null - ), + DataSchema.builder() + .withDataSource("foo") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P2D")) + ) + ) + .build(), new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java index ee90a3335a1..8f1393f2c67 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java @@ -519,7 +519,7 @@ public class TaskQueueTest extends IngestionTestBase final String password = "AbCd_1234"; final ObjectMapper mapper = getObjectMapper(); - final HttpInputSourceConfig httpInputSourceConfig = new HttpInputSourceConfig(Collections.singleton("http")); + final HttpInputSourceConfig httpInputSourceConfig = new HttpInputSourceConfig(Collections.singleton("http"), null); mapper.setInjectableValues(new InjectableValues.Std() .addValue(HttpInputSourceConfig.class, httpInputSourceConfig) .addValue(ObjectMapper.class, new DefaultObjectMapper()) @@ -549,19 +549,21 @@ public class TaskQueueTest extends IngestionTestBase new NoopTaskContextEnricher() ); - final DataSchema dataSchema = new DataSchema( - "DS", - new TimestampSpec(null, null, null), - new DimensionsSpec(null), - null, - new UniformGranularitySpec(Granularities.YEAR, Granularities.DAY, null), - null - ); + final DataSchema dataSchema = + DataSchema.builder() + .withDataSource("DS") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.builder().build()) + .withGranularity( + new UniformGranularitySpec(Granularities.YEAR, Granularities.DAY, null) + ) + .build(); final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( new HttpInputSource(Collections.singletonList(URI.create("http://host.org")), "user", new DefaultPasswordProvider(password), null, + null, httpInputSourceConfig), new NoopInputFormat(), null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java index e788545507c..d21e43ade17 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java @@ -47,14 +47,11 @@ public class CsvInputSourceSamplerTest extends InitializedNullHandlingTest { final TimestampSpec timestampSpec = new TimestampSpec(null, null, DateTimes.of("1970")); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); - final DataSchema dataSchema = new DataSchema( - "sampler", - timestampSpec, - dimensionsSpec, - null, - null, - null - ); + final DataSchema dataSchema = DataSchema.builder() + .withDataSource("sampler") + .withTimestamp(timestampSpec) + .withDimensions(dimensionsSpec) + .build(); final List strCsvRows = ImmutableList.of( "FirstName,LastName,Number,Gender", @@ -64,7 +61,7 @@ public class CsvInputSourceSamplerTest extends InitializedNullHandlingTest "Michael,Jackson,,Male" ); final InputSource inputSource = new InlineInputSource(String.join("\n", strCsvRows)); - final InputFormat inputFormat = new CsvInputFormat(null, null, null, true, 0); + final InputFormat inputFormat = new CsvInputFormat(null, null, null, true, 0, null); final InputSourceSampler inputSourceSampler = new InputSourceSampler(new DefaultObjectMapper()); final SamplerResponse response = inputSourceSampler.sample( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerDiscoveryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerDiscoveryTest.java index c486c15f0f2..0220aacd892 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerDiscoveryTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerDiscoveryTest.java @@ -66,14 +66,11 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest final SamplerResponse response = inputSourceSampler.sample( inputSource, new JsonInputFormat(null, null, null, null, null), - new DataSchema( - "test", - new TimestampSpec("t", null, null), - DimensionsSpec.builder().useSchemaDiscovery(true).build(), - null, - null, - null - ), + DataSchema.builder() + .withDataSource("test") + .withTimestamp(new TimestampSpec("t", null, null)) + .withDimensions(DimensionsSpec.builder().useSchemaDiscovery(true).build()) + .build(), null ); @@ -131,14 +128,11 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest final SamplerResponse response = inputSourceSampler.sample( inputSource, new JsonInputFormat(null, null, null, null, null), - new DataSchema( - "test", - new TimestampSpec("t", null, null), - DimensionsSpec.builder().useSchemaDiscovery(true).build(), - null, - null, - null - ), + DataSchema.builder() + .withDataSource("test") + .withTimestamp(new TimestampSpec("t", null, null)) + .withDimensions(DimensionsSpec.builder().useSchemaDiscovery(true).build()) + .build(), null ); @@ -189,14 +183,12 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest public void testTypesClassicDiscovery() { final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n')); - final DataSchema dataSchema = new DataSchema( - "test", - new TimestampSpec("t", null, null), - DimensionsSpec.builder().build(), - null, - null, - null - ); + final DataSchema dataSchema = + DataSchema.builder() + .withDataSource("test") + .withTimestamp(new TimestampSpec("t", null, null)) + .withDimensions(DimensionsSpec.builder().build()) + .build(); final SamplerResponse response = inputSourceSampler.sample( inputSource, new JsonInputFormat(null, null, null, null, null), @@ -248,23 +240,20 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest public void testTypesNoDiscoveryExplicitSchema() { final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n')); - final DataSchema dataSchema = new DataSchema( - "test", - new TimestampSpec("t", null, null), - DimensionsSpec.builder().setDimensions( - ImmutableList.of(new StringDimensionSchema("string"), - new LongDimensionSchema("long"), - new DoubleDimensionSchema("double"), - new StringDimensionSchema("bool"), - new AutoTypeColumnSchema("variant", null), - new AutoTypeColumnSchema("array", null), - new AutoTypeColumnSchema("nested", null) - ) - ).build(), - null, - null, - null - ); + final DataSchema dataSchema = + DataSchema.builder() + .withDataSource("test") + .withTimestamp(new TimestampSpec("t", null, null)) + .withDimensions( + new StringDimensionSchema("string"), + new LongDimensionSchema("long"), + new DoubleDimensionSchema("double"), + new StringDimensionSchema("bool"), + new AutoTypeColumnSchema("variant", null), + new AutoTypeColumnSchema("array", null), + new AutoTypeColumnSchema("nested", null) + ) + .build(); final SamplerResponse response = inputSourceSampler.sample( inputSource, new JsonInputFormat(null, null, null, null, null), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java index 087b12cef40..694053a4e49 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import net.thisptr.jackson.jq.internal.misc.Lists; +import com.google.common.collect.Lists; import org.apache.druid.client.indexing.SamplerResponse; import org.apache.druid.client.indexing.SamplerResponse.SamplerResponseRow; import org.apache.druid.data.input.InputFormat; @@ -1460,7 +1460,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest case STR_JSON: return new JsonInputFormat(null, null, null, null, null); case STR_CSV: - return new CsvInputFormat(ImmutableList.of("t", "dim1", "dim2", "met1"), null, null, false, 0); + return new CsvInputFormat(ImmutableList.of("t", "dim1", "dim2", "met1"), null, null, false, 0, null); default: throw new IAE("Unknown parser type: %s", parserType); } @@ -1497,24 +1497,24 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest ) throws IOException { if (useInputFormatApi) { - return new DataSchema( - "sampler", - timestampSpec, - dimensionsSpec, - aggregators, - granularitySpec, - transformSpec - ); + return DataSchema.builder() + .withDataSource("sampler") + .withTimestamp(timestampSpec) + .withDimensions(dimensionsSpec) + .withAggregators(aggregators) + .withGranularity(granularitySpec) + .withTransform(transformSpec) + .build(); } else { final Map parserMap = getParserMap(createInputRowParser(timestampSpec, dimensionsSpec)); - return new DataSchema( - "sampler", - parserMap, - aggregators, - granularitySpec, - transformSpec, - OBJECT_MAPPER - ); + return DataSchema.builder() + .withDataSource("sampler") + .withParserMap(parserMap) + .withAggregators(aggregators) + .withGranularity(granularitySpec) + .withTransform(transformSpec) + .withObjectMapper(OBJECT_MAPPER) + .build(); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSourceTest.java index 75a9bd0831b..b1176d99125 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSourceTest.java @@ -80,7 +80,7 @@ public class RecordSupplierInputSourceTest extends InitializedNullHandlingTest final List colNames = IntStream.range(0, NUM_COLS) .mapToObj(i -> StringUtils.format("col_%d", i)) .collect(Collectors.toList()); - final InputFormat inputFormat = new CsvInputFormat(colNames, null, null, false, 0); + final InputFormat inputFormat = new CsvInputFormat(colNames, null, null, false, 0, null); final InputSourceReader reader = inputSource.reader( new InputRowSchema( new TimestampSpec("col_0", "auto", null), @@ -114,7 +114,7 @@ public class RecordSupplierInputSourceTest extends InitializedNullHandlingTest final List colNames = IntStream.range(0, NUM_COLS) .mapToObj(i -> StringUtils.format("col_%d", i)) .collect(Collectors.toList()); - final InputFormat inputFormat = new CsvInputFormat(colNames, null, null, false, 0); + final InputFormat inputFormat = new CsvInputFormat(colNames, null, null, false, 0, null); final InputSourceReader reader = inputSource.reader( new InputRowSchema( new TimestampSpec("col_0", "auto", null), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java index 0f280059e0a..8b1bd4fb096 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java @@ -33,10 +33,8 @@ import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.AllGranularity; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; -import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthConfig; @@ -107,16 +105,13 @@ public class SeekableStreamIndexTaskRunnerAuthTest } }; - DataSchema dataSchema = new DataSchema( - "datasource", - new TimestampSpec(null, null, null), - new DimensionsSpec(Collections.emptyList()), - new AggregatorFactory[]{}, - new ArbitraryGranularitySpec(new AllGranularity(), Collections.emptyList()), - TransformSpec.NONE, - null, - null - ); + DataSchema dataSchema = + DataSchema.builder() + .withDataSource("datasource") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(new DimensionsSpec(Collections.emptyList())) + .withGranularity(new ArbitraryGranularitySpec(new AllGranularity(), Collections.emptyList())) + .build(); SeekableStreamIndexTaskTuningConfig tuningConfig = mock(SeekableStreamIndexTaskTuningConfig.class); SeekableStreamIndexTaskIOConfig ioConfig = new TestSeekableStreamIndexTaskIOConfig(); @@ -385,7 +380,7 @@ public class SeekableStreamIndexTaskRunnerAuthTest false, DateTimes.nowUtc().minusDays(2), DateTimes.nowUtc(), - new CsvInputFormat(null, null, true, null, 0) + new CsvInputFormat(null, null, true, null, 0, null) ); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index 258ebff7b50..7346fe9fb6e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -95,7 +95,6 @@ import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -164,25 +163,23 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport protected static final ObjectMapper OBJECT_MAPPER; protected static final DataSchema OLD_DATA_SCHEMA; - protected static final DataSchema NEW_DATA_SCHEMA = new DataSchema( - "test_ds", - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null - ); + protected static final DataSchema NEW_DATA_SCHEMA = + DataSchema.builder() + .withDataSource("test_ds") + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .build(); protected static final InputFormat INPUT_FORMAT = new JsonInputFormat( new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of(), @@ -211,37 +208,38 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport static { OBJECT_MAPPER = new TestUtils().getTestObjectMapper(); OBJECT_MAPPER.registerSubtypes(new NamedType(JSONParseSpec.class, "json")); - OLD_DATA_SCHEMA = new DataSchema( - "test_ds", - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of(), - false - ), - StandardCharsets.UTF_8.name() - ), - Map.class - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - OBJECT_MAPPER - ); + OLD_DATA_SCHEMA = DataSchema.builder() + .withDataSource("test_ds") + .withParserMap( + OBJECT_MAPPER.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ) + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of(), + false + ), + StandardCharsets.UTF_8.name() + ), + Map.class + ) + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .withObjectMapper(OBJECT_MAPPER) + .build(); } public SeekableStreamIndexTaskTestBase( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java index 87cd196c268..6510e2cfbdc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java @@ -48,7 +48,6 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; @@ -106,36 +105,37 @@ public class SeekableStreamSamplerSpecTest extends EasyMockSupport @Test(timeout = 10_000L) public void testSampleWithInputRowParser() throws Exception { - final DataSchema dataSchema = new DataSchema( - "test_ds", - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of(), - false - ) - ), - Map.class - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - OBJECT_MAPPER - ); + DataSchema dataSchema = DataSchema.builder() + .withDataSource("test_ds") + .withParserMap( + OBJECT_MAPPER.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ) + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of(), + false + ) + ), + Map.class + ) + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .withObjectMapper(OBJECT_MAPPER) + .build(); final SeekableStreamSupervisorIOConfig supervisorIOConfig = new TestableSeekableStreamSupervisorIOConfig( STREAM, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java index 3e0e46d7a03..baff5fc765b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java @@ -25,7 +25,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; @@ -59,7 +58,6 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.query.DruidMetrics; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -522,6 +520,7 @@ public class SeekableStreamSupervisorSpecTest extends EasyMockSupport null, null, null, + null, null ) { @@ -1260,18 +1259,19 @@ public class SeekableStreamSupervisorSpecTest extends EasyMockSupport dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim2")); - return new DataSchema( - DATASOURCE, - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(dimensions), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.NONE, - ImmutableList.of() - ), - null - ); + return DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions(dimensions) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity( + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of() + ) + ) + .build(); } private SeekableStreamSupervisorIOConfig getIOConfig(int taskCount, boolean scaleOut) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index daf85ac39c9..40bbe84b623 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -31,7 +31,6 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; @@ -83,7 +82,6 @@ import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.metadata.PendingSegmentRecord; import org.apache.druid.query.DruidMetrics; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -2562,18 +2560,19 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim2")); - return new DataSchema( - DATASOURCE, - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(dimensions), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.NONE, - ImmutableList.of() - ), - null - ); + return DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions(dimensions) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity( + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of() + ) + ) + .build(); } private static SeekableStreamSupervisorIOConfig getIOConfig() @@ -2683,6 +2682,7 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport null, null, null, + null, null ) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java index 88249509ef9..61396fc7ae6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java @@ -28,7 +28,6 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.junit.Assert; import org.junit.Test; @@ -51,7 +50,7 @@ public class TaskAnnouncementTest "theid", new TaskResource("rofl", 2), new IndexTask.IndexIngestionSpec( - new DataSchema("foo", null, new AggregatorFactory[0], null, null, new DefaultObjectMapper()), + DataSchema.builder().withDataSource("foo").withObjectMapper(new DefaultObjectMapper()).build(), ioConfig, null ), diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index 433608bbe24..3e8bf724d82 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -31,7 +31,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/integration-tests-ex/image/pom.xml b/integration-tests-ex/image/pom.xml index 600501edb92..b5b7885cac1 100644 --- a/integration-tests-ex/image/pom.xml +++ b/integration-tests-ex/image/pom.xml @@ -46,7 +46,7 @@ Reference: https://dzone.com/articles/build-docker-image-from-maven druid org.apache.druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/integration-tests-ex/tools/pom.xml b/integration-tests-ex/tools/pom.xml index c3715f4adb3..3f193a9f9c7 100644 --- a/integration-tests-ex/tools/pom.xml +++ b/integration-tests-ex/tools/pom.xml @@ -31,7 +31,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT ../../pom.xml diff --git a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/ServerManagerForQueryErrorTest.java b/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/ServerManagerForQueryErrorTest.java index 60a057ece79..d5b82e9c8d5 100644 --- a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/ServerManagerForQueryErrorTest.java +++ b/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/ServerManagerForQueryErrorTest.java @@ -47,7 +47,6 @@ import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.ServerManager; import org.apache.druid.server.initialization.ServerConfig; @@ -98,7 +97,6 @@ public class ServerManagerForQueryErrorTest extends ServerManager Cache cache, CacheConfig cacheConfig, SegmentManager segmentManager, - JoinableFactoryWrapper joinableFactoryWrapper, ServerConfig serverConfig ) { @@ -111,7 +109,6 @@ public class ServerManagerForQueryErrorTest extends ServerManager cache, cacheConfig, segmentManager, - joinableFactoryWrapper, serverConfig ); } diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index e93fb7464e3..5320d0c2451 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT diff --git a/integration-tests/src/main/java/org/apache/druid/server/coordination/ServerManagerForQueryErrorTest.java b/integration-tests/src/main/java/org/apache/druid/server/coordination/ServerManagerForQueryErrorTest.java index 7b434667fa9..8bfceb54470 100644 --- a/integration-tests/src/main/java/org/apache/druid/server/coordination/ServerManagerForQueryErrorTest.java +++ b/integration-tests/src/main/java/org/apache/druid/server/coordination/ServerManagerForQueryErrorTest.java @@ -47,7 +47,6 @@ import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -96,7 +95,6 @@ public class ServerManagerForQueryErrorTest extends ServerManager Cache cache, CacheConfig cacheConfig, SegmentManager segmentManager, - JoinableFactoryWrapper joinableFactoryWrapper, ServerConfig serverConfig ) { @@ -109,7 +107,6 @@ public class ServerManagerForQueryErrorTest extends ServerManager cache, cacheConfig, segmentManager, - joinableFactoryWrapper, serverConfig ); } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index e95d09bd508..a7bae1b30b9 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -692,16 +692,26 @@ public class ITAutoCompactionTest extends AbstractIndexerTest LOG.info("Auto compaction test with range partitioning"); - final DimensionRangePartitionsSpec rangePartitionsSpec = new DimensionRangePartitionsSpec( + final DimensionRangePartitionsSpec inputRangePartitionsSpec = new DimensionRangePartitionsSpec( 5, null, ImmutableList.of("city"), false ); - submitCompactionConfig(rangePartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false, engine); + DimensionRangePartitionsSpec expectedRangePartitionsSpec = inputRangePartitionsSpec; + if (engine == CompactionEngine.MSQ) { + // Range spec is transformed to its effective maxRowsPerSegment equivalent in MSQ + expectedRangePartitionsSpec = new DimensionRangePartitionsSpec( + null, + 7, + ImmutableList.of("city"), + false + ); + } + submitCompactionConfig(inputRangePartitionsSpec, NO_SKIP_OFFSET, 1, null, null, null, null, false, engine); forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); - verifySegmentsCompacted(rangePartitionsSpec, 2); + verifySegmentsCompacted(expectedRangePartitionsSpec, 2); checkCompactionIntervals(intervalsBeforeCompaction); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java index 1312685fe5a..53deb831a2d 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITBestEffortRollupParallelIndexTest.java @@ -38,7 +38,7 @@ import org.testng.annotations.Test; import java.io.Closeable; import java.util.function.Function; -@Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED}) +@Test(groups = {TestNGGroup.BATCH_INDEX}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITBestEffortRollupParallelIndexTest extends AbstractITBatchIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java index 2af07e017b9..7619bcf8b6e 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCombiningInputSourceParallelIndexTest.java @@ -32,7 +32,7 @@ import java.io.Closeable; import java.util.Map; import java.util.function.Function; -@Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.QUICKSTART_COMPATIBLE, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED}) +@Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.QUICKSTART_COMPATIBLE}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITCombiningInputSourceParallelIndexTest extends AbstractITBatchIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHttpInputSourceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHttpInputSourceTest.java index 25eb83a7c28..dc8f9a92a7d 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHttpInputSourceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITHttpInputSourceTest.java @@ -29,7 +29,7 @@ import java.io.Closeable; import java.io.IOException; import java.util.UUID; -@Test(groups = {TestNGGroup.INPUT_SOURCE, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED}) +@Test(groups = {TestNGGroup.INPUT_SOURCE}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITHttpInputSourceTest extends AbstractITBatchIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java index 5df5a708bc5..88510139d91 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceNonTransactionalSerializedTest.java @@ -25,7 +25,7 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Guice; import org.testng.annotations.Test; -@Test(groups = {TestNGGroup.KAFKA_INDEX_SLOW, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED}) +@Test(groups = {TestNGGroup.KAFKA_INDEX_SLOW}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITKafkaIndexingServiceNonTransactionalSerializedTest extends AbstractKafkaIndexingServiceTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java index 424d3c67068..fdd06ff4f88 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalSerializedTest.java @@ -25,7 +25,7 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Guice; import org.testng.annotations.Test; -@Test(groups = {TestNGGroup.TRANSACTIONAL_KAFKA_INDEX_SLOW, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED}) +@Test(groups = TestNGGroup.TRANSACTIONAL_KAFKA_INDEX_SLOW) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITKafkaIndexingServiceTransactionalSerializedTest extends AbstractKafkaIndexingServiceTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITLocalInputSourceAllInputFormatTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITLocalInputSourceAllInputFormatTest.java index 5e7678e8b6b..c3c87ac13f6 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITLocalInputSourceAllInputFormatTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITLocalInputSourceAllInputFormatTest.java @@ -30,7 +30,7 @@ import org.testng.annotations.Test; import java.util.List; import java.util.Map; -@Test(groups = {TestNGGroup.INPUT_FORMAT, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED}) +@Test(groups = {TestNGGroup.INPUT_FORMAT}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITLocalInputSourceAllInputFormatTest extends AbstractLocalInputSourceParallelIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java index 5695156fa9b..78823477434 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITPerfectRollupParallelIndexTest.java @@ -36,7 +36,7 @@ import org.testng.annotations.Test; import java.io.Closeable; import java.util.function.Function; -@Test(groups = {TestNGGroup.PERFECT_ROLLUP_PARALLEL_BATCH_INDEX, TestNGGroup.SHUFFLE_DEEP_STORE, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED}) +@Test(groups = {TestNGGroup.PERFECT_ROLLUP_PARALLEL_BATCH_INDEX, TestNGGroup.SHUFFLE_DEEP_STORE}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITPerfectRollupParallelIndexTest extends AbstractITBatchIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSystemTableBatchIndexTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSystemTableBatchIndexTaskTest.java index f98626ad9b5..3681a381b92 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSystemTableBatchIndexTaskTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITSystemTableBatchIndexTaskTest.java @@ -30,7 +30,7 @@ import org.testng.annotations.Test; import java.io.Closeable; import java.util.function.Function; -@Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED}) +@Test(groups = {TestNGGroup.BATCH_INDEX}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITSystemTableBatchIndexTaskTest extends AbstractITBatchIndexTest { diff --git a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceDataFormatTest.java b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceDataFormatTest.java index 2286d237855..c1fb5612272 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceDataFormatTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/parallelized/ITKafkaIndexingServiceDataFormatTest.java @@ -38,7 +38,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -@Test(groups = TestNGGroup.KAFKA_DATA_FORMAT) +@Test(groups = {TestNGGroup.KAFKA_DATA_FORMAT, TestNGGroup.CDS_TASK_SCHEMA_PUBLISH_DISABLED, TestNGGroup.CDS_COORDINATOR_METADATA_QUERY_DISABLED}) @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITKafkaIndexingServiceDataFormatTest extends AbstractKafkaIndexingServiceTest { diff --git a/licenses.yaml b/licenses.yaml index 304880db3db..0ecffdb0a0a 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -809,7 +809,7 @@ name: com.nimbusds nimbus-jose-jwt license_category: binary module: extensions/druid-pac4j license_name: Apache License version 2.0 -version: 9.37.2 +version: 8.22.1 libraries: - com.nimbusds: nimbus-jose-jwt @@ -1837,33 +1837,38 @@ name: Apache Maven license_category: binary module: java-core license_name: Apache License version 2.0 -version: 3.1.1 +version: 3.6.0 libraries: - - org.apache.maven: maven-aether-provider - - org.apache.maven: maven-model - - org.apache.maven: maven-model-builder - org.apache.maven: maven-repository-metadata - - org.apache.maven: maven-settings - - org.apache.maven: maven-settings-builder + - org.apache.maven: maven-builder-support notices: - - maven-aether-provider: | - Maven Aether Provider - Copyright 2001-2013 The Apache Software Foundation - - maven-model: | - Maven Model - Copyright 2001-2013 The Apache Software Foundation - - maven-model-builder: | - Maven Model Builder - Copyright 2001-2013 The Apache Software Foundation - maven-repository-metadata: | Maven Repository Metadata Model - Copyright 2001-2013 The Apache Software Foundation - - maven-settings: | - Maven Settings - Copyright 2001-2013 The Apache Software Foundation - - maven-settings-builder: | - Maven Settings Builder - Copyright 2001-2013 The Apache Software Foundation + Copyright 2001-2018 The Apache Software Foundation + - maven-builder-support: | + Maven Builder Support + Copyright 2001-2018 The Apache Software Foundation +--- + +name: Maven Artifact Resolver Provider +license_category: binary +module: java-core +license_name: Apache License version 2.0 +version: 3.6.0 +libraries: + - org.apache.maven: maven-resolver-provider + - org.apache.maven: maven-model + - org.apache.maven: maven-model-builder +notices: + - maven-resolver-provider: | + Maven Artifact Resolver Provider + Copyright 2001-2018 The Apache Software Foundation + - maven-model: | + Maven Model + Copyright 2001-2018 The Apache Software Foundation + - maven-model-builder: | + Maven Model Builder + Copyright 2001-2018 The Apache Software Foundation --- name: Apache Maven Artifact @@ -1879,6 +1884,67 @@ notices: Copyright 2001-2018 The Apache Software Foundation --- +name: Maven Artifact Resolver Connector Basic +license_category: binary +module: java-core +license_name: Apache License version 2.0 +version: 1.3.1 +libraries: + - org.apache.maven.resolver: maven-resolver-connector-basic + - org.apache.maven.resolver: maven-resolver-spi + - org.apache.maven.resolver: maven-resolver-api + - org.apache.maven.resolver: maven-resolver-util +notices: + - maven-resolver-connector-basic: | + Maven Artifact Resolver Connector Basic + Copyright 2001-2018 The Apache Software Foundation + - maven-resolver-spi: | + Maven Artifact Resolver SPI + Copyright 2001-2018 The Apache Software Foundation + - maven-resolver-api: | + Maven Artifact Resolver API + Copyright 2001-2018 The Apache Software Foundation + - maven-resolver-util: | + Maven Artifact Resolver Utilities + Copyright 2001-2018 The Apache Software Foundation +--- + +name: Maven Artifact Resolver Transport HTTP +license_category: binary +module: java-core +license_name: Apache License version 2.0 +version: 1.3.1 +libraries: + - org.apache.maven.resolver: maven-resolver-transport-http +notices: + - maven-resolver-transport-http: | + Maven Artifact Resolver Transport HTTP + Copyright 2001-2018 The Apache Software Foundation + +--- + +name: Maven Artifact Resolver Implementation +license_category: binary +module: java-core +license_name: Apache License version 2.0 +version: 1.3.1 +libraries: + - org.apache.maven.resolver: maven-resolver-impl +notices: + - maven-resolver-impl: | + Maven Artifact Resolver Implementation + Copyright 2001-2018 The Apache Software Foundation +--- + +name: Plexus Component Annotations +license_category: binary +module: java-core +license_name: Apache License version 2.0 +version: 1.7.1 +libraries: + - org.codehaus.plexus: plexus-component-annotations +--- + name: Apache Maven Wagon API license_category: binary module: java-core @@ -1967,7 +2033,7 @@ name: Plexus Interpolation API license_category: binary module: java-core license_name: Apache License version 2.0 -version: 1.19 +version: 1.25 libraries: - org.codehaus.plexus: plexus-interpolation @@ -3245,7 +3311,7 @@ name: JCodings license_category: binary module: java-core license_name: MIT License -version: 1.0.43 +version: 1.0.50 copyright: JRuby Team license_file_path: licenses/bin/jcodings.MIT libraries: @@ -3257,7 +3323,7 @@ name: Joni license_category: binary module: java-core license_name: MIT License -version: 2.1.27 +version: 2.1.34 copyright: JRuby Team license_file_path: licenses/bin/joni.MIT libraries: diff --git a/pom.xml b/pom.xml index 11738830512..2a5d6d90212 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT pom Druid @@ -231,11 +231,11 @@ extensions-contrib/prometheus-emitter extensions-contrib/opentelemetry-emitter extensions-contrib/kubernetes-overlord-extensions + extensions-contrib/grpc-query extensions-contrib/druid-iceberg-extensions extensions-contrib/druid-deltalake-extensions extensions-contrib/spectator-histogram extensions-contrib/rabbit-stream-indexing-service - distribution @@ -1363,6 +1363,11 @@ + + org.jruby.joni + joni + 2.1.34 + @@ -1765,8 +1770,6 @@ @{jacocoArgLine} ${jdk.strong.encapsulation.argLine} ${jdk.security.manager.allow.argLine} - -Xmx2048m -XX:MaxDirectMemorySize=2500m -XX:+ExitOnOutOfMemoryError @@ -1778,6 +1781,8 @@ -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager -Daws.region=us-east-1 -Ddruid.test.stupidPool.poison=true + -XX:OnOutOfMemoryError=${project.parent.basedir}/dev/chmod-heap-dumps.sh + -XX:HeapDumpPath=${project.parent.basedir}/target -Ddruid.indexing.doubleStorage=double ${jfrProfilerArgLine} diff --git a/processing/pom.xml b/processing/pom.xml index bfb39cf2f77..7ae4cc075a2 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT diff --git a/processing/src/main/java/org/apache/druid/collections/QueueNonBlockingPool.java b/processing/src/main/java/org/apache/druid/collections/QueueNonBlockingPool.java new file mode 100644 index 00000000000..1c3309c958f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/collections/QueueNonBlockingPool.java @@ -0,0 +1,48 @@ +/* + * 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.collections; + +import java.util.NoSuchElementException; +import java.util.concurrent.BlockingQueue; + +/** + * Implementation of {@link NonBlockingPool} based on a pre-created {@link BlockingQueue} that never actually blocks. + * If the pool is empty when {@link #take()} is called, it throws {@link NoSuchElementException}. + */ +public class QueueNonBlockingPool implements NonBlockingPool +{ + private final BlockingQueue queue; + + public QueueNonBlockingPool(final BlockingQueue queue) + { + this.queue = queue; + } + + @Override + public ResourceHolder take() + { + final T item = queue.poll(); + if (item == null) { + throw new NoSuchElementException("No items available"); + } + + return new ReferenceCountingResourceHolder<>(item, () -> queue.add(item)); + } +} diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java b/processing/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java index 81c8a26ecd3..2a5adfe0a4b 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/CSVParseSpec.java @@ -93,7 +93,7 @@ public class CSVParseSpec extends ParseSpec @Override public Parser makeParser() { - return new CSVParser(listDelimiter, columns, hasHeaderRow, skipHeaderRows); + return new CSVParser(listDelimiter, columns, hasHeaderRow, skipHeaderRows, false); } @Override diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java b/processing/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java index a041e031a3b..023e209b8b9 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java @@ -47,10 +47,11 @@ public class CsvInputFormat extends FlatTextInputFormat @JsonProperty("listDelimiter") @Nullable String listDelimiter, @Deprecated @JsonProperty("hasHeaderRow") @Nullable Boolean hasHeaderRow, @JsonProperty("findColumnsFromHeader") @Nullable Boolean findColumnsFromHeader, - @JsonProperty("skipHeaderRows") int skipHeaderRows + @JsonProperty("skipHeaderRows") int skipHeaderRows, + @JsonProperty("tryParseNumbers") @Nullable Boolean tryParseNumbers ) { - super(columns, listDelimiter, String.valueOf(SEPARATOR), hasHeaderRow, findColumnsFromHeader, skipHeaderRows); + super(columns, listDelimiter, String.valueOf(SEPARATOR), hasHeaderRow, findColumnsFromHeader, skipHeaderRows, tryParseNumbers); } @Override @@ -80,7 +81,8 @@ public class CsvInputFormat extends FlatTextInputFormat isFindColumnsFromHeader(), getSkipHeaderRows(), line -> Arrays.asList(parser.parseLine(StringUtils.fromUtf8(line))), - useListBasedInputRows() + useListBasedInputRows(), + shouldTryParseNumbers() ); } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java b/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java index 238a2f4dc2c..06590584b86 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java @@ -52,7 +52,8 @@ public class DelimitedInputFormat extends FlatTextInputFormat @JsonProperty("delimiter") @Nullable String delimiter, @Deprecated @JsonProperty("hasHeaderRow") @Nullable Boolean hasHeaderRow, @JsonProperty("findColumnsFromHeader") @Nullable Boolean findColumnsFromHeader, - @JsonProperty("skipHeaderRows") int skipHeaderRows + @JsonProperty("skipHeaderRows") int skipHeaderRows, + @JsonProperty("tryParseNumbers") @Nullable Boolean tryParseNumbers ) { super( @@ -61,7 +62,8 @@ public class DelimitedInputFormat extends FlatTextInputFormat delimiter == null ? DEFAULT_DELIMITER : delimiter, hasHeaderRow, findColumnsFromHeader, - skipHeaderRows + skipHeaderRows, + tryParseNumbers ); } @@ -85,7 +87,8 @@ public class DelimitedInputFormat extends FlatTextInputFormat getDelimiter(), useListBasedInputRows() ? getColumns().size() : DelimitedBytes.UNKNOWN_FIELD_COUNT ), - useListBasedInputRows() + useListBasedInputRows(), + shouldTryParseNumbers() ); } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedParseSpec.java b/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedParseSpec.java index 5940e70e11f..3327989e9f5 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedParseSpec.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedParseSpec.java @@ -69,18 +69,6 @@ public class DelimitedParseSpec extends ParseSpec } } - @Deprecated - public DelimitedParseSpec( - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, - String delimiter, - String listDelimiter, - List columns - ) - { - this(timestampSpec, dimensionsSpec, delimiter, listDelimiter, columns, false, 0); - } - @JsonProperty("delimiter") public String getDelimiter() { @@ -119,7 +107,8 @@ public class DelimitedParseSpec extends ParseSpec listDelimiter, columns, hasHeaderRow, - skipHeaderRows + skipHeaderRows, + false ); } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedValueReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedValueReader.java index 227edffbcb0..01d2565993f 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedValueReader.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedValueReader.java @@ -51,7 +51,7 @@ public class DelimitedValueReader extends TextReader.Bytes { private final boolean findColumnsFromHeader; private final int skipHeaderRows; - private final Function multiValueFunction; + private final Function transformationFunction; private final DelimitedValueParser parser; /** @@ -82,14 +82,19 @@ public class DelimitedValueReader extends TextReader.Bytes boolean findColumnsFromHeader, int skipHeaderRows, DelimitedValueParser parser, - boolean useListBasedInputRows + boolean useListBasedInputRows, + boolean tryParseNumbers ) { super(inputRowSchema, source); this.findColumnsFromHeader = findColumnsFromHeader; this.skipHeaderRows = skipHeaderRows; final String finalListDelimeter = listDelimiter == null ? Parsers.DEFAULT_LIST_DELIMITER : listDelimiter; - this.multiValueFunction = ParserUtils.getMultiValueFunction(finalListDelimeter, Splitter.on(finalListDelimeter)); + this.transformationFunction = ParserUtils.getTransformationFunction( + finalListDelimeter, + Splitter.on(finalListDelimeter), + tryParseNumbers + ); if (!findColumnsFromHeader && columns != null) { // If findColumnsFromHeader, inputRowSignature will be set later. @@ -134,7 +139,7 @@ public class DelimitedValueReader extends TextReader.Bytes private List readLineAsList(byte[] line) throws IOException { final List parsed = parser.parseLine(line); - return new ArrayList<>(Lists.transform(parsed, multiValueFunction)); + return new ArrayList<>(Lists.transform(parsed, transformationFunction)); } private Map readLineAsMap(byte[] line) throws IOException @@ -142,7 +147,7 @@ public class DelimitedValueReader extends TextReader.Bytes final List parsed = parser.parseLine(line); return Utils.zipMapPartial( Preconditions.checkNotNull(inputRowSignature, "inputRowSignature").getColumnNames(), - Iterables.transform(parsed, multiValueFunction) + Iterables.transform(parsed, transformationFunction) ); } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/FlatTextInputFormat.java b/processing/src/main/java/org/apache/druid/data/input/impl/FlatTextInputFormat.java index 8e8e052cd00..39086b6838e 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/FlatTextInputFormat.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/FlatTextInputFormat.java @@ -46,6 +46,7 @@ public abstract class FlatTextInputFormat implements InputFormat private final String delimiter; private final boolean findColumnsFromHeader; private final int skipHeaderRows; + private final boolean tryParseNumbers; FlatTextInputFormat( @Nullable List columns, @@ -53,7 +54,8 @@ public abstract class FlatTextInputFormat implements InputFormat String delimiter, @Nullable Boolean hasHeaderRow, @Nullable Boolean findColumnsFromHeader, - int skipHeaderRows + int skipHeaderRows, + @Nullable Boolean tryParseNumbers ) { this.columns = columns == null ? Collections.emptyList() : columns; @@ -79,6 +81,8 @@ public abstract class FlatTextInputFormat implements InputFormat "Cannot have same delimiter and list delimiter of [%s]", delimiter ); + this.tryParseNumbers = tryParseNumbers == null ? false : tryParseNumbers; + if (!this.columns.isEmpty()) { for (String column : this.columns) { Preconditions.checkArgument( @@ -131,6 +135,13 @@ public abstract class FlatTextInputFormat implements InputFormat return skipHeaderRows; } + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public boolean shouldTryParseNumbers() + { + return tryParseNumbers; + } + @Override public long getWeightedSize(String path, long size) { diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java b/processing/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java index e03495ce02a..b0c415d322b 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/HttpEntity.java @@ -34,6 +34,7 @@ import java.io.InputStream; import java.net.URI; import java.net.URLConnection; import java.util.Base64; +import java.util.Map; public class HttpEntity extends RetryingInputEntity { @@ -45,15 +46,19 @@ public class HttpEntity extends RetryingInputEntity @Nullable private final PasswordProvider httpAuthenticationPasswordProvider; + private final Map requestHeaders; + HttpEntity( URI uri, @Nullable String httpAuthenticationUsername, - @Nullable PasswordProvider httpAuthenticationPasswordProvider + @Nullable PasswordProvider httpAuthenticationPasswordProvider, + @Nullable Map requestHeaders ) { this.uri = uri; this.httpAuthenticationUsername = httpAuthenticationUsername; this.httpAuthenticationPasswordProvider = httpAuthenticationPasswordProvider; + this.requestHeaders = requestHeaders; } @Override @@ -65,7 +70,7 @@ public class HttpEntity extends RetryingInputEntity @Override protected InputStream readFrom(long offset) throws IOException { - return openInputStream(uri, httpAuthenticationUsername, httpAuthenticationPasswordProvider, offset); + return openInputStream(uri, httpAuthenticationUsername, httpAuthenticationPasswordProvider, offset, requestHeaders); } @Override @@ -80,10 +85,15 @@ public class HttpEntity extends RetryingInputEntity return t -> t instanceof IOException; } - public static InputStream openInputStream(URI object, String userName, PasswordProvider passwordProvider, long offset) + public static InputStream openInputStream(URI object, String userName, PasswordProvider passwordProvider, long offset, final Map requestHeaders) throws IOException { final URLConnection urlConnection = object.toURL().openConnection(); + if (requestHeaders != null && requestHeaders.size() > 0) { + for (Map.Entry entry : requestHeaders.entrySet()) { + urlConnection.addRequestProperty(entry.getKey(), entry.getValue()); + } + } if (!Strings.isNullOrEmpty(userName) && passwordProvider != null) { String userPass = userName + ":" + passwordProvider.getPassword(); String basicAuthString = "Basic " + Base64.getEncoder().encodeToString(StringUtils.toUtf8(userPass)); diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java b/processing/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java index 0ef8194f1e1..0c4c9197e5f 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java @@ -36,6 +36,7 @@ import org.apache.druid.data.input.impl.systemfield.SystemField; import org.apache.druid.data.input.impl.systemfield.SystemFieldDecoratorFactory; import org.apache.druid.data.input.impl.systemfield.SystemFieldInputSource; import org.apache.druid.data.input.impl.systemfield.SystemFields; +import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; @@ -47,6 +48,7 @@ import java.io.File; import java.net.URI; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.stream.Stream; @@ -64,6 +66,7 @@ public class HttpInputSource private final PasswordProvider httpAuthenticationPasswordProvider; private final SystemFields systemFields; private final HttpInputSourceConfig config; + private final Map requestHeaders; @JsonCreator public HttpInputSource( @@ -71,6 +74,7 @@ public class HttpInputSource @JsonProperty("httpAuthenticationUsername") @Nullable String httpAuthenticationUsername, @JsonProperty("httpAuthenticationPassword") @Nullable PasswordProvider httpAuthenticationPasswordProvider, @JsonProperty(SYSTEM_FIELDS_PROPERTY) @Nullable SystemFields systemFields, + @JsonProperty("requestHeaders") @Nullable Map requestHeaders, @JacksonInject HttpInputSourceConfig config ) { @@ -80,17 +84,11 @@ public class HttpInputSource this.httpAuthenticationUsername = httpAuthenticationUsername; this.httpAuthenticationPasswordProvider = httpAuthenticationPasswordProvider; this.systemFields = systemFields == null ? SystemFields.none() : systemFields; + this.requestHeaders = requestHeaders == null ? Collections.emptyMap() : requestHeaders; + throwIfForbiddenHeaders(config, this.requestHeaders); this.config = config; } - @JsonIgnore - @Nonnull - @Override - public Set getTypes() - { - return Collections.singleton(TYPE_KEY); - } - public static void throwIfInvalidProtocols(HttpInputSourceConfig config, List uris) { for (URI uri : uris) { @@ -100,6 +98,25 @@ public class HttpInputSource } } + public static void throwIfForbiddenHeaders(HttpInputSourceConfig config, Map requestHeaders) + { + for (Map.Entry entry : requestHeaders.entrySet()) { + if (!config.getAllowedHeaders().contains(StringUtils.toLowerCase(entry.getKey()))) { + throw InvalidInput.exception("Got forbidden header [%s], allowed headers are only [%s]. You can control the allowed headers by updating druid.ingestion.http.allowedHeaders", + entry.getKey(), config.getAllowedHeaders() + ); + } + } + } + + @JsonIgnore + @Nonnull + @Override + public Set getTypes() + { + return Collections.singleton(TYPE_KEY); + } + @JsonProperty public List getUris() { @@ -128,6 +145,14 @@ public class HttpInputSource return httpAuthenticationPasswordProvider; } + @Nullable + @JsonProperty("requestHeaders") + @JsonInclude(JsonInclude.Include.NON_NULL) + public Map getRequestHeaders() + { + return requestHeaders; + } + @Override public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) { @@ -148,6 +173,7 @@ public class HttpInputSource httpAuthenticationUsername, httpAuthenticationPasswordProvider, systemFields, + requestHeaders, config ); } @@ -181,7 +207,8 @@ public class HttpInputSource createSplits(inputFormat, null).map(split -> new HttpEntity( split.get(), httpAuthenticationUsername, - httpAuthenticationPasswordProvider + httpAuthenticationPasswordProvider, + requestHeaders )).iterator() ), SystemFieldDecoratorFactory.fromInputSource(this), @@ -203,13 +230,21 @@ public class HttpInputSource && Objects.equals(httpAuthenticationUsername, that.httpAuthenticationUsername) && Objects.equals(httpAuthenticationPasswordProvider, that.httpAuthenticationPasswordProvider) && Objects.equals(systemFields, that.systemFields) + && Objects.equals(requestHeaders, that.requestHeaders) && Objects.equals(config, that.config); } @Override public int hashCode() { - return Objects.hash(uris, httpAuthenticationUsername, httpAuthenticationPasswordProvider, systemFields, config); + return Objects.hash( + uris, + httpAuthenticationUsername, + httpAuthenticationPasswordProvider, + systemFields, + requestHeaders, + config + ); } @Override @@ -226,6 +261,7 @@ public class HttpInputSource ", httpAuthenticationUsername=" + httpAuthenticationUsername + ", httpAuthenticationPasswordProvider=" + httpAuthenticationPasswordProvider + (systemFields.getFields().isEmpty() ? "" : ", systemFields=" + systemFields) + + ", requestHeaders = " + requestHeaders + "}"; } } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/HttpInputSourceConfig.java b/processing/src/main/java/org/apache/druid/data/input/impl/HttpInputSourceConfig.java index 310c6690461..1299edbc574 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/HttpInputSourceConfig.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/HttpInputSourceConfig.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.java.util.common.StringUtils; import javax.annotation.Nullable; +import java.util.Collections; import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -38,14 +39,21 @@ public class HttpInputSourceConfig @JsonProperty private final Set allowedProtocols; + @JsonProperty + private final Set allowedHeaders; + @JsonCreator public HttpInputSourceConfig( - @JsonProperty("allowedProtocols") @Nullable Set allowedProtocols + @JsonProperty("allowedProtocols") @Nullable Set allowedProtocols, + @JsonProperty("allowedHeaders") @Nullable Set allowedHeaders ) { this.allowedProtocols = allowedProtocols == null || allowedProtocols.isEmpty() ? DEFAULT_ALLOWED_PROTOCOLS : allowedProtocols.stream().map(StringUtils::toLowerCase).collect(Collectors.toSet()); + this.allowedHeaders = allowedHeaders == null + ? Collections.emptySet() + : allowedHeaders.stream().map(StringUtils::toLowerCase).collect(Collectors.toSet()); } public Set getAllowedProtocols() @@ -53,6 +61,11 @@ public class HttpInputSourceConfig return allowedProtocols; } + public Set getAllowedHeaders() + { + return allowedHeaders; + } + @Override public boolean equals(Object o) { @@ -63,13 +76,16 @@ public class HttpInputSourceConfig return false; } HttpInputSourceConfig that = (HttpInputSourceConfig) o; - return Objects.equals(allowedProtocols, that.allowedProtocols); + return Objects.equals(allowedProtocols, that.allowedProtocols) && Objects.equals( + allowedHeaders, + that.allowedHeaders + ); } @Override public int hashCode() { - return Objects.hash(allowedProtocols); + return Objects.hash(allowedProtocols, allowedHeaders); } @Override @@ -77,6 +93,7 @@ public class HttpInputSourceConfig { return "HttpInputSourceConfig{" + "allowedProtocols=" + allowedProtocols + + ", allowedHeaders=" + allowedHeaders + '}'; } } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/RegexReader.java b/processing/src/main/java/org/apache/druid/data/input/impl/RegexReader.java index aabf4fe1603..66f4d25b474 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/RegexReader.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/RegexReader.java @@ -43,7 +43,7 @@ public class RegexReader extends TextReader.Strings { private final String pattern; private final Pattern compiledPattern; - private final Function multiValueFunction; + private final Function transformationFunction; private List columns; @@ -60,7 +60,11 @@ public class RegexReader extends TextReader.Strings this.pattern = pattern; this.compiledPattern = compiledPattern; final String finalListDelimeter = listDelimiter == null ? Parsers.DEFAULT_LIST_DELIMITER : listDelimiter; - this.multiValueFunction = ParserUtils.getMultiValueFunction(finalListDelimeter, Splitter.on(finalListDelimeter)); + this.transformationFunction = ParserUtils.getTransformationFunction( + finalListDelimeter, + Splitter.on(finalListDelimeter), + false + ); this.columns = columns; } @@ -94,7 +98,7 @@ public class RegexReader extends TextReader.Strings columns = ParserUtils.generateFieldNames(matcher.groupCount()); } - return Utils.zipMapPartial(columns, Iterables.transform(values, multiValueFunction)); + return Utils.zipMapPartial(columns, Iterables.transform(values, transformationFunction)); } catch (Exception e) { throw new ParseException(line, e, "Unable to parse row [%s]", line); diff --git a/processing/src/main/java/org/apache/druid/frame/allocation/AppendableMemory.java b/processing/src/main/java/org/apache/druid/frame/allocation/AppendableMemory.java index 11edf396c9b..8d961f298ce 100644 --- a/processing/src/main/java/org/apache/druid/frame/allocation/AppendableMemory.java +++ b/processing/src/main/java/org/apache/druid/frame/allocation/AppendableMemory.java @@ -54,10 +54,10 @@ public class AppendableMemory implements Closeable // One holder for every Memory we've allocated. private final List> blockHolders = new ArrayList<>(); - // The amount of space that has been used from each Memory block. Same length as "memoryHolders". + // The amount of space that has been used from each Memory block. Same length as "blockHolders". private final IntList limits = new IntArrayList(); - // The global starting position for each Memory block (blockNumber -> position). Same length as "memoryHolders". + // The global starting position for each Memory block (blockNumber -> position). Same length as "blockHolders". private final LongArrayList globalStartPositions = new LongArrayList(); // Whether the blocks we've allocated are "packed"; meaning all non-final block limits equal the allocationSize. @@ -104,6 +104,36 @@ public class AppendableMemory implements Closeable return cursor; } + /** + * Maximum number that can be successfully passed to {@link #reserveAdditional(int)}. + */ + public int availableToReserve() + { + final int currentBlockIdx = currentBlockNumber(); + final long availableInCurrentBlock; + final boolean currentBlockIsEmpty; + + if (currentBlockIdx < 0) { + availableInCurrentBlock = 0; + currentBlockIsEmpty = false; + } else { + final int usedInCurrentBlock = limits.getInt(currentBlockIdx); + availableInCurrentBlock = blockHolders.get(currentBlockIdx).get().getCapacity() - usedInCurrentBlock; + currentBlockIsEmpty = usedInCurrentBlock == 0; + } + + // If currentBlockIsEmpty, add availableInCurrentBlock to account for reclamation in reclaimLastBlockIfEmpty(). + final long availableInAllocator = allocator.available() + (currentBlockIsEmpty ? availableInCurrentBlock : 0); + + return (int) Math.min( + Integer.MAX_VALUE, + Math.max( + availableInAllocator, + availableInCurrentBlock + ) + ); + } + /** * Ensure that at least "bytes" amount of space is available after the cursor. Allocates a new block if needed. * Note: the amount of bytes is guaranteed to be in a *single* block. @@ -126,11 +156,13 @@ public class AppendableMemory implements Closeable return true; } + releaseLastBlockIfEmpty(); + if (bytes > allocator.available()) { return false; } - final int idx = blockHolders.size() - 1; + final int idx = currentBlockNumber(); if (idx < 0 || bytes + limits.getInt(idx) > blockHolders.get(idx).get().getCapacity()) { // Allocation needed. @@ -228,6 +260,9 @@ public class AppendableMemory implements Closeable cursor.set(currentBlockMemory, newLimit, currentBlockMemory.getCapacity() - newLimit); } + /** + * Current used size, in bytes. + */ public long size() { long sz = 0; @@ -295,12 +330,21 @@ public class AppendableMemory implements Closeable cursor.set(blockMemory, 0, blockMemory.getCapacity()); } - private int currentBlockNumber() + private void releaseLastBlockIfEmpty() { - if (blockHolders.isEmpty()) { - return NO_BLOCK; - } else { - return blockHolders.size() - 1; + final int lastBlockNumber = currentBlockNumber(); + if (lastBlockNumber != NO_BLOCK && limits.getInt(lastBlockNumber) == 0) { + blockHolders.remove(lastBlockNumber).close(); + limits.removeInt(lastBlockNumber); } } + + /** + * Returns the index into {@link #blockHolders} and {@link #limits} of the current block, or {@link #NO_BLOCK} + * if there are no blocks. + */ + private int currentBlockNumber() + { + return blockHolders.size() - 1; + } } diff --git a/processing/src/main/java/org/apache/druid/frame/allocation/ArenaMemoryAllocator.java b/processing/src/main/java/org/apache/druid/frame/allocation/ArenaMemoryAllocator.java index 7b8c23e2936..ebc49333552 100644 --- a/processing/src/main/java/org/apache/druid/frame/allocation/ArenaMemoryAllocator.java +++ b/processing/src/main/java/org/apache/druid/frame/allocation/ArenaMemoryAllocator.java @@ -37,6 +37,7 @@ public class ArenaMemoryAllocator implements MemoryAllocator private final WritableMemory arena; private long allocations = 0; private long position = 0; + private WritableMemory lastAllocation; private ArenaMemoryAllocator(WritableMemory arena) { @@ -64,20 +65,23 @@ public class ArenaMemoryAllocator implements MemoryAllocator @Override public Optional> allocate(final long size) { - if (position + size < arena.getCapacity()) { + if (position + size <= arena.getCapacity()) { final long start = position; allocations++; position += size; + final WritableMemory memory = arena.writableRegion(start, size, ByteOrder.LITTLE_ENDIAN); + lastAllocation = memory; + return Optional.of( new ResourceHolder() { - private WritableMemory memory = arena.writableRegion(start, size, ByteOrder.LITTLE_ENDIAN); + boolean closed; @Override public WritableMemory get() { - if (memory == null) { + if (closed) { throw new ISE("Already closed"); } @@ -87,10 +91,21 @@ public class ArenaMemoryAllocator implements MemoryAllocator @Override public void close() { - memory = null; + if (closed) { + return; + } + + closed = true; + + //noinspection ObjectEquality + if (memory == lastAllocation) { + // Last allocation closed; decrement position to enable partial arena reuse. + position -= memory.getCapacity(); + lastAllocation = null; + } if (--allocations == 0) { - // All allocations closed; reset position to enable arena reuse. + // All allocations closed; reset position to enable full arena reuse. position = 0; } } diff --git a/processing/src/main/java/org/apache/druid/frame/allocation/HeapMemoryAllocator.java b/processing/src/main/java/org/apache/druid/frame/allocation/HeapMemoryAllocator.java index ee3af073f88..a78c53c4a37 100644 --- a/processing/src/main/java/org/apache/druid/frame/allocation/HeapMemoryAllocator.java +++ b/processing/src/main/java/org/apache/druid/frame/allocation/HeapMemoryAllocator.java @@ -77,8 +77,10 @@ public class HeapMemoryAllocator implements MemoryAllocator @Override public void close() { - memory = null; - bytesAllocated -= size; + if (memory != null) { + memory = null; + bytesAllocated -= size; + } } } ); diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ChannelClosedForWritesException.java b/processing/src/main/java/org/apache/druid/frame/channel/ChannelClosedForWritesException.java new file mode 100644 index 00000000000..93379017491 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/channel/ChannelClosedForWritesException.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.channel; + +/** + * Exception thrown by {@link ReadableByteChunksFrameChannel#addChunk(byte[])} when the channel has been closed + * for writes, i.e., after {@link ReadableByteChunksFrameChannel#doneWriting()} or + * {@link ReadableByteChunksFrameChannel#close()} has been called. + */ +public class ChannelClosedForWritesException extends RuntimeException +{ + public ChannelClosedForWritesException() + { + super("Channel is no longer accepting writes"); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannel.java index a4a40d70a38..79ad621de28 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannel.java @@ -132,13 +132,15 @@ public class ReadableByteChunksFrameChannel implements ReadableFrameChannel * chunks. (This is not enforced; addChunk will continue to accept new chunks even if the channel is over its limit.) * * When done adding chunks call {@code doneWriting}. + * + * @throws ChannelClosedForWritesException if the channel is closed */ @Nullable public ListenableFuture addChunk(final byte[] chunk) { synchronized (lock) { if (noMoreWrites) { - throw new ISE("Channel is no longer accepting writes"); + throw new ChannelClosedForWritesException(); } try { diff --git a/processing/src/main/java/org/apache/druid/frame/field/RowReader.java b/processing/src/main/java/org/apache/druid/frame/field/RowReader.java index 861609ad188..c79c9b25d67 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/RowReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/RowReader.java @@ -57,7 +57,7 @@ public class RowReader /** * Read a particular field value as an object. * - * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorHolderFactory} + * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorFactory} * for reading many rows out of a frame. */ public Object readField(final Memory memory, final long rowPosition, final long rowLength, final int fieldNumber) @@ -77,7 +77,7 @@ public class RowReader /** * Read an entire row as a list of objects. * - * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorHolderFactory} + * For performance reasons, prefer {@link org.apache.druid.frame.read.FrameReader#makeCursorFactory} * for reading many rows out of a frame. */ public List readRow(final Memory memory, final long rowPosition, final long rowLength) 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 1c51a914e0d..be2263be138 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 @@ -480,9 +480,8 @@ public class StringFieldReader implements FieldReader public boolean isNull(int rowNum) { final long fieldPosition = coach.computeFieldPosition(rowNum); - byte[] nullBytes = new byte[3]; - dataRegion.getByteArray(fieldPosition, nullBytes, 0, 3); - return Arrays.equals(nullBytes, EXPECTED_BYTES_FOR_NULL); + return dataRegion.getByte(fieldPosition) == StringFieldWriter.NULL_ROW + && dataRegion.getByte(fieldPosition + 1) == StringFieldWriter.ROW_TERMINATOR; } @Override diff --git a/processing/src/main/java/org/apache/druid/frame/key/RowKeyComparisonRunLengths.java b/processing/src/main/java/org/apache/druid/frame/key/RowKeyComparisonRunLengths.java index cc05dea993e..ab6797a7c00 100644 --- a/processing/src/main/java/org/apache/druid/frame/key/RowKeyComparisonRunLengths.java +++ b/processing/src/main/java/org/apache/druid/frame/key/RowKeyComparisonRunLengths.java @@ -88,11 +88,12 @@ public class RowKeyComparisonRunLengths ); } - ColumnType columnType = rowSignature.getColumnType(keyColumn.columnName()) - .orElseThrow(() -> DruidException.defensive("Need column types")); + ColumnType columnType = + rowSignature.getColumnType(keyColumn.columnName()) + .orElseThrow(() -> DruidException.defensive("No type for column[%s]", keyColumn.columnName())); // First key column to be processed - if (runLengthEntryBuilders.size() == 0) { + if (runLengthEntryBuilders.isEmpty()) { final boolean isByteComparable = isByteComparable(columnType); runLengthEntryBuilders.add( new RunLengthEntryBuilder(isByteComparable, keyColumn.order()) diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FrameChannelMerger.java b/processing/src/main/java/org/apache/druid/frame/processor/FrameChannelMerger.java index e806f98b926..662fd001b02 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FrameChannelMerger.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FrameChannelMerger.java @@ -19,8 +19,10 @@ package org.apache.druid.frame.processor; +import it.unimi.dsi.fastutil.ints.IntAVLTreeSet; import it.unimi.dsi.fastutil.ints.IntOpenHashSet; import it.unimi.dsi.fastutil.ints.IntSet; +import it.unimi.dsi.fastutil.ints.IntSets; import org.apache.druid.frame.Frame; import org.apache.druid.frame.channel.FrameWithPartition; import org.apache.druid.frame.channel.ReadableFrameChannel; @@ -68,7 +70,11 @@ public class FrameChannelMerger implements FrameProcessor private final long rowLimit; private long rowsOutput = 0; private int currentPartition = 0; - private int remainingChannels; + + /** + * Channels that still have input to read. + */ + private final IntSet remainingChannels; // ColumnSelectorFactory that always reads from the current row in the merged sequence. final MultiColumnSelectorFactory mergedColumnSelectorFactory; @@ -119,7 +125,7 @@ public class FrameChannelMerger implements FrameProcessor this.partitions = partitionsToUse; this.rowLimit = rowLimit; this.currentFrames = new FramePlus[inputChannels.size()]; - this.remainingChannels = 0; + this.remainingChannels = new IntAVLTreeSet(IntSets.fromTo(0, inputChannels.size())); this.tournamentTree = new TournamentTree( inputChannels.size(), (k1, k2) -> { @@ -241,7 +247,7 @@ public class FrameChannelMerger implements FrameProcessor if (rowLimit != UNLIMITED && rowsOutput >= rowLimit) { // Limit reached; we're done. Arrays.fill(currentFrames, null); - remainingChannels = 0; + remainingChannels.clear(); } else { // Continue reading the currentChannel. final FramePlus channelFramePlus = currentFrames[currentChannel]; @@ -251,7 +257,6 @@ public class FrameChannelMerger implements FrameProcessor // Done reading current frame from "channel". // Clear it and see if there is another one available for immediate loading. currentFrames[currentChannel] = null; - remainingChannels--; final ReadableFrameChannel channel = inputChannels.get(currentChannel); @@ -265,10 +270,10 @@ public class FrameChannelMerger implements FrameProcessor break; } else { currentFrames[currentChannel] = framePlus; - remainingChannels++; } } else if (channel.isFinished()) { // Done reading this channel. Fall through and continue with other channels. + remainingChannels.remove(currentChannel); } else { // Nothing available, not finished; we can't continue. Finish up the current frame and return it. break; @@ -282,9 +287,12 @@ public class FrameChannelMerger implements FrameProcessor } } + /** + * Returns whether all input is done being read. + */ private boolean finished() { - return remainingChannels == 0; + return remainingChannels.isEmpty(); } @Override @@ -302,7 +310,7 @@ public class FrameChannelMerger implements FrameProcessor final IntSet await = new IntOpenHashSet(); for (int i = 0; i < inputChannels.size(); i++) { - if (currentFrames[i] == null) { + if (currentFrames[i] == null && remainingChannels.contains(i)) { final ReadableFrameChannel channel = inputChannels.get(i); if (channel.canRead()) { @@ -312,9 +320,10 @@ public class FrameChannelMerger implements FrameProcessor await.add(i); } else { currentFrames[i] = framePlus; - remainingChannels++; } - } else if (!channel.isFinished()) { + } else if (channel.isFinished()) { + remainingChannels.remove(i); + } else { await.add(i); } } 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 c0f79d30e58..f255fbe13a6 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 @@ -46,12 +46,14 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.IdentityHashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.CancellationException; +import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; @@ -61,7 +63,6 @@ import java.util.stream.Collectors; * If you want single threaded execution, use {@code Execs.singleThreaded()}. It is not a good idea to use this with a * same-thread executor like {@code Execs.directExecutor()}, because it will lead to deep call stacks. */ -@SuppressWarnings("CheckReturnValue") public class FrameProcessorExecutor { private static final Logger log = new Logger(FrameProcessorExecutor.class); @@ -70,6 +71,10 @@ public class FrameProcessorExecutor private final Object lock = new Object(); + // Currently-active cancellationIds. + @GuardedBy("lock") + private final Set activeCancellationIds = new HashSet<>(); + // Futures that are active and therefore cancelable. // Does not include return futures: those are in cancelableReturnFutures. @GuardedBy("lock") @@ -96,19 +101,12 @@ public class FrameProcessorExecutor this.exec = exec; } - /** - * Returns the underlying executor service used by this executor. - */ - public ListeningExecutorService getExecutorService() - { - return exec; - } - /** * Runs a processor until it is done, and returns a future that resolves when execution is complete. * - * If "cancellationId" is provided, it can be used with the {@link #cancel(String)} method to cancel all processors - * currently running with the same cancellationId. + * If "cancellationId" is provided, it must have previously been registered with {@link #registerCancellationId}. + * Then, it can be used with the {@link #cancel(String)} method to cancel all processors with that + * same cancellationId. */ public ListenableFuture runFully(final FrameProcessor processor, @Nullable final String cancellationId) { @@ -116,6 +114,11 @@ public class FrameProcessorExecutor final List outputChannels = processor.outputChannels(); final SettableFuture finished = registerCancelableFuture(SettableFuture.create(), true, cancellationId); + if (finished.isDone()) { + // Possibly due to starting life out being canceled. + return finished; + } + class ExecutorRunnable implements Runnable { private final AwaitAnyWidget awaitAnyWidget = new AwaitAnyWidget(inputChannels); @@ -152,7 +155,7 @@ public class FrameProcessorExecutor final IntSet await = result.awaitSet(); if (await.isEmpty()) { - exec.submit(ExecutorRunnable.this); + exec.execute(ExecutorRunnable.this); } else if (result.isAwaitAll() || await.size() == 1) { final List> readabilityFutures = new ArrayList<>(); @@ -164,7 +167,7 @@ public class FrameProcessorExecutor } if (readabilityFutures.isEmpty()) { - exec.submit(ExecutorRunnable.this); + exec.execute(ExecutorRunnable.this); } else { runProcessorAfterFutureResolves(Futures.allAsList(readabilityFutures)); } @@ -272,7 +275,7 @@ public class FrameProcessorExecutor public void onSuccess(final V ignored) { try { - exec.submit(ExecutorRunnable.this); + exec.execute(ExecutorRunnable.this); } catch (Throwable e) { fail(e); @@ -390,7 +393,7 @@ public class FrameProcessorExecutor logProcessorStatusString(processor, finished, null); registerCancelableProcessor(processor, cancellationId); - exec.submit(runnable); + exec.execute(runnable); return finished; } @@ -423,8 +426,20 @@ public class FrameProcessorExecutor } /** - * Cancels all processors associated with a given cancellationId. Waits for the processors to exit before - * returning. + * Registers a cancellationId, so it can be provided to {@link #runFully} or {@link #runAllFully}. To avoid the + * set of active cancellationIds growing without bound, callers must also call {@link #cancel(String)} on the + * same cancellationId when done using it. + */ + public void registerCancellationId(final String cancellationId) + { + synchronized (lock) { + activeCancellationIds.add(cancellationId); + } + } + + /** + * Deregisters a cancellationId and cancels any currently-running processors associated with that cancellationId. + * Waits for any canceled processors to exit before returning. */ public void cancel(final String cancellationId) throws InterruptedException { @@ -435,6 +450,7 @@ public class FrameProcessorExecutor final Set> returnFuturesToCancel; synchronized (lock) { + activeCancellationIds.remove(cancellationId); futuresToCancel = cancelableFutures.removeAll(cancellationId); processorsToCancel = cancelableProcessors.removeAll(cancellationId); returnFuturesToCancel = cancelableReturnFutures.removeAll(cancellationId); @@ -457,6 +473,33 @@ public class FrameProcessorExecutor } } + /** + * Returns an {@link Executor} that executes using the same underlying service, and that is also connected to + * cancellation through {@link #cancel(String)}. + * + * @param cancellationId cancellation ID for the executor + */ + public Executor asExecutor(@Nullable final String cancellationId) + { + return command -> runFully(new RunnableFrameProcessor(command), cancellationId); + } + + /** + * Shuts down the underlying executor service immediately. + */ + public void shutdownNow() + { + exec.shutdownNow(); + } + + /** + * Returns the underlying executor service used by this executor. + */ + ListeningExecutorService getExecutorService() + { + return exec; + } + /** * Register a future that will be canceled when the provided {@code cancellationId} is canceled. * @@ -472,6 +515,12 @@ public class FrameProcessorExecutor { if (cancellationId != null) { synchronized (lock) { + if (!activeCancellationIds.contains(cancellationId)) { + // Cancel and return immediately. + future.cancel(true); + return future; + } + final SetMultimap> map = isReturn ? cancelableReturnFutures : cancelableFutures; map.put(cancellationId, future); future.addListener( 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 f75ced24b12..4e659b67cff 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 @@ -29,9 +29,7 @@ import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.WritableFrameChannel; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.segment.FrameCursor; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; @@ -122,8 +120,7 @@ public class FrameProcessors // FrameCursorHolderFactory has no closer stuff at all and is totally safe. If this ever changes, this method will // probably need to wrap the cursor in something closeable, or be reworked to just return the CursorHolder so that // callers can deal with closing the stuff. - return (FrameCursor) new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).makeCursorHolder(cursorBuildSpec) - .asCursor(); + return (FrameCursor) frameReader.makeCursorFactory(frame).makeCursorHolder(cursorBuildSpec).asCursor(); } /** 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 a1a1c0f8712..7f79a319c28 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 @@ -306,9 +306,11 @@ public class RunAllFullyWidget } if (isDone) { - finished.compareAndSet(null, Either.value(processorManager.result())); - synchronized (runAllFullyLock) { + if (finished.get() == null) { + finished.compareAndSet(null, Either.value(processorManager.result())); + } + cleanupIfNoMoreProcessors(); } } else { diff --git a/processing/src/main/java/org/apache/druid/frame/processor/RunnableFrameProcessor.java b/processing/src/main/java/org/apache/druid/frame/processor/RunnableFrameProcessor.java new file mode 100644 index 00000000000..697879490e1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/processor/RunnableFrameProcessor.java @@ -0,0 +1,65 @@ +/* + * 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; + +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; + +import java.util.Collections; +import java.util.List; + +/** + * Frame processor that simply runs a {@link Runnable}, once. + */ +public class RunnableFrameProcessor implements FrameProcessor +{ + private final Runnable runnable; + + public RunnableFrameProcessor(Runnable runnable) + { + this.runnable = runnable; + } + + @Override + public List inputChannels() + { + return Collections.emptyList(); + } + + @Override + public List outputChannels() + { + return Collections.emptyList(); + } + + @Override + public ReturnOrAwait runIncrementally(IntSet readableInputs) + { + runnable.run(); + return ReturnOrAwait.returnObject(null); + } + + @Override + public void cleanup() + { + // Nothing to do. + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java b/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java index e30f2e77b02..b8b74a2b797 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java @@ -297,7 +297,7 @@ public class SuperSorter setAllDoneIfPossible(); } }, - exec.getExecutorService() + exec.asExecutor(cancellationId) ); return FutureUtils.futureWithBaggage( @@ -813,7 +813,7 @@ public class SuperSorter }, // Must run in exec, instead of in the same thread, to avoid running callback immediately if the // worker happens to finish super-quickly. - exec.getExecutorService() + exec.asExecutor(cancellationId) ); } diff --git a/processing/src/main/java/org/apache/druid/frame/processor/manager/ConcurrencyLimitedProcessorManager.java b/processing/src/main/java/org/apache/druid/frame/processor/manager/ConcurrencyLimitedProcessorManager.java new file mode 100644 index 00000000000..751507423b0 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/processor/manager/ConcurrencyLimitedProcessorManager.java @@ -0,0 +1,74 @@ +/* + * 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.common.guava.FutureUtils; +import org.apache.druid.frame.processor.Bouncer; + +import java.util.Optional; + +/** + * Manager that limits the number of processors that may run concurrently. + */ +public class ConcurrencyLimitedProcessorManager implements ProcessorManager +{ + private final ProcessorManager delegate; + private final Bouncer bouncer; + + public ConcurrencyLimitedProcessorManager(ProcessorManager delegate, int limit) + { + this.delegate = delegate; + this.bouncer = new Bouncer(limit); + } + + @Override + public ListenableFuture>> next() + { + final ListenableFuture ticket = bouncer.ticket(); + return FutureUtils.transformAsync( + ticket, + t -> FutureUtils.transform( + delegate.next(), + nextProcessor -> nextProcessor.map( + retVal -> new ProcessorAndCallback<>( + retVal.processor(), + r -> { + FutureUtils.getUncheckedImmediately(ticket).giveBack(); + retVal.onComplete(r); + } + ) + ) + ) + ); + } + + @Override + public R result() + { + return delegate.result(); + } + + @Override + public void close() + { + delegate.close(); + } +} 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 ce169b72b1d..1f23b88fc6f 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 @@ -29,11 +29,11 @@ import org.apache.druid.frame.key.FrameComparisonWidgetImpl; import org.apache.druid.frame.key.KeyColumn; import org.apache.druid.frame.read.columnar.FrameColumnReader; import org.apache.druid.frame.read.columnar.FrameColumnReaders; -import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorHolderFactory; -import org.apache.druid.frame.segment.row.RowFrameCursorHolderFactory; +import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorFactory; +import org.apache.druid.frame.segment.row.RowFrameCursorFactory; import org.apache.druid.frame.write.FrameWriterUtils; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.segment.CursorHolderFactory; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -137,15 +137,15 @@ public class FrameReader } /** - * Create a {@link CursorHolderFactory} for the given frame. + * Create a {@link CursorFactory} for the given frame. */ - public CursorHolderFactory makeCursorHolderFactory(final Frame frame) + public CursorFactory makeCursorFactory(final Frame frame) { switch (frame.type()) { case COLUMNAR: - return new ColumnarFrameCursorHolderFactory(frame, signature, columnReaders); + return new ColumnarFrameCursorFactory(frame, signature, columnReaders); case ROW_BASED: - return new RowFrameCursorHolderFactory(frame, this, fieldReaders); + return new RowFrameCursorFactory(frame, this, fieldReaders); default: throw DruidException.defensive("Unrecognized frame type [%s]", frame.type()); } 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 9b4dc85cb1e..6ceb6b72917 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 @@ -51,7 +51,7 @@ public class FrameColumnReaders return new DoubleFrameColumnReader(columnNumber); case STRING: - return new StringFrameColumnReader(columnNumber, false); + return new StringFrameColumnReader(columnNumber); case COMPLEX: return new ComplexFrameColumnReader(columnNumber); @@ -59,7 +59,7 @@ public class FrameColumnReaders case ARRAY: switch (columnType.getElementType().getType()) { case STRING: - return new StringFrameColumnReader(columnNumber, true); + return new StringArrayFrameColumnReader(columnNumber); case LONG: return new LongArrayFrameColumnReader(columnNumber); case FLOAT: diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/StringArrayFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/StringArrayFrameColumnReader.java new file mode 100644 index 00000000000..31c56bf38e7 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/StringArrayFrameColumnReader.java @@ -0,0 +1,385 @@ +/* + * 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 com.google.common.primitives.Ints; +import it.unimi.dsi.fastutil.objects.ObjectArrays; +import org.apache.datasketches.memory.Memory; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.read.FrameReaderUtils; +import org.apache.druid.frame.write.FrameWriterUtils; +import org.apache.druid.frame.write.columnar.FrameColumnWriters; +import org.apache.druid.frame.write.columnar.StringFrameColumnWriter; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn; +import org.apache.druid.query.rowsandcols.column.accessor.ObjectColumnAccessorBase; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.ObjectColumnSelector; +import org.apache.druid.segment.column.BaseColumn; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.data.ReadableOffset; +import org.apache.druid.segment.vector.ReadableVectorInspector; +import org.apache.druid.segment.vector.ReadableVectorOffset; +import org.apache.druid.segment.vector.VectorObjectSelector; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.util.Comparator; + +/** + * Reader for {@link ColumnType#STRING_ARRAY}. + * This is similar to {@link StringFrameColumnReader} reading mvds in reading bytes from frame + */ +public class StringArrayFrameColumnReader implements FrameColumnReader +{ + private final int columnNumber; + + /** + * Create a new reader. + * + * @param columnNumber column number + */ + StringArrayFrameColumnReader(int columnNumber) + { + this.columnNumber = columnNumber; + } + + @Override + public Column readRACColumn(Frame frame) + { + final Memory memory = frame.region(columnNumber); + validate(memory); + + final long positionOfLengths = getStartOfStringLengthSection(frame.numRows()); + final long positionOfPayloads = getStartOfStringDataSection(memory, frame.numRows()); + + StringArrayFrameColumn frameCol = new StringArrayFrameColumn( + frame, + memory, + positionOfLengths, + positionOfPayloads + ); + + return new ColumnAccessorBasedColumn(frameCol); + } + + @Override + public ColumnPlus readColumn(final Frame frame) + { + final Memory memory = frame.region(columnNumber); + validate(memory); + + final long startOfStringLengthSection = getStartOfStringLengthSection(frame.numRows()); + final long startOfStringDataSection = getStartOfStringDataSection(memory, frame.numRows()); + + final BaseColumn baseColumn = new StringArrayFrameColumn( + frame, + memory, + startOfStringLengthSection, + startOfStringDataSection + ); + + return new ColumnPlus( + baseColumn, + new ColumnCapabilitiesImpl().setType(ColumnType.STRING_ARRAY) + .setHasMultipleValues(false) + .setDictionaryEncoded(false), + frame.numRows() + ); + } + + private void validate(final Memory region) + { + // Check if column is big enough for a header + if (region.getCapacity() < StringFrameColumnWriter.DATA_OFFSET) { + throw DruidException.defensive("Column[%s] is not big enough for a header", columnNumber); + } + + final byte typeCode = region.getByte(0); + if (typeCode != FrameColumnWriters.TYPE_STRING_ARRAY) { + throw DruidException.defensive( + "Column[%s] does not have the correct type code; expected[%s], got[%s]", + columnNumber, + FrameColumnWriters.TYPE_STRING_ARRAY, + typeCode + ); + } + } + + private static long getStartOfCumulativeLengthSection() + { + return StringFrameColumnWriter.DATA_OFFSET; + } + + private static long getStartOfStringLengthSection(final int numRows) + { + return StringFrameColumnWriter.DATA_OFFSET + (long) Integer.BYTES * numRows; + } + + private long getStartOfStringDataSection( + final Memory memory, + final int numRows + ) + { + if (numRows < 0) { + throw DruidException.defensive("Encountered -ve numRows [%s] while reading frame", numRows); + } + final int totalNumValues = FrameColumnReaderUtils.getAdjustedCumulativeRowLength( + memory, + getStartOfCumulativeLengthSection(), + numRows - 1 + ); + + return getStartOfStringLengthSection(numRows) + (long) Integer.BYTES * totalNumValues; + } + + private static class StringArrayFrameColumn extends ObjectColumnAccessorBase implements BaseColumn + { + private final Frame frame; + private final Memory memory; + private final long startOfStringLengthSection; + private final long startOfStringDataSection; + + private StringArrayFrameColumn( + Frame frame, + Memory memory, + long startOfStringLengthSection, + long startOfStringDataSection + ) + { + this.frame = frame; + this.memory = memory; + this.startOfStringLengthSection = startOfStringLengthSection; + this.startOfStringDataSection = startOfStringDataSection; + } + + @Override + public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + { + return new ObjectColumnSelector() + { + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // Do nothing. + } + + @Nullable + @Override + public Object getObject() + { + return getRowAsObject(frame.physicalRow(offset.getOffset()), true); + } + + @Override + public Class classOfObject() + { + return Object[].class; + } + }; + } + + @Override + public VectorObjectSelector makeVectorObjectSelector(final ReadableVectorOffset offset) + { + class StringArrayFrameVectorObjectSelector implements VectorObjectSelector + { + private final Object[] vector = new Object[offset.getMaxVectorSize()]; + private int id = ReadableVectorInspector.NULL_ID; + + @Override + public Object[] getObjectVector() + { + computeVectorIfNeeded(); + return vector; + } + + @Override + public int getMaxVectorSize() + { + return offset.getMaxVectorSize(); + } + + @Override + public int getCurrentVectorSize() + { + return offset.getCurrentVectorSize(); + } + + private void computeVectorIfNeeded() + { + if (id == offset.getId()) { + return; + } + + if (offset.isContiguous()) { + final int start = offset.getStartOffset(); + + for (int i = 0; i < offset.getCurrentVectorSize(); i++) { + final int physicalRow = frame.physicalRow(i + start); + vector[i] = getRowAsObject(physicalRow, true); + } + } else { + final int[] offsets = offset.getOffsets(); + + for (int i = 0; i < offset.getCurrentVectorSize(); i++) { + final int physicalRow = frame.physicalRow(offsets[i]); + vector[i] = getRowAsObject(physicalRow, true); + } + } + + id = offset.getId(); + } + } + + return new StringArrayFrameVectorObjectSelector(); + } + + @Override + public void close() + { + // Do nothing. + } + + @Override + public ColumnType getType() + { + return ColumnType.STRING_ARRAY; + } + + @Override + public int numRows() + { + return frame.numRows(); + } + + @Override + protected Object getVal(int rowNum) + { + return getRowAsObject(frame.physicalRow(rowNum), true); + } + + @Override + protected Comparator getComparator() + { + return Comparator.nullsFirst(ColumnType.STRING_ARRAY.getStrategy()); + } + + /** + * Returns a ByteBuffer containing UTF-8 encoded string number {@code index}. The ByteBuffer is always newly + * created, so it is OK to change its position, limit, etc. However, it may point to shared memory, so it is + * not OK to write to its contents. + */ + @Nullable + private ByteBuffer getStringUtf8(final int index) + { + if (startOfStringLengthSection > Long.MAX_VALUE - (long) Integer.BYTES * index) { + throw DruidException.defensive("length index would overflow trying to read the frame memory!"); + } + + final int dataEndVariableIndex = memory.getInt(startOfStringLengthSection + (long) Integer.BYTES * index); + if (startOfStringDataSection > Long.MAX_VALUE - dataEndVariableIndex) { + throw DruidException.defensive("data end index would overflow trying to read the frame memory!"); + } + + final long dataStart; + final long dataEnd = startOfStringDataSection + dataEndVariableIndex; + + if (index == 0) { + dataStart = startOfStringDataSection; + } else { + final int dataStartVariableIndex = memory.getInt(startOfStringLengthSection + (long) Integer.BYTES * (index + - 1)); + if (startOfStringDataSection > Long.MAX_VALUE - dataStartVariableIndex) { + throw DruidException.defensive("data start index would overflow trying to read the frame memory!"); + } + dataStart = startOfStringDataSection + dataStartVariableIndex; + } + + final int dataLength = Ints.checkedCast(dataEnd - dataStart); + + if ((dataLength == 0 && NullHandling.replaceWithDefault()) || + (dataLength == 1 && memory.getByte(dataStart) == FrameWriterUtils.NULL_STRING_MARKER)) { + return null; + } + + return FrameReaderUtils.readByteBuffer(memory, dataStart, dataLength); + } + + @Nullable + private String getString(final int index) + { + final ByteBuffer stringUtf8 = getStringUtf8(index); + + if (stringUtf8 == null) { + return null; + } else { + return StringUtils.fromUtf8(stringUtf8); + } + } + + /** + * Returns the object at the given physical row number. + * + * @param physicalRow physical row number + * @param decode if true, return java.lang.String. If false, return UTF-8 ByteBuffer. + */ + @Nullable + private Object getRowAsObject(final int physicalRow, final boolean decode) + { + final int cumulativeRowLength = FrameColumnReaderUtils.getCumulativeRowLength( + memory, + getStartOfCumulativeLengthSection(), + physicalRow + ); + final int rowLength; + + if (FrameColumnReaderUtils.isNullRow(cumulativeRowLength)) { + return null; + } else if (physicalRow == 0) { + rowLength = cumulativeRowLength; + } else { + rowLength = cumulativeRowLength - FrameColumnReaderUtils.getAdjustedCumulativeRowLength( + memory, + getStartOfCumulativeLengthSection(), + physicalRow - 1 + ); + } + + if (rowLength == 0) { + return ObjectArrays.EMPTY_ARRAY; + } else { + final Object[] row = new Object[rowLength]; + + for (int i = 0; i < rowLength; i++) { + final int index = cumulativeRowLength - rowLength + i; + row[i] = decode ? getString(index) : getStringUtf8(index); + } + + return row; + } + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java index d9fb9d83a9f..1bd008d5378 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java @@ -19,18 +19,17 @@ package org.apache.druid.frame.read.columnar; -import com.google.common.annotations.VisibleForTesting; import com.google.common.primitives.Ints; -import it.unimi.dsi.fastutil.objects.ObjectArrays; +import org.apache.commons.lang.ObjectUtils; import org.apache.datasketches.memory.Memory; import org.apache.druid.common.config.NullHandling; import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidInput; import org.apache.druid.frame.Frame; import org.apache.druid.frame.read.FrameReaderUtils; import org.apache.druid.frame.write.FrameWriterUtils; import org.apache.druid.frame.write.columnar.FrameColumnWriters; import org.apache.druid.frame.write.columnar.StringFrameColumnWriter; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.filter.DruidPredicateFactory; @@ -40,13 +39,11 @@ import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn; import org.apache.druid.query.rowsandcols.column.accessor.ObjectColumnAccessorBase; import org.apache.druid.segment.BaseSingleValueDimensionSelector; -import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.DimensionSelectorUtils; import org.apache.druid.segment.IdLookup; import org.apache.druid.segment.column.BaseColumn; -import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.DictionaryEncodedColumn; @@ -67,23 +64,20 @@ import java.util.Comparator; import java.util.List; /** - * Reader for {@link StringFrameColumnWriter}, types {@link ColumnType#STRING} and {@link ColumnType#STRING_ARRAY}. + * Reader for {@link StringFrameColumnWriter}, type {@link ColumnType#STRING}. */ public class StringFrameColumnReader implements FrameColumnReader { private final int columnNumber; - private final boolean asArray; /** * Create a new reader. * * @param columnNumber column number - * @param asArray true for {@link ColumnType#STRING_ARRAY}, false for {@link ColumnType#STRING} */ - StringFrameColumnReader(int columnNumber, boolean asArray) + StringFrameColumnReader(int columnNumber) { this.columnNumber = columnNumber; - this.asArray = asArray; } @Override @@ -92,18 +86,20 @@ public class StringFrameColumnReader implements FrameColumnReader final Memory memory = frame.region(columnNumber); validate(memory); + if (isMultiValue(memory)) { + throw InvalidInput.exception("Encountered a multi value column. Window processing does not support MVDs. " + + "Consider using UNNEST or MV_TO_ARRAY."); + } final long positionOfLengths = getStartOfStringLengthSection(frame.numRows(), false); final long positionOfPayloads = getStartOfStringDataSection(memory, frame.numRows(), false); - StringFrameColumn frameCol = - new StringFrameColumn( - frame, - false, - memory, - positionOfLengths, - positionOfPayloads, - asArray || isMultiValue(memory) // Read MVDs as String arrays - ); + StringFrameColumn frameCol = new StringFrameColumn( + frame, + false, + memory, + positionOfLengths, + positionOfPayloads + ); return new ColumnAccessorBasedColumn(frameCol); } @@ -118,35 +114,19 @@ public class StringFrameColumnReader implements FrameColumnReader final long startOfStringLengthSection = getStartOfStringLengthSection(frame.numRows(), multiValue); final long startOfStringDataSection = getStartOfStringDataSection(memory, frame.numRows(), multiValue); - final BaseColumn baseColumn; - - if (asArray) { - baseColumn = new StringArrayFrameColumn( - frame, - multiValue, - memory, - startOfStringLengthSection, - startOfStringDataSection - ); - } else { - baseColumn = new StringFrameColumn( - frame, - multiValue, - memory, - startOfStringLengthSection, - startOfStringDataSection, - false - ); - } + final BaseColumn baseColumn = new StringFrameColumn( + frame, + multiValue, + memory, + startOfStringLengthSection, + startOfStringDataSection + ); return new ColumnPlus( baseColumn, - new ColumnCapabilitiesImpl().setType(asArray ? ColumnType.STRING_ARRAY : ColumnType.STRING) - .setHasMultipleValues(!asArray && multiValue) - .setDictionaryEncoded(false) - .setHasBitmapIndexes(false) - .setHasSpatialIndexes(false) - .setHasNulls(ColumnCapabilities.Capable.UNKNOWN), + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) + .setHasMultipleValues(multiValue) + .setDictionaryEncoded(false), frame.numRows() ); } @@ -159,12 +139,11 @@ public class StringFrameColumnReader implements FrameColumnReader } final byte typeCode = region.getByte(0); - final byte expectedTypeCode = asArray ? FrameColumnWriters.TYPE_STRING_ARRAY : FrameColumnWriters.TYPE_STRING; - if (typeCode != expectedTypeCode) { + if (typeCode != FrameColumnWriters.TYPE_STRING) { throw DruidException.defensive( "Column[%s] does not have the correct type code; expected[%s], got[%s]", columnNumber, - expectedTypeCode, + FrameColumnWriters.TYPE_STRING, typeCode ); } @@ -172,7 +151,7 @@ public class StringFrameColumnReader implements FrameColumnReader private static boolean isMultiValue(final Memory memory) { - return memory.getByte(1) == 1; + return memory.getByte(StringFrameColumnWriter.MULTI_VALUE_POSITION) == StringFrameColumnWriter.MULTI_VALUE_BYTE; } private static long getStartOfCumulativeLengthSection() @@ -213,8 +192,7 @@ public class StringFrameColumnReader implements FrameColumnReader return getStartOfStringLengthSection(numRows, multiValue) + (long) Integer.BYTES * totalNumValues; } - @VisibleForTesting - static class StringFrameColumn extends ObjectColumnAccessorBase implements DictionaryEncodedColumn + private static class StringFrameColumn extends ObjectColumnAccessorBase implements DictionaryEncodedColumn { private final Frame frame; private final Memory memory; @@ -226,18 +204,12 @@ public class StringFrameColumnReader implements FrameColumnReader */ private final boolean multiValue; - /** - * Whether the column is being read as {@link ColumnType#STRING_ARRAY} (true) or {@link ColumnType#STRING} (false). - */ - private final boolean asArray; - private StringFrameColumn( Frame frame, boolean multiValue, Memory memory, long startOfStringLengthSection, - long startOfStringDataSection, - final boolean asArray + long startOfStringDataSection ) { this.frame = frame; @@ -245,7 +217,6 @@ public class StringFrameColumnReader implements FrameColumnReader this.memory = memory; this.startOfStringLengthSection = startOfStringLengthSection; this.startOfStringDataSection = startOfStringDataSection; - this.asArray = asArray; } @Override @@ -292,247 +263,6 @@ public class StringFrameColumnReader implements FrameColumnReader @Override public DimensionSelector makeDimensionSelector(ReadableOffset offset, @Nullable ExtractionFn extractionFn) - { - if (asArray) { - throw new ISE("Cannot call makeDimensionSelector on field of type [%s]", ColumnType.STRING_ARRAY); - } - - return makeDimensionSelectorInternal(offset, extractionFn); - } - - @Override - public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(ReadableVectorOffset offset) - { - // Callers should use object selectors, because we have no dictionary. - throw new UnsupportedOperationException(); - } - - @Override - public MultiValueDimensionVectorSelector makeMultiValueDimensionVectorSelector(ReadableVectorOffset vectorOffset) - { - // Callers should use object selectors, because we have no dictionary. - throw new UnsupportedOperationException(); - } - - @Override - public VectorObjectSelector makeVectorObjectSelector(final ReadableVectorOffset offset) - { - class StringFrameVectorObjectSelector implements VectorObjectSelector - { - private final Object[] vector = new Object[offset.getMaxVectorSize()]; - private int id = ReadableVectorInspector.NULL_ID; - - @Override - public Object[] getObjectVector() - { - computeVectorIfNeeded(); - return vector; - } - - @Override - public int getMaxVectorSize() - { - return offset.getMaxVectorSize(); - } - - @Override - public int getCurrentVectorSize() - { - return offset.getCurrentVectorSize(); - } - - private void computeVectorIfNeeded() - { - if (id == offset.getId()) { - return; - } - - if (offset.isContiguous()) { - final int start = offset.getStartOffset(); - - for (int i = 0; i < offset.getCurrentVectorSize(); i++) { - final int physicalRow = frame.physicalRow(i + start); - vector[i] = getRowAsObject(physicalRow, true); - } - } else { - final int[] offsets = offset.getOffsets(); - - for (int i = 0; i < offset.getCurrentVectorSize(); i++) { - final int physicalRow = frame.physicalRow(offsets[i]); - vector[i] = getRowAsObject(physicalRow, true); - } - } - - id = offset.getId(); - } - } - - return new StringFrameVectorObjectSelector(); - } - - @Override - public int length() - { - return frame.numRows(); - } - - @Override - public void close() - { - // Do nothing. - } - - @Override - public ColumnType getType() - { - return asArray ? ColumnType.STRING_ARRAY : ColumnType.STRING; - } - - @Override - public int numRows() - { - return length(); - } - - @Override - protected Object getVal(int rowNum) - { - return getString(frame.physicalRow(rowNum)); - } - - @Override - protected Comparator getComparator() - { - return Comparator.nullsFirst(Comparator.comparing(o -> ((String) o))); - } - - /** - * Returns a ByteBuffer containing UTF-8 encoded string number {@code index}. The ByteBuffer is always newly - * created, so it is OK to change its position, limit, etc. However, it may point to shared memory, so it is - * not OK to write to its contents. - */ - @Nullable - private ByteBuffer getStringUtf8(final int index) - { - final long dataStart; - final long dataEnd = - startOfStringDataSection + - memory.getInt(startOfStringLengthSection + (long) Integer.BYTES * index); - - if (index == 0) { - dataStart = startOfStringDataSection; - } else { - dataStart = - startOfStringDataSection + - memory.getInt(startOfStringLengthSection + (long) Integer.BYTES * (index - 1)); - } - - final int dataLength = Ints.checkedCast(dataEnd - dataStart); - - if ((dataLength == 0 && NullHandling.replaceWithDefault()) || - (dataLength == 1 && memory.getByte(dataStart) == FrameWriterUtils.NULL_STRING_MARKER)) { - return null; - } - - return FrameReaderUtils.readByteBuffer(memory, dataStart, dataLength); - } - - @Nullable - private String getString(final int index) - { - final ByteBuffer stringUtf8 = getStringUtf8(index); - - if (stringUtf8 == null) { - return null; - } else { - return StringUtils.fromUtf8(stringUtf8); - } - } - - /** - * Returns the object at the given physical row number. - * - * When {@link #asArray}, the return value is always of type {@code Object[]}. Otherwise, the return value - * is either an empty list (if the row is empty), a single String (if the row has one value), or a List - * of Strings (if the row has more than one value). - * - * @param physicalRow physical row number - * @param decode if true, return java.lang.String. If false, return UTF-8 ByteBuffer. - */ - @Nullable - private Object getRowAsObject(final int physicalRow, final boolean decode) - { - if (multiValue) { - final int cumulativeRowLength = FrameColumnReaderUtils.getCumulativeRowLength( - memory, - getStartOfCumulativeLengthSection(), - physicalRow - ); - final int rowLength; - - if (FrameColumnReaderUtils.isNullRow(cumulativeRowLength)) { - return null; - } else if (physicalRow == 0) { - rowLength = cumulativeRowLength; - } else { - rowLength = cumulativeRowLength - FrameColumnReaderUtils.getAdjustedCumulativeRowLength( - memory, - getStartOfCumulativeLengthSection(), - physicalRow - 1 - ); - } - - if (rowLength == 0) { - return asArray ? ObjectArrays.EMPTY_ARRAY : Collections.emptyList(); - } else if (rowLength == 1) { - final int index = cumulativeRowLength - 1; - final Object o = decode ? getString(index) : getStringUtf8(index); - return asArray ? new Object[]{o} : o; - } else { - final Object[] row = new Object[rowLength]; - - for (int i = 0; i < rowLength; i++) { - final int index = cumulativeRowLength - rowLength + i; - row[i] = decode ? getString(index) : getStringUtf8(index); - } - - return asArray ? row : Arrays.asList(row); - } - } else { - final Object o = decode ? getString(physicalRow) : getStringUtf8(physicalRow); - return asArray ? new Object[]{o} : o; - } - } - - /** - * Returns the value at the given physical row number as a list of ByteBuffers. Only valid when !asArray, i.e., - * when type is {@link ColumnType#STRING}. - * - * @param physicalRow physical row number - */ - private List getRowAsListUtf8(final int physicalRow) - { - if (asArray) { - throw DruidException.defensive("Unexpected call for array column"); - } - - final Object object = getRowAsObject(physicalRow, false); - - if (object == null) { - return Collections.singletonList(null); - } else if (object instanceof List) { - //noinspection unchecked - return (List) object; - } else { - return Collections.singletonList((ByteBuffer) object); - } - } - - /** - * Selector used by this column. It's versatile: it can run as string array (asArray = true) or regular string - * column (asArray = false). - */ - private DimensionSelector makeDimensionSelectorInternal(ReadableOffset offset, @Nullable ExtractionFn extractionFn) { if (multiValue) { class MultiValueSelector implements DimensionSelector @@ -671,41 +401,230 @@ public class StringFrameColumnReader implements FrameColumnReader return new SingleValueSelector(); } } - } - static class StringArrayFrameColumn implements BaseColumn - { - private final StringFrameColumn delegate; - - private StringArrayFrameColumn( - Frame frame, - boolean multiValue, - Memory memory, - long startOfStringLengthSection, - long startOfStringDataSection - ) + @Override + public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(ReadableVectorOffset offset) { - this.delegate = new StringFrameColumn( - frame, - multiValue, - memory, - startOfStringLengthSection, - startOfStringDataSection, - true - ); + // Callers should use object selectors, because we have no dictionary. + throw new UnsupportedOperationException(); } @Override - @SuppressWarnings("rawtypes") - public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + public MultiValueDimensionVectorSelector makeMultiValueDimensionVectorSelector(ReadableVectorOffset vectorOffset) { - return delegate.makeDimensionSelectorInternal(offset, null); + // Callers should use object selectors, because we have no dictionary. + throw new UnsupportedOperationException(); + } + + @Override + public VectorObjectSelector makeVectorObjectSelector(final ReadableVectorOffset offset) + { + class StringFrameVectorObjectSelector implements VectorObjectSelector + { + private final Object[] vector = new Object[offset.getMaxVectorSize()]; + private int id = ReadableVectorInspector.NULL_ID; + + @Override + public Object[] getObjectVector() + { + computeVectorIfNeeded(); + return vector; + } + + @Override + public int getMaxVectorSize() + { + return offset.getMaxVectorSize(); + } + + @Override + public int getCurrentVectorSize() + { + return offset.getCurrentVectorSize(); + } + + private void computeVectorIfNeeded() + { + if (id == offset.getId()) { + return; + } + + if (offset.isContiguous()) { + final int start = offset.getStartOffset(); + + for (int i = 0; i < offset.getCurrentVectorSize(); i++) { + final int physicalRow = frame.physicalRow(i + start); + vector[i] = getRowAsObject(physicalRow, true); + } + } else { + final int[] offsets = offset.getOffsets(); + + for (int i = 0; i < offset.getCurrentVectorSize(); i++) { + final int physicalRow = frame.physicalRow(offsets[i]); + vector[i] = getRowAsObject(physicalRow, true); + } + } + + id = offset.getId(); + } + } + + return new StringFrameVectorObjectSelector(); + } + + @Override + public int length() + { + return frame.numRows(); } @Override public void close() { - delegate.close(); + // Do nothing. + } + + @Override + public ColumnType getType() + { + return ColumnType.STRING; + } + + @Override + public int numRows() + { + return length(); + } + + @Override + protected Object getVal(int rowNum) + { + return getRowAsObject(frame.physicalRow(rowNum), true); + } + + @Override + protected Comparator getComparator() + { + return Comparator.nullsFirst(Comparator.comparing(o -> ((String) o))); + } + + @Override + public int compareRows(int rowNum1, int rowNum2) + { + return ObjectUtils.compare(getStringUtf8(rowNum1), getStringUtf8(rowNum2)); + } + + /** + * Returns a ByteBuffer containing UTF-8 encoded string number {@code index}. The ByteBuffer is always newly + * created, so it is OK to change its position, limit, etc. However, it may point to shared memory, so it is + * not OK to write to its contents. + */ + @Nullable + private ByteBuffer getStringUtf8(final int index) + { + final long dataStart; + final long dataEnd = + startOfStringDataSection + + memory.getInt(startOfStringLengthSection + (long) Integer.BYTES * index); + + if (index == 0) { + dataStart = startOfStringDataSection; + } else { + dataStart = + startOfStringDataSection + + memory.getInt(startOfStringLengthSection + (long) Integer.BYTES * (index - 1)); + } + + final int dataLength = Ints.checkedCast(dataEnd - dataStart); + + if ((dataLength == 0 && NullHandling.replaceWithDefault()) || + (dataLength == 1 && memory.getByte(dataStart) == FrameWriterUtils.NULL_STRING_MARKER)) { + return null; + } + + return FrameReaderUtils.readByteBuffer(memory, dataStart, dataLength); + } + + @Nullable + private String getString(final int index) + { + final ByteBuffer stringUtf8 = getStringUtf8(index); + + if (stringUtf8 == null) { + return null; + } else { + return StringUtils.fromUtf8(stringUtf8); + } + } + + /** + * Returns the object at the given physical row number. + * + * @param physicalRow physical row number + * @param decode if true, return java.lang.String. If false, return UTF-8 ByteBuffer. + */ + @Nullable + private Object getRowAsObject(final int physicalRow, final boolean decode) + { + if (multiValue) { + final int cumulativeRowLength = FrameColumnReaderUtils.getCumulativeRowLength( + memory, + getStartOfCumulativeLengthSection(), + physicalRow + ); + final int rowLength; + + if (FrameColumnReaderUtils.isNullRow(cumulativeRowLength)) { + return null; + } else if (physicalRow == 0) { + rowLength = cumulativeRowLength; + } else { + rowLength = cumulativeRowLength - FrameColumnReaderUtils.getAdjustedCumulativeRowLength( + memory, + getStartOfCumulativeLengthSection(), + physicalRow - 1 + ); + } + + if (rowLength == 0) { + return Collections.emptyList(); + } else if (rowLength == 1) { + final int index = cumulativeRowLength - 1; + final Object o = decode ? getString(index) : getStringUtf8(index); + return o; + } else { + final Object[] row = new Object[rowLength]; + + for (int i = 0; i < rowLength; i++) { + final int index = cumulativeRowLength - rowLength + i; + row[i] = decode ? getString(index) : getStringUtf8(index); + } + + return Arrays.asList(row); + } + } else { + final Object o = decode ? getString(physicalRow) : getStringUtf8(physicalRow); + return o; + } + } + + /** + * Returns the value at the given physical row number as a list of ByteBuffers. + * + * @param physicalRow physical row number + */ + private List getRowAsListUtf8(final int physicalRow) + { + final Object object = getRowAsObject(physicalRow, false); + + if (object == null) { + return Collections.singletonList(null); + } else if (object instanceof List) { + //noinspection unchecked + return (List) object; + } else { + return Collections.singletonList((ByteBuffer) object); + } } } } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java index 82cb092779d..ac5541b893d 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursor.java @@ -19,16 +19,16 @@ package org.apache.druid.frame.segment; -import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorHolderFactory; -import org.apache.druid.frame.segment.row.RowFrameCursorHolderFactory; +import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorFactory; +import org.apache.druid.frame.segment.row.RowFrameCursorFactory; import org.apache.druid.query.BaseQuery; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.SimpleSettableOffset; /** - * An implementation of {@link Cursor} used by {@link RowFrameCursorHolderFactory} - * and {@link ColumnarFrameCursorHolderFactory}. + * An implementation of {@link Cursor} used by {@link RowFrameCursorFactory} + * and {@link ColumnarFrameCursorFactory}. * * Adds the methods {@link #getCurrentRow()} and {@link #setCurrentRow(int)} so the cursor can be moved to * particular rows. diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameSegment.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameSegment.java index 14b2b0ec682..3df54bc184d 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameSegment.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameSegment.java @@ -23,9 +23,9 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumns; import org.apache.druid.segment.CloseableShapeshifter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -70,9 +70,9 @@ public class FrameSegment implements Segment } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return new FrameStorageAdapter(frame, frameReader, segmentId.getInterval()); + return frameReader.makeCursorFactory(frame); } @Override diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java deleted file mode 100644 index 51ef56928d4..00000000000 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameStorageAdapter.java +++ /dev/null @@ -1,130 +0,0 @@ -/* - * 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.segment; - -import org.apache.druid.frame.Frame; -import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.CursorHolderFactory; -import org.apache.druid.segment.DimensionDictionarySelector; -import org.apache.druid.segment.Metadata; -import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.data.Indexed; -import org.apache.druid.segment.data.ListIndexed; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.Collections; - -/** - * A {@link StorageAdapter} implementation based on a single {@link Frame}. - * - * This class is used for both columnar and row-based frames. - */ -public class FrameStorageAdapter implements StorageAdapter -{ - private final Frame frame; - private final FrameReader frameReader; - private final Interval interval; - private final CursorHolderFactory cursorFactory; - - public FrameStorageAdapter(Frame frame, FrameReader frameReader, Interval interval) - { - this.frame = frame; - this.frameReader = frameReader; - this.interval = interval; - this.cursorFactory = frameReader.makeCursorHolderFactory(frame); - } - - @Override - public Interval getInterval() - { - return interval; - } - - @Override - public RowSignature getRowSignature() - { - return frameReader.signature(); - } - - @Override - public Indexed getAvailableDimensions() - { - return new ListIndexed<>(frameReader.signature().getColumnNames()); - } - - @Override - public Iterable getAvailableMetrics() - { - return Collections.emptyList(); - } - - @Override - public int getDimensionCardinality(String column) - { - return DimensionDictionarySelector.CARDINALITY_UNKNOWN; - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - // It's ok to return null always, because callers are required to handle the case where the min value is not known. - return null; - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - // It's ok to return null always, because callers are required to handle the case where the max value is not known. - return null; - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - return frameReader.columnCapabilities(frame, column); - } - - @Override - public int getNumRows() - { - return frame.numRows(); - } - - @Override - @Nullable - public Metadata getMetadata() - { - return null; - } - - @Override - public CursorHolder makeCursorHolder(CursorBuildSpec spec) - { - return cursorFactory.makeCursorHolder(spec); - } -} diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorFactory.java similarity index 86% rename from processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java rename to processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorFactory.java index ee1a01f17fe..550cbaf88df 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorHolderFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/ColumnarFrameCursorFactory.java @@ -25,7 +25,7 @@ import org.apache.druid.frame.read.columnar.FrameColumnReader; import org.apache.druid.frame.segment.FrameCursor; import org.apache.druid.frame.segment.FrameCursorUtils; import org.apache.druid.frame.segment.FrameFilteredOffset; -import org.apache.druid.frame.segment.row.RowFrameCursorHolderFactory; +import org.apache.druid.frame.segment.row.RowFrameCursorFactory; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.Order; @@ -35,11 +35,12 @@ import org.apache.druid.query.filter.vector.VectorValueMatcher; import org.apache.druid.segment.ColumnCache; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.CursorHolderFactory; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleSettableOffset; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.vector.FilteredVectorOffset; import org.apache.druid.segment.vector.NoFilterVectorOffset; @@ -54,19 +55,19 @@ import java.util.Collections; import java.util.List; /** - * A {@link CursorHolderFactory} implementation based on a single columnar {@link Frame}. + * A {@link CursorFactory} implementation based on a single columnar {@link Frame}. * * This class is only used for columnar frames. It is not used for row-based frames. * - * @see RowFrameCursorHolderFactory the row-based version + * @see RowFrameCursorFactory the row-based version */ -public class ColumnarFrameCursorHolderFactory implements CursorHolderFactory +public class ColumnarFrameCursorFactory implements CursorFactory { private final Frame frame; private final RowSignature signature; private final List columnReaders; - public ColumnarFrameCursorHolderFactory( + public ColumnarFrameCursorFactory( final Frame frame, final RowSignature signature, final List columnReaders @@ -179,4 +180,25 @@ public class ColumnarFrameCursorHolderFactory implements CursorHolderFactory } }; } + + @Override + public RowSignature getRowSignature() + { + return signature; + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + final int columnNumber = signature.indexOf(column); + + if (columnNumber < 0) { + return null; + } else { + // Better than frameReader.frameSignature().getColumnCapabilities(columnName), because this method has more + // insight into what's actually going on with this column (nulls, multivalue, etc). + return columnReaders.get(columnNumber).readColumn(frame).getCapabilities(); + } + } } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java index d3a03ca83bb..5cf38583079 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java @@ -43,7 +43,7 @@ import java.util.Map; * A {@link QueryableIndex} implementation based on a single columnar {@link Frame}. There is no internal caching * of columns here, so callers should generally wrap this in a {@link org.apache.druid.segment.ColumnCache}. * - * This class exists so {@link ColumnarFrameCursorHolderFactory} can reuse code meant for regular segment-backed + * This class exists so {@link ColumnarFrameCursorFactory} can reuse code meant for regular segment-backed * {@link QueryableIndex}. Some methods are implemented by throwing {@link UnsupportedOperationException}, wherever * it is not expected that those methods are actually going to be needed. */ diff --git a/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorFactory.java similarity index 85% rename from processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java rename to processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorFactory.java index f84eb62a4fa..0132ab720a0 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorHolderFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/row/RowFrameCursorFactory.java @@ -26,35 +26,37 @@ import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.segment.FrameCursor; import org.apache.druid.frame.segment.FrameCursorUtils; import org.apache.druid.frame.segment.FrameFilteredOffset; -import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorHolderFactory; +import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorFactory; import org.apache.druid.query.OrderBy; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.CursorHolderFactory; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleSettableOffset; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; import java.util.Collections; import java.util.List; /** - * A {@link CursorHolderFactory} implementation based on a single row-based {@link Frame}. + * A {@link CursorFactory} implementation based on a single row-based {@link Frame}. * * This class is only used for row-based frames. * - * @see ColumnarFrameCursorHolderFactory the columnar version + * @see ColumnarFrameCursorFactory the columnar version */ -public class RowFrameCursorHolderFactory implements CursorHolderFactory +public class RowFrameCursorFactory implements CursorFactory { private final Frame frame; private final FrameReader frameReader; private final List fieldReaders; - public RowFrameCursorHolderFactory( + public RowFrameCursorFactory( final Frame frame, final FrameReader frameReader, final List fieldReaders @@ -115,4 +117,17 @@ public class RowFrameCursorHolderFactory implements CursorHolderFactory } }; } + + @Override + public RowSignature getRowSignature() + { + return frameReader.signature(); + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + return frameReader.signature().getColumnCapabilities(column); + } } diff --git a/processing/src/main/java/org/apache/druid/frame/write/RowBasedFrameWriter.java b/processing/src/main/java/org/apache/druid/frame/write/RowBasedFrameWriter.java index beae50dca9d..c33b4c32b8a 100644 --- a/processing/src/main/java/org/apache/druid/frame/write/RowBasedFrameWriter.java +++ b/processing/src/main/java/org/apache/druid/frame/write/RowBasedFrameWriter.java @@ -23,6 +23,7 @@ import com.google.common.base.Throwables; import com.google.common.primitives.Ints; import org.apache.datasketches.memory.Memory; import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.error.DruidException; import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.allocation.AppendableMemory; @@ -313,10 +314,22 @@ public class RowBasedFrameWriter implements FrameWriter // Reset to beginning of loop. i = -1; + final int priorAllocation = BASE_DATA_ALLOCATION_SIZE * reserveMultiple; + // Try again with a bigger allocation. reserveMultiple *= 2; - if (!dataMemory.reserveAdditional(Ints.checkedCast((long) BASE_DATA_ALLOCATION_SIZE * reserveMultiple))) { + final int nextAllocation = Math.min( + dataMemory.availableToReserve(), + Ints.checkedCast((long) BASE_DATA_ALLOCATION_SIZE * reserveMultiple) + ); + + if (nextAllocation > priorAllocation) { + if (!dataMemory.reserveAdditional(nextAllocation)) { + // Shouldn't see this unless availableToReserve lied to us. + throw DruidException.defensive("Unexpected failure of dataMemory.reserveAdditional"); + } + } else { return false; } diff --git a/processing/src/main/java/org/apache/druid/frame/write/columnar/StringFrameColumnWriter.java b/processing/src/main/java/org/apache/druid/frame/write/columnar/StringFrameColumnWriter.java index 8eee0fd0cef..75f64613388 100644 --- a/processing/src/main/java/org/apache/druid/frame/write/columnar/StringFrameColumnWriter.java +++ b/processing/src/main/java/org/apache/druid/frame/write/columnar/StringFrameColumnWriter.java @@ -44,6 +44,9 @@ public abstract class StringFrameColumnWriter imp public static final long DATA_OFFSET = 1 /* type code */ + 1 /* single or multi-value? */; + public static final byte MULTI_VALUE_BYTE = (byte) 0x01; + public static final long MULTI_VALUE_POSITION = 1; + private final T selector; private final byte typeCode; protected final ColumnCapabilities.Capable multiValue; @@ -228,7 +231,7 @@ public abstract class StringFrameColumnWriter imp long currentPosition = startPosition; memory.putByte(currentPosition, typeCode); - memory.putByte(currentPosition + 1, writeMultiValue ? (byte) 1 : (byte) 0); + memory.putByte(currentPosition + 1, writeMultiValue ? MULTI_VALUE_BYTE : (byte) 0); currentPosition += 2; if (writeMultiValue) { diff --git a/processing/src/main/java/org/apache/druid/guice/ExpressionModule.java b/processing/src/main/java/org/apache/druid/guice/ExpressionModule.java index 917cf967f14..e1064234e56 100644 --- a/processing/src/main/java/org/apache/druid/guice/ExpressionModule.java +++ b/processing/src/main/java/org/apache/druid/guice/ExpressionModule.java @@ -81,6 +81,7 @@ public class ExpressionModule implements Module .add(HyperUniqueExpressions.HllEstimateExprMacro.class) .add(HyperUniqueExpressions.HllRoundEstimateExprMacro.class) .add(NestedDataExpressions.JsonObjectExprMacro.class) + .add(NestedDataExpressions.JsonMergeExprMacro.class) .add(NestedDataExpressions.JsonKeysExprMacro.class) .add(NestedDataExpressions.JsonPathsExprMacro.class) .add(NestedDataExpressions.JsonValueExprMacro.class) diff --git a/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java b/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java index 30cc388f1d9..cad8fdfd831 100644 --- a/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java +++ b/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java @@ -37,7 +37,6 @@ import org.apache.druid.query.FrameBasedInlineDataSourceSerializer; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.context.ResponseContextDeserializer; import org.apache.druid.query.rowsandcols.RowsAndColumns; -import org.apache.druid.query.rowsandcols.semantic.WireTransferable; import org.joda.time.DateTimeZone; import java.io.IOException; @@ -189,20 +188,7 @@ public class DruidDefaultSerializersModule extends SimpleModule ); addDeserializer(ResponseContext.class, new ResponseContextDeserializer()); - addSerializer(RowsAndColumns.class, new JsonSerializer() - { - @Override - public void serialize( - RowsAndColumns value, - JsonGenerator gen, - SerializerProvider serializers - ) throws IOException - { - // It would be really cool if jackson offered an output stream that would allow us to push bytes - // through, but it doesn't right now, so we have to build a byte[] instead. Maybe something to contribute - // back to Jackson at some point. - gen.writeBinary(WireTransferable.fromRAC(value).bytesToTransfer()); - } - }); + addSerializer(RowsAndColumns.class, new RowsAndColumns.RowsAndColumnsSerializer()); + addDeserializer(RowsAndColumns.class, new RowsAndColumns.RowsAndColumnsDeserializer()); } } diff --git a/processing/src/main/java/org/apache/druid/java/util/common/parsers/AbstractFlatTextFormatParser.java b/processing/src/main/java/org/apache/druid/java/util/common/parsers/AbstractFlatTextFormatParser.java index 1826b73857a..4e9ff77cdf4 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/parsers/AbstractFlatTextFormatParser.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/parsers/AbstractFlatTextFormatParser.java @@ -52,7 +52,7 @@ public abstract class AbstractFlatTextFormatParser implements Parser valueFunction; + private final Function transformationFunction; private final boolean hasHeaderRow; private final int maxSkipHeaderRows; @@ -64,11 +64,16 @@ public abstract class AbstractFlatTextFormatParser implements Parser fieldNames, final boolean hasHeaderRow, - final int maxSkipHeaderRows + final int maxSkipHeaderRows, + final boolean tryParseNumbers ) { - this(listDelimiter, hasHeaderRow, maxSkipHeaderRows); + this(listDelimiter, hasHeaderRow, maxSkipHeaderRows, tryParseNumbers); setFieldNames(fieldNames); } @@ -56,7 +58,7 @@ public class CSVParser extends AbstractFlatTextFormatParser @VisibleForTesting CSVParser(@Nullable final String listDelimiter, final String header) { - this(listDelimiter, false, 0); + this(listDelimiter, false, 0, false); setFieldNames(header); } diff --git a/processing/src/main/java/org/apache/druid/java/util/common/parsers/DelimitedParser.java b/processing/src/main/java/org/apache/druid/java/util/common/parsers/DelimitedParser.java index df75cc3f734..c69af8974f8 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/parsers/DelimitedParser.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/parsers/DelimitedParser.java @@ -35,10 +35,11 @@ public class DelimitedParser extends AbstractFlatTextFormatParser @Nullable final String delimiter, @Nullable final String listDelimiter, final boolean hasHeaderRow, - final int maxSkipHeaderRows + final int maxSkipHeaderRows, + final boolean tryParseNumbers ) { - super(listDelimiter, hasHeaderRow, maxSkipHeaderRows); + super(listDelimiter, hasHeaderRow, maxSkipHeaderRows, tryParseNumbers); final String finalDelimiter = delimiter != null ? delimiter : FlatTextFormat.DELIMITED.getDefaultDelimiter(); Preconditions.checkState( @@ -55,10 +56,11 @@ public class DelimitedParser extends AbstractFlatTextFormatParser @Nullable final String listDelimiter, final Iterable fieldNames, final boolean hasHeaderRow, - final int maxSkipHeaderRows + final int maxSkipHeaderRows, + final boolean tryParseNumbers ) { - this(delimiter, listDelimiter, hasHeaderRow, maxSkipHeaderRows); + this(delimiter, listDelimiter, hasHeaderRow, maxSkipHeaderRows, tryParseNumbers); setFieldNames(fieldNames); } @@ -66,7 +68,7 @@ public class DelimitedParser extends AbstractFlatTextFormatParser @VisibleForTesting DelimitedParser(@Nullable final String delimiter, @Nullable final String listDelimiter, final String header) { - this(delimiter, listDelimiter, false, 0); + this(delimiter, listDelimiter, false, 0, false); setFieldNames(header); } diff --git a/processing/src/main/java/org/apache/druid/java/util/common/parsers/ParserUtils.java b/processing/src/main/java/org/apache/druid/java/util/common/parsers/ParserUtils.java index 045fec91d86..b716737d15e 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/parsers/ParserUtils.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/parsers/ParserUtils.java @@ -22,6 +22,9 @@ package org.apache.druid.java.util.common.parsers; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Splitter; +import com.google.common.primitives.Doubles; +import com.google.common.primitives.Longs; +import org.apache.commons.lang.math.NumberUtils; import org.apache.druid.common.config.NullHandling; import org.joda.time.DateTimeZone; @@ -52,22 +55,62 @@ public class ParserUtils } } - public static Function getMultiValueFunction( + /** + * @return a transformation function on an input value. The function performs the following transformations on the input string: + *
  • Splits it into multiple values using the {@code listSplitter} if the {@code list delimiter} is present in the input.
  • + *
  • If {@code tryParseNumbers} is true, the function will also attempt to parse any numeric values present in the input: + * integers as {@code Long} and floating-point numbers as {@code Double}. If the input is not a number or parsing fails, the input + * is returned as-is as a string.
  • + */ + public static Function getTransformationFunction( final String listDelimiter, - final Splitter listSplitter + final Splitter listSplitter, + final boolean tryParseNumbers ) { return (input) -> { - if (input != null && input.contains(listDelimiter)) { - return StreamSupport.stream(listSplitter.split(input).spliterator(), false) - .map(NullHandling::emptyToNullIfNeeded) - .collect(Collectors.toList()); - } else { + if (input == null) { return NullHandling.emptyToNullIfNeeded(input); } + + if (input.contains(listDelimiter)) { + return StreamSupport.stream(listSplitter.split(input).spliterator(), false) + .map(NullHandling::emptyToNullIfNeeded) + .map(value -> tryParseNumbers ? ParserUtils.tryParseStringAsNumber(value) : value) + .collect(Collectors.toList()); + } else { + return tryParseNumbers ? + tryParseStringAsNumber(input) : + NullHandling.emptyToNullIfNeeded(input); + + } }; } + /** + * Attempts to parse the input string into a numeric value, if applicable. If the input is a number, the method first + * tries to parse the input number as a {@code Long}. If parsing as a {@code Long} fails, it then attempts to parse + * the input number as a {@code Double}. For all other scenarios, the input is returned as-is as a {@code String} type. + */ + @Nullable + private static Object tryParseStringAsNumber(@Nullable final String input) + { + if (!NumberUtils.isNumber(input)) { + return NullHandling.emptyToNullIfNeeded(input); + } + + final Long l = Longs.tryParse(input); + if (l != null) { + return l; + } + final Double d = Doubles.tryParse(input); + if (d != null) { + return d; + } + // fall back to given input if we cannot parse the input as a Long & Double for whatever reason + return input; + } + public static ArrayList generateFieldNames(int length) { final ArrayList names = new ArrayList<>(length); diff --git a/processing/src/main/java/org/apache/druid/java/util/metrics/NoopOshiSysMonitor.java b/processing/src/main/java/org/apache/druid/java/util/metrics/NoopOshiSysMonitor.java index d44390ac8c1..9a693f7bfdf 100644 --- a/processing/src/main/java/org/apache/druid/java/util/metrics/NoopOshiSysMonitor.java +++ b/processing/src/main/java/org/apache/druid/java/util/metrics/NoopOshiSysMonitor.java @@ -19,13 +19,15 @@ package org.apache.druid.java.util.metrics; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.emitter.service.ServiceEmitter; public class NoopOshiSysMonitor extends OshiSysMonitor { public NoopOshiSysMonitor() { - super(); + super(ImmutableMap.of(), new OshiSysMonitorConfig(ImmutableList.of())); } @Override diff --git a/processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitor.java b/processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitor.java index ecc1ff0d97e..130c353c8ad 100644 --- a/processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitor.java +++ b/processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitor.java @@ -19,6 +19,7 @@ package org.apache.druid.java.util.metrics; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -39,6 +40,7 @@ import oshi.software.os.OperatingSystem; import java.util.List; import java.util.Map; +import java.util.function.Consumer; /** * SysMonitor implemented using {@link oshi} @@ -72,43 +74,31 @@ public class OshiSysMonitor extends FeedDefiningMonitor private final TcpStats tcpStats; private final Map dimensions; + private final OshiSysMonitorConfig config; + private final Map> monitoringFunctions = ImmutableMap.of( + "mem", this::monitorMemStats, + "swap", this::monitorSwapStats, + "fs", this::monitorFsStats, + "disk", this::monitorDiskStats, + "net", this::monitorNetStats, + "cpu", this::monitorCpuStats, + "sys", this::monitorSysStats, + "tcp", this::monitorTcpStats + ); - public OshiSysMonitor() + public OshiSysMonitor(Map dimensions, OshiSysMonitorConfig config) { - this(ImmutableMap.of()); - } - - public OshiSysMonitor(Map dimensions) - { - this(dimensions, DEFAULT_METRICS_FEED); - } - - public OshiSysMonitor(Map dimensions, String feed) - { - super(feed); - Preconditions.checkNotNull(dimensions); - this.dimensions = ImmutableMap.copyOf(dimensions); - - this.si = new SystemInfo(); - this.hal = si.getHardware(); - this.os = si.getOperatingSystem(); - - this.memStats = new MemStats(); - this.swapStats = new SwapStats(); - this.fsStats = new FsStats(); - this.diskStats = new DiskStats(); - this.netStats = new NetStats(); - this.cpuStats = new CpuStats(); - this.sysStats = new SysStats(); - this.tcpStats = new TcpStats(); - + this(dimensions, config, new SystemInfo()); } // Create an object with mocked systemInfo for testing purposes - public OshiSysMonitor(SystemInfo systemInfo) + @VisibleForTesting + public OshiSysMonitor(Map dimensions, OshiSysMonitorConfig config, SystemInfo systemInfo) { - super("metrics"); - this.dimensions = ImmutableMap.of(); + super(DEFAULT_METRICS_FEED); + Preconditions.checkNotNull(dimensions); + this.dimensions = ImmutableMap.copyOf(dimensions); + this.config = config; this.si = systemInfo; this.hal = si.getHardware(); @@ -127,14 +117,11 @@ public class OshiSysMonitor extends FeedDefiningMonitor @Override public boolean doMonitor(ServiceEmitter emitter) { - monitorMemStats(emitter); - monitorSwapStats(emitter); - monitorFsStats(emitter); - monitorDiskStats(emitter); - monitorNetStats(emitter); - monitorCpuStats(emitter); - monitorSysStats(emitter); - monitorTcpStats(emitter); + monitoringFunctions.forEach((key, function) -> { + if (config.shouldEmitMetricCategory(key)) { + function.accept(emitter); + } + }); return true; } diff --git a/processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitorConfig.java b/processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitorConfig.java new file mode 100644 index 00000000000..e8fba8c346b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitorConfig.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableSet; + +import javax.validation.constraints.NotNull; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class OshiSysMonitorConfig +{ + public static final String PREFIX = "druid.monitoring.sys"; + + @JsonProperty("categories") + @NotNull + private Set categories; + + public OshiSysMonitorConfig(@JsonProperty("categories") List categories) + { + this.categories = categories == null ? new HashSet<>() : ImmutableSet.copyOf(categories); + } + + public boolean shouldEmitMetricCategory(String category) + { + return categories.isEmpty() || categories.contains(category); + } +} diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java b/processing/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java index efe0328a837..16ae5024830 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java @@ -91,10 +91,7 @@ public class ExpressionTypeConversion return type; } if (type.isArray() || other.isArray()) { - if (!Objects.equals(type, other)) { - throw new Types.IncompatibleTypeException(type, other); - } - return type; + return leastRestrictiveType(type, other); } if (type.is(ExprType.COMPLEX) || other.is(ExprType.COMPLEX)) { if (type.getComplexTypeName() == null) { @@ -134,12 +131,8 @@ public class ExpressionTypeConversion if (other == null) { return type; } - // arrays cannot be auto converted if (type.isArray() || other.isArray()) { - if (!Objects.equals(type, other)) { - throw new Types.IncompatibleTypeException(type, other); - } - return type; + return leastRestrictiveType(type, other); } if (type.is(ExprType.COMPLEX) || other.is(ExprType.COMPLEX)) { if (type.getComplexTypeName() == null) { diff --git a/processing/src/main/java/org/apache/druid/query/FilteredDataSource.java b/processing/src/main/java/org/apache/druid/query/FilteredDataSource.java index 62b42b15c78..2d3104a0041 100644 --- a/processing/src/main/java/org/apache/druid/query/FilteredDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/FilteredDataSource.java @@ -25,9 +25,8 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.planning.DataSourceAnalysis; -import org.apache.druid.segment.FilteredStorageAdapter; +import org.apache.druid.segment.FilteredSegment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.WrappedSegmentReference; import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; @@ -135,12 +134,7 @@ public class FilteredDataSource implements DataSource ); return JvmUtils.safeAccumulateThreadCpuTime( cpuTimeAccumulator, - () -> - baseSegment -> - new WrappedSegmentReference( - segmentMapFn.apply(baseSegment), - storageAdapter -> new FilteredStorageAdapter(storageAdapter, filter) - ) + () -> baseSegment -> new FilteredSegment(segmentMapFn.apply(baseSegment), filter) ); } diff --git a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java index 38a8b1e91d5..68d3db9a36d 100644 --- a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java @@ -21,9 +21,7 @@ package org.apache.druid.query; import org.apache.druid.frame.Frame; import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.planning.DataSourceAnalysis; @@ -88,10 +86,9 @@ public class FrameBasedInlineDataSource implements DataSource Frame frame = frameSignaturePair.getFrame(); RowSignature frameSignature = frameSignaturePair.getRowSignature(); FrameReader frameReader = FrameReader.create(frameSignature); - final CursorHolder holder = - new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY).makeCursorHolder( - CursorBuildSpec.FULL_SCAN - ); + final CursorHolder holder = frameReader.makeCursorFactory(frame).makeCursorHolder( + CursorBuildSpec.FULL_SCAN + ); return Sequences.simple(Collections.singletonList(holder.asCursor())).withBaggage(holder); } ); diff --git a/processing/src/main/java/org/apache/druid/query/NoopQueryProcessingPool.java b/processing/src/main/java/org/apache/druid/query/NoopQueryProcessingPool.java new file mode 100644 index 00000000000..efb9a53776a --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/NoopQueryProcessingPool.java @@ -0,0 +1,134 @@ +/* + * 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.query; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.error.DruidException; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +/** + * Implementation of {@link QueryProcessingPool} that throws when any query execution task unit is submitted to it. It is + * semantically shutdown from the moment it is created, and since the shutdown methods are supposed to be idempotent, + * they do not throw like the execution methods + */ +public class NoopQueryProcessingPool implements QueryProcessingPool +{ + private static final NoopQueryProcessingPool INSTANCE = new NoopQueryProcessingPool(); + + public static NoopQueryProcessingPool instance() + { + return INSTANCE; + } + + @Override + public ListenableFuture submitRunnerTask(PrioritizedQueryRunnerCallable task) + { + throw unsupportedException(); + } + + @Override + public ListenableFuture submit(Callable callable) + { + throw unsupportedException(); + } + + @Override + public ListenableFuture submit(Runnable runnable) + { + throw unsupportedException(); + } + + @Override + public ListenableFuture submit(Runnable runnable, T t) + { + throw unsupportedException(); + } + + @Override + public List> invokeAll(Collection> collection) + { + throw unsupportedException(); + } + + @Override + public List> invokeAll(Collection> collection, long l, TimeUnit timeUnit) + { + throw unsupportedException(); + } + + @Override + public void shutdown() + { + // No op, since it is already shutdown + } + + @Override + public List shutdownNow() + { + return Collections.emptyList(); + } + + @Override + public boolean isShutdown() + { + return true; + } + + @Override + public boolean isTerminated() + { + return true; + } + + @Override + public boolean awaitTermination(long l, TimeUnit timeUnit) + { + return true; + } + + @Override + public T invokeAny(Collection> collection) + { + throw unsupportedException(); + } + + @Override + public T invokeAny(Collection> collection, long l, TimeUnit timeUnit) + { + throw unsupportedException(); + } + + @Override + public void execute(Runnable runnable) + { + throw unsupportedException(); + } + + private DruidException unsupportedException() + { + return DruidException.defensive("Unexpected call made to NoopQueryProcessingPool"); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/QueryContext.java b/processing/src/main/java/org/apache/druid/query/QueryContext.java index 8b2f7e27e68..52395275967 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContext.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContext.java @@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.QueryContexts.Vectorize; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.TypedInFilter; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import javax.annotation.Nullable; import java.io.IOException; @@ -341,7 +340,7 @@ public class QueryContext public int getVectorSize() { - return getVectorSize(QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE); + return getVectorSize(QueryContexts.DEFAULT_VECTOR_SIZE); } public int getVectorSize(int defaultSize) diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index d3545a8494a..ced9f0d4e2d 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -63,6 +63,7 @@ public class QueryContexts public static final String REWRITE_JOIN_TO_FILTER_ENABLE_KEY = "enableRewriteJoinToFilter"; public static final String JOIN_FILTER_REWRITE_MAX_SIZE_KEY = "joinFilterRewriteMaxSize"; public static final String MAX_NUMERIC_IN_FILTERS = "maxNumericInFilters"; + public static final String CURSOR_AUTO_ARRANGE_FILTERS = "cursorAutoArrangeFilters"; // This flag controls whether a SQL join query with left scan should be attempted to be run as direct table access // instead of being wrapped inside a query. With direct table access enabled, Druid can push down the join operation to // data servers. @@ -106,6 +107,7 @@ public class QueryContexts public static final boolean DEFAULT_USE_RESULTLEVEL_CACHE = true; public static final Vectorize DEFAULT_VECTORIZE = Vectorize.TRUE; public static final Vectorize DEFAULT_VECTORIZE_VIRTUAL_COLUMN = Vectorize.TRUE; + public static final int DEFAULT_VECTOR_SIZE = 512; public static final int DEFAULT_PRIORITY = 0; public static final int DEFAULT_UNCOVERED_INTERVALS_LIMIT = 0; public static final long DEFAULT_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5); diff --git a/processing/src/main/java/org/apache/druid/query/ResultSerializationMode.java b/processing/src/main/java/org/apache/druid/query/ResultSerializationMode.java new file mode 100644 index 00000000000..3d42e8cbc82 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/ResultSerializationMode.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.query; + +/** + * Serialization medium of the query results on the broker. It is currently used to communicate the result's format between + * the main query processing walker and the individual toolchests while materializing subquery's rows + */ +public enum ResultSerializationMode +{ + /** + * Materialize the inner results as rows + */ + ROWS, + + /** + * Materialize the inner results as frames + */ + FRAMES; + + public static final String CTX_SERIALIZATION_PARAMETER = "serialization"; +} diff --git a/processing/src/main/java/org/apache/druid/query/UnnestDataSource.java b/processing/src/main/java/org/apache/druid/query/UnnestDataSource.java index b330ba5ce19..06acca13a75 100644 --- a/processing/src/main/java/org/apache/druid/query/UnnestDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/UnnestDataSource.java @@ -26,9 +26,8 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.UnnestStorageAdapter; +import org.apache.druid.segment.UnnestSegment; import org.apache.druid.segment.VirtualColumn; -import org.apache.druid.segment.WrappedSegmentReference; import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; @@ -146,12 +145,7 @@ public class UnnestDataSource implements DataSource ); return JvmUtils.safeAccumulateThreadCpuTime( cpuTimeAccumulator, - () -> - baseSegment -> - new WrappedSegmentReference( - segmentMapFn.apply(baseSegment), - storageAdapter -> new UnnestStorageAdapter(storageAdapter, virtualColumn, unnestFilter) - ) + () -> baseSegment -> new UnnestSegment(segmentMapFn.apply(baseSegment), virtualColumn, unnestFilter) ); } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java index c4c9a7875ef..4d12327c896 100755 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java @@ -454,4 +454,13 @@ public class AggregatorUtil } return false; } + + public static List getCombiningAggregators(List aggs) + { + List combining = new ArrayList<>(aggs.size()); + for (AggregatorFactory agg : aggs) { + combining.add(agg.getCombiningFactory()); + } + return combining; + } } diff --git a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java index c24f6f4f90a..2b4a0cbb571 100644 --- a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryRunnerFactory.java @@ -85,7 +85,7 @@ public class DataSourceMetadataQueryRunnerFactory public DataSourceMetadataQueryRunner(Segment segment) { - this.segmentInterval = segment.asStorageAdapter().getInterval(); + this.segmentInterval = segment.getDataInterval(); this.inspector = segment.as(MaxIngestedEventTimeInspector.class); } diff --git a/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java index 873b4f83188..0926ce78e0a 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java +++ b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java @@ -22,6 +22,7 @@ package org.apache.druid.query.expression; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectReader; import org.apache.druid.guice.annotations.Json; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; @@ -99,6 +100,117 @@ public class NestedDataExpressions } } + public static class JsonMergeExprMacro implements ExprMacroTable.ExprMacro + { + public static final String NAME = "json_merge"; + + private final ObjectMapper jsonMapper; + + @Inject + public JsonMergeExprMacro( + @Json ObjectMapper jsonMapper + ) + { + this.jsonMapper = jsonMapper; + } + + @Override + public String name() + { + return NAME; + } + + @Override + public Expr apply(List args) + { + if (args.size() < 2) { + throw validationFailed("must have at least two arguments"); + } + + final class ParseJsonExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr + { + public ParseJsonExpr(List args) + { + super(JsonMergeExprMacro.this, args); + } + + @Override + public ExprEval eval(ObjectBinding bindings) + { + ExprEval arg = args.get(0).eval(bindings); + Object obj; + + if (arg.value() == null) { + throw JsonMergeExprMacro.this.validationFailed( + "invalid input expected %s but got %s instead", + ExpressionType.STRING, + arg.type() + ); + } + + try { + obj = jsonMapper.readValue(getArgAsJson(arg), Object.class); + } + catch (JsonProcessingException e) { + throw JsonMergeExprMacro.this.processingFailed(e, "bad string input [%s]", arg.asString()); + } + + ObjectReader updater = jsonMapper.readerForUpdating(obj); + + for (int i = 1; i < args.size(); i++) { + ExprEval argSub = args.get(i).eval(bindings); + + try { + String str = getArgAsJson(argSub); + if (str != null) { + obj = updater.readValue(str); + } + } + catch (JsonProcessingException e) { + throw JsonMergeExprMacro.this.processingFailed(e, "bad string input [%s]", argSub.asString()); + } + } + + return ExprEval.ofComplex(ExpressionType.NESTED_DATA, obj); + } + + @Nullable + @Override + public ExpressionType getOutputType(InputBindingInspector inspector) + { + return ExpressionType.NESTED_DATA; + } + + private String getArgAsJson(ExprEval arg) + { + if (arg.value() == null) { + return null; + } + + if (arg.type().is(ExprType.STRING)) { + return arg.asString(); + } + + if (arg.type().is(ExprType.COMPLEX)) { + try { + return jsonMapper.writeValueAsString(unwrap(arg)); + } + catch (JsonProcessingException e) { + throw JsonMergeExprMacro.this.processingFailed(e, "bad complex input [%s]", arg.asString()); + } + } + + throw JsonMergeExprMacro.this.validationFailed( + "invalid input expected %s but got %s instead", + ExpressionType.STRING, + arg.type() + ); + } + } + return new ParseJsonExpr(args); + } + } + public static class ToJsonStringExprMacro implements ExprMacroTable.ExprMacro { public static final String NAME = "to_json_string"; diff --git a/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java index 955553d2dd3..bcd0c514a00 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java @@ -77,7 +77,7 @@ public interface DimFilter extends Cacheable * * The Filter returned by this method across multiple calls must be the same object: parts of the query stack * compare Filters, and returning the same object allows these checks to avoid deep comparisons. - * (see {@link org.apache.druid.segment.join.HashJoinSegmentStorageAdapter#makeCursorHolder(CursorBuildSpec)} + * (see {@link org.apache.druid.segment.join.HashJoinSegmentCursorFactory#makeCursorHolder(CursorBuildSpec)} * for an example} * * @param mayIncludeUnknown whether the optimized filter may need to operate in "includeUnknown" mode. diff --git a/processing/src/main/java/org/apache/druid/query/filter/Filter.java b/processing/src/main/java/org/apache/druid/query/filter/Filter.java index f30681c8669..147406b2261 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/Filter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/Filter.java @@ -47,27 +47,26 @@ public interface Filter * cursor. If both are set, the cursor will effectively perform a logical AND to combine them. * See {@link FilterBundle} for additional details. * - * @param columnIndexSelector - provides {@link org.apache.druid.segment.column.ColumnIndexSupplier} to fetch column - * indexes and {@link org.apache.druid.collections.bitmap.BitmapFactory} to manipulate - * them - * @param bitmapResultFactory - wrapper for {@link ImmutableBitmap} operations to tie into - * {@link org.apache.druid.query.QueryMetrics} and build the output indexes - * @param applyRowCount - upper bound on number of rows this filter would be applied to, after removing rows - * short-circuited by prior bundle operations. For example, given "x AND y", if "x" is - * resolved using an index, then "y" will receive the number of rows that matched - * the filter "x". As another example, given "x OR y", if "x" is resolved using an - * index, then "y" will receive the number of rows that did *not* match the filter "x". - * @param totalRowCount - total number of rows to be scanned if no indexes are applied - * @param includeUnknown - mapping for Druid native two state logic system into SQL three-state logic system. If - * set to true, bitmaps returned by this method should include true bits for any rows - * where the matching result is 'unknown', such as from the input being null valued. - * See {@link NullHandling#useThreeValueLogic()} - * @return - {@link FilterBundle} containing any indexes and/or matchers that are needed to build - * a cursor - * @param - Type of {@link BitmapResultFactory} results, {@link ImmutableBitmap} by default + * @param filterBundleBuilder contains {@link BitmapColumnIndex} and {@link ColumnIndexSelector}, and some additional + * info needed. + * @param bitmapResultFactory wrapper for {@link ImmutableBitmap} operations to tie into + * {@link org.apache.druid.query.QueryMetrics} and build the output indexes + * @param applyRowCount upper bound on number of rows this filter would be applied to, after removing rows + * short-circuited by prior bundle operations. For example, given "x AND y", if "x" is + * resolved using an index, then "y" will receive the number of rows that matched + * the filter "x". As another example, given "x OR y", if "x" is resolved using an + * index, then "y" will receive the number of rows that did *not* match the filter "x". + * @param totalRowCount total number of rows to be scanned if no indexes are applied + * @param includeUnknown mapping for Druid native two state logic system into SQL three-state logic system. If + * set to true, bitmaps returned by this method should include true bits for any rows + * where the matching result is 'unknown', such as from the input being null valued. + * See {@link NullHandling#useThreeValueLogic()} + * @param type of {@link BitmapResultFactory} results, {@link ImmutableBitmap} by default + * @return {@link FilterBundle} containing any indexes and/or matchers that are needed to build + * a cursor */ default FilterBundle makeFilterBundle( - ColumnIndexSelector columnIndexSelector, + FilterBundle.Builder filterBundleBuilder, BitmapResultFactory bitmapResultFactory, int applyRowCount, int totalRowCount, @@ -76,7 +75,7 @@ public interface Filter { final FilterBundle.IndexBundle indexBundle; final boolean needMatcher; - final BitmapColumnIndex columnIndex = getBitmapColumnIndex(columnIndexSelector); + final BitmapColumnIndex columnIndex = filterBundleBuilder.getBitmapColumnIndex(); if (columnIndex != null) { final long bitmapConstructionStartNs = System.nanoTime(); final T result = columnIndex.computeBitmapResult( @@ -107,7 +106,7 @@ public interface Filter new FilterBundle.MatcherBundleInfo(this::toString, null, null), this::makeMatcher, this::makeVectorMatcher, - this.canVectorizeMatcher(columnIndexSelector) + this.canVectorizeMatcher(filterBundleBuilder.getColumnIndexSelector()) ); } else { matcherBundle = null; @@ -122,7 +121,6 @@ public interface Filter * examine details about the index prior to computing it, via {@link BitmapColumnIndex#getIndexCapabilities()}. * * @param selector Object used to create BitmapColumnIndex - * * @return BitmapColumnIndex that can build ImmutableBitmap of matched row numbers */ @Nullable @@ -132,7 +130,6 @@ public interface Filter * Get a {@link ValueMatcher} that applies this filter to row values. * * @param factory Object used to create ValueMatchers - * * @return ValueMatcher that applies this filter to row values. */ ValueMatcher makeMatcher(ColumnSelectorFactory factory); @@ -141,7 +138,6 @@ public interface Filter * Get a {@link VectorValueMatcher} that applies this filter to row vectors. * * @param factory Object used to create ValueMatchers - * * @return VectorValueMatcher that applies this filter to row vectors. */ default VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory) @@ -151,6 +147,7 @@ public interface Filter /** * Returns true if this filter can produce a vectorized matcher from its "makeVectorMatcher" method. + * * @param inspector Supplies type information for the selectors this filter will match against */ default boolean canVectorizeMatcher(ColumnInspector inspector) @@ -176,7 +173,7 @@ public interface Filter * Return a copy of this filter that is identical to the this filter except that it operates on different columns, * based on a renaming map where the key is the column to be renamed in the filter, and the value is the new * column name. - * + *

    * For example, if I have a filter (A = hello), and I have a renaming map (A -> B), * this should return the filter (B = hello) * diff --git a/processing/src/main/java/org/apache/druid/query/filter/FilterBundle.java b/processing/src/main/java/org/apache/druid/query/filter/FilterBundle.java index e105b0d6163..8511642a0c4 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/FilterBundle.java +++ b/processing/src/main/java/org/apache/druid/query/filter/FilterBundle.java @@ -24,16 +24,21 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.query.filter.vector.VectorValueMatcher; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnIndexCapabilities; import org.apache.druid.segment.column.SimpleColumnIndexCapabilities; import org.apache.druid.segment.data.Offset; import org.apache.druid.segment.filter.FalseFilter; +import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.vector.ReadableVectorOffset; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; import java.util.List; import java.util.concurrent.TimeUnit; import java.util.function.Function; @@ -57,6 +62,21 @@ import java.util.regex.Pattern; */ public class FilterBundle { + @Nullable + private final IndexBundle indexBundle; + @Nullable + private final MatcherBundle matcherBundle; + + public FilterBundle(@Nullable IndexBundle index, @Nullable MatcherBundle matcherBundle) + { + Preconditions.checkArgument( + index != null || matcherBundle != null, + "At least one of index or matcher must be not null" + ); + this.indexBundle = index; + this.matcherBundle = matcherBundle; + } + public static FilterBundle allFalse(long constructionTime, ImmutableBitmap emptyBitmap) { return new FilterBundle( @@ -69,25 +89,6 @@ public class FilterBundle ); } - @Nullable - private final IndexBundle indexBundle; - @Nullable - private final MatcherBundle matcherBundle; - - public FilterBundle( - @Nullable IndexBundle index, - @Nullable MatcherBundle matcherBundle - ) - { - Preconditions.checkArgument( - index != null || matcherBundle != null, - "At least one of index or matcher must be not null" - ); - this.indexBundle = index; - this.matcherBundle = matcherBundle; - } - - @Nullable public IndexBundle getIndex() { @@ -151,6 +152,95 @@ public class FilterBundle boolean canVectorize(); } + /** + * Wraps info needed to build a {@link FilterBundle}, and provides an estimated compute cost for + * {@link BitmapColumnIndex#computeBitmapResult}. + */ + public static class Builder + { + private final Filter filter; + private final ColumnIndexSelector columnIndexSelector; + @Nullable + private final BitmapColumnIndex bitmapColumnIndex; + private final List childBuilders; + private final int estimatedIndexComputeCost; + + public Builder(Filter filter, ColumnIndexSelector columnIndexSelector, boolean cursorAutoArrangeFilters) + { + this.filter = filter; + this.columnIndexSelector = columnIndexSelector; + this.bitmapColumnIndex = filter.getBitmapColumnIndex(columnIndexSelector); + // Construct Builder instances for all child filters recursively. + if (filter instanceof BooleanFilter) { + Collection childFilters = ((BooleanFilter) filter).getFilters(); + this.childBuilders = new ArrayList<>(childFilters.size()); + for (Filter childFilter : childFilters) { + this.childBuilders.add(new FilterBundle.Builder(childFilter, columnIndexSelector, cursorAutoArrangeFilters)); + } + } else { + this.childBuilders = new ArrayList<>(0); + } + if (cursorAutoArrangeFilters) { + // Sort child builders by cost in ASCENDING order, should be stable by default. + this.childBuilders.sort(Comparator.comparingInt(FilterBundle.Builder::getEstimatedIndexComputeCost)); + this.estimatedIndexComputeCost = calculateEstimatedIndexComputeCost(); + } else { + this.estimatedIndexComputeCost = Integer.MAX_VALUE; + } + } + + private int calculateEstimatedIndexComputeCost() + { + if (this.bitmapColumnIndex == null) { + return Integer.MAX_VALUE; + } + int cost = this.bitmapColumnIndex.estimatedComputeCost(); + if (cost == Integer.MAX_VALUE) { + return Integer.MAX_VALUE; + } + + for (FilterBundle.Builder childBuilder : childBuilders) { + int childCost = childBuilder.getEstimatedIndexComputeCost(); + if (childCost >= Integer.MAX_VALUE - cost) { + return Integer.MAX_VALUE; + } + cost += childCost; + } + return cost; + } + + public ColumnIndexSelector getColumnIndexSelector() + { + return columnIndexSelector; + } + + @Nullable + public BitmapColumnIndex getBitmapColumnIndex() + { + return bitmapColumnIndex; + } + + public List getChildBuilders() + { + return childBuilders; + } + + public int getEstimatedIndexComputeCost() + { + return estimatedIndexComputeCost; + } + + public FilterBundle build( + BitmapResultFactory bitmapResultFactory, + int applyRowCount, + int totalRowCount, + boolean includeUnknown + ) + { + return filter.makeFilterBundle(this, bitmapResultFactory, applyRowCount, totalRowCount, includeUnknown); + } + } + public static class SimpleIndexBundle implements IndexBundle { private final IndexBundleInfo info; @@ -211,11 +301,7 @@ public class FilterBundle } @Override - public ValueMatcher valueMatcher( - ColumnSelectorFactory selectorFactory, - Offset baseOffset, - boolean descending - ) + public ValueMatcher valueMatcher(ColumnSelectorFactory selectorFactory, Offset baseOffset, boolean descending) { return matcherFn.apply(selectorFactory); } @@ -339,12 +425,11 @@ public class FilterBundle */ public String describe() { - final StringBuilder sb = new StringBuilder() - .append("index: ") - .append(filter.get()) - .append(" (selectionSize = ") - .append(selectionSize) - .append(")\n"); + final StringBuilder sb = new StringBuilder().append("index: ") + .append(filter.get()) + .append(" (selectionSize = ") + .append(selectionSize) + .append(")\n"); if (indexes != null) { for (final IndexBundleInfo info : indexes) { @@ -358,23 +443,26 @@ public class FilterBundle @Override public String toString() { - return "{" + - "filter=\"" + filter.get() + '\"' + - ", selectionSize=" + selectionSize + - ", buildTime=" + TimeUnit.NANOSECONDS.toMicros(buildTimeNs) + "μs" + - (indexes != null ? ", indexes=" + indexes : "") + - '}'; + return "{" + + "filter=\"" + + filter.get() + + '\"' + + ", selectionSize=" + + selectionSize + + ", buildTime=" + + TimeUnit.NANOSECONDS.toMicros(buildTimeNs) + + "μs" + + (indexes != null ? ", indexes=" + indexes : "") + + '}'; } } public static class MatcherBundleInfo { private static final Pattern PATTERN_LINE_START = Pattern.compile("(?m)^"); - - private final Supplier filter; @Nullable final List matchers; - + private final Supplier filter; @Nullable private final IndexBundleInfo partialIndex; @@ -415,10 +503,7 @@ public class FilterBundle */ public String describe() { - final StringBuilder sb = new StringBuilder() - .append("matcher: ") - .append(filter.get()) - .append("\n"); + final StringBuilder sb = new StringBuilder().append("matcher: ").append(filter.get()).append("\n"); if (partialIndex != null) { sb.append(" with partial ") @@ -437,11 +522,13 @@ public class FilterBundle @Override public String toString() { - return "{" + - "filter=\"" + filter.get() + '\"' + - (partialIndex != null ? ", partialIndex=" + partialIndex : "") + - (matchers != null ? ", matchers=" + matchers : "") + - '}'; + return "{" + + "filter=\"" + + filter.get() + + '\"' + + (partialIndex != null ? ", partialIndex=" + partialIndex : "") + + (matchers != null ? ", matchers=" + matchers : "") + + '}'; } } } diff --git a/processing/src/main/java/org/apache/druid/query/filter/FilterTuning.java b/processing/src/main/java/org/apache/druid/query/filter/FilterTuning.java index 831d50261e2..892192128e0 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/FilterTuning.java +++ b/processing/src/main/java/org/apache/druid/query/filter/FilterTuning.java @@ -30,7 +30,7 @@ import java.util.Objects; /** * This class provides a mechanism to influence whether or not indexes are used for a {@link Filter} during processing - * by {@link Filter#makeFilterBundle(ColumnIndexSelector, BitmapResultFactory, int, int, boolean)} + * by {@link Filter#makeFilterBundle(FilterBundle.Builder, BitmapResultFactory, int, int, boolean)} * (i.e. will a {@link Filter} be a "pre" filter in which we union indexes for all values that match the filter to * create a {@link org.apache.druid.segment.BitmapOffset}/{@link org.apache.druid.segment.vector.BitmapVectorOffset}, * or will it be used as a "post" filter and evaluated while scanning row values from the diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorMatch.java b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorMatch.java index f07a272e8d0..09c813c9348 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorMatch.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorMatch.java @@ -20,7 +20,7 @@ package org.apache.druid.query.filter.vector; import com.google.common.base.Preconditions; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.query.QueryContexts; import javax.annotation.Nullable; @@ -31,7 +31,7 @@ import javax.annotation.Nullable; */ public class VectorMatch implements ReadableVectorMatch { - private static final int[] DEFAULT_ALL_TRUE_VECTOR = new int[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE]; + private static final int[] DEFAULT_ALL_TRUE_VECTOR = new int[QueryContexts.DEFAULT_VECTOR_SIZE]; private static final VectorMatch ALL_FALSE = new VectorMatch(new int[0], 0); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java index dcde45aeaae..665eb5dd125 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -21,6 +21,8 @@ package org.apache.druid.query.groupby; import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; +import org.apache.druid.collections.NonBlockingPool; +import org.apache.druid.guice.annotations.Global; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.Query; @@ -30,12 +32,14 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import javax.annotation.Nullable; +import java.nio.ByteBuffer; + /** * */ @@ -43,21 +47,24 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory processingBufferPool; @Inject public GroupByQueryRunnerFactory( GroupingEngine groupingEngine, - GroupByQueryQueryToolChest toolChest + GroupByQueryQueryToolChest toolChest, + @Global NonBlockingPool processingBufferPool ) { this.groupingEngine = groupingEngine; this.toolChest = toolChest; + this.processingBufferPool = processingBufferPool; } @Override public QueryRunner createRunner(final Segment segment) { - return new GroupByQueryRunner(segment, groupingEngine); + return new GroupByQueryRunner(segment, groupingEngine, processingBufferPool); } /** @@ -69,14 +76,9 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory> queryRunners ) { - return new QueryRunner() - { - @Override - public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) - { - QueryRunner rowQueryRunner = groupingEngine.mergeRunners(queryProcessingPool, queryRunners); - return rowQueryRunner.run(queryPlus, responseContext); - } + return (queryPlus, responseContext) -> { + QueryRunner rowQueryRunner = groupingEngine.mergeRunners(queryProcessingPool, queryRunners); + return rowQueryRunner.run(queryPlus, responseContext); }; } @@ -88,16 +90,22 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory { - private final StorageAdapter adapter; + private final CursorFactory cursorFactory; @Nullable private final TimeBoundaryInspector timeBoundaryInspector; private final GroupingEngine groupingEngine; + private final NonBlockingPool processingBufferPool; - public GroupByQueryRunner(Segment segment, final GroupingEngine groupingEngine) + public GroupByQueryRunner( + Segment segment, + final GroupingEngine groupingEngine, + final NonBlockingPool processingBufferPool + ) { - this.adapter = segment.asStorageAdapter(); + this.cursorFactory = segment.asCursorFactory(); this.timeBoundaryInspector = segment.as(TimeBoundaryInspector.class); this.groupingEngine = groupingEngine; + this.processingBufferPool = processingBufferPool; } @Override @@ -110,8 +118,9 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory configSupplier; - private final NonBlockingPool bufferPool; - GroupByResourcesReservationPool groupByResourcesReservationPool; + private final GroupByResourcesReservationPool groupByResourcesReservationPool; private final ObjectMapper jsonMapper; private final ObjectMapper spillMapper; private final QueryWatcher queryWatcher; @@ -126,7 +126,6 @@ public class GroupingEngine public GroupingEngine( DruidProcessingConfig processingConfig, Supplier configSupplier, - @Global NonBlockingPool bufferPool, @Merging GroupByResourcesReservationPool groupByResourcesReservationPool, @Json ObjectMapper jsonMapper, @Smile ObjectMapper spillMapper, @@ -135,7 +134,6 @@ public class GroupingEngine { this.processingConfig = processingConfig; this.configSupplier = configSupplier; - this.bufferPool = bufferPool; this.groupByResourcesReservationPool = groupByResourcesReservationPool; this.jsonMapper = jsonMapper; this.spillMapper = spillMapper; @@ -459,30 +457,33 @@ public class GroupingEngine } /** - * Process a groupBy query on a single {@link StorageAdapter}. This is used by + * Process a groupBy query on a single {@link CursorFactory}. This is used by * {@link GroupByQueryRunnerFactory#createRunner} to create per-segment * QueryRunners. * * This method is only called on data servers, like Historicals (not the Broker). * * @param query the groupBy query - * @param storageAdapter storage adatper for the segment in question + * @param cursorFactory cursor factory for the segment in question * @param timeBoundaryInspector time boundary inspector for the segment in question + * @param bufferPool processing buffer pool + * @param groupByQueryMetrics metrics instance, will be populated if nonnull * - * @return result sequence for the storage adapter + * @return result sequence for the cursor factory */ public Sequence process( GroupByQuery query, - StorageAdapter storageAdapter, + CursorFactory cursorFactory, @Nullable TimeBoundaryInspector timeBoundaryInspector, + NonBlockingPool bufferPool, @Nullable GroupByQueryMetrics groupByQueryMetrics ) { final GroupByQueryConfig querySpecificConfig = configSupplier.get().withOverrides(query); - if (storageAdapter == null) { + if (cursorFactory == null) { throw new ISE( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." ); } @@ -505,9 +506,12 @@ public class GroupingEngine : DateTimes.utc(Long.parseLong(fudgeTimestampString)); final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, groupByQueryMetrics); - final CursorHolder cursorHolder = closer.register(storageAdapter.makeCursorHolder(buildSpec)); + final CursorHolder cursorHolder = closer.register(cursorFactory.makeCursorHolder(buildSpec)); - final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(storageAdapter); + if (cursorHolder.isPreAggregated()) { + query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs())); + } + final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(cursorFactory); // group by specific vectorization check final boolean canVectorize = cursorHolder.canVectorize() && @@ -517,7 +521,6 @@ public class GroupingEngine if (shouldVectorize) { result = VectorGroupByEngine.process( query, - storageAdapter, timeBoundaryInspector, cursorHolder, bufferHolder.get(), @@ -529,7 +532,6 @@ public class GroupingEngine } else { result = GroupByQueryEngine.process( query, - storageAdapter, timeBoundaryInspector, cursorHolder, buildSpec, @@ -874,7 +876,8 @@ public class GroupingEngine public static int getCardinalityForArrayAggregation( GroupByQueryConfig querySpecificConfig, GroupByQuery query, - StorageAdapter storageAdapter, + ColumnInspector columnInspector, + List groupingSelectors, ByteBuffer buffer ) { @@ -893,7 +896,7 @@ public class GroupingEngine } else if (dimensions.size() == 1) { // Only real columns can use array-based aggregation, since virtual columns cannot currently report their // cardinality. We need to check if a virtual column exists with the same name, since virtual columns can shadow - // real columns, and we might miss that since we're going directly to the StorageAdapter (which only knows about + // real columns, and we might miss that since we're going directly to the CursorFactory (which only knows about // real columns). if (query.getVirtualColumns().exists(Iterables.getOnlyElement(dimensions).getDimension())) { return -1; @@ -905,8 +908,8 @@ public class GroupingEngine } final String columnName = Iterables.getOnlyElement(dimensions).getDimension(); - columnCapabilities = storageAdapter.getColumnCapabilities(columnName); - cardinality = storageAdapter.getDimensionCardinality(columnName); + columnCapabilities = columnInspector.getColumnCapabilities(columnName); + cardinality = groupingSelectors.get(0).getValueCardinality(); } else { // Cannot use array-based aggregation with more than one dimension. return -1; diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunner.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunner.java index 6e2064d5f1f..75e8fa2c2ce 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunner.java @@ -375,7 +375,7 @@ public class GroupByMergingQueryRunner implements QueryRunner GuavaUtils.cancelAll(true, future, futures); throw new QueryInterruptedException(e); } - catch (TimeoutException e) { + catch (QueryTimeoutException | TimeoutException e) { log.info("Query timeout, cancelling pending results for query id [%s]", query.getId()); GuavaUtils.cancelAll(true, future, futures); throw new QueryTimeoutException(); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java index c6a32d36416..2e67a892b01 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java @@ -53,7 +53,6 @@ import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; @@ -64,18 +63,20 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; import java.io.Closeable; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; import java.util.stream.Stream; /** - * Contains logic to process a groupBy query on a single {@link StorageAdapter} in a non-vectorized manner. - * Processing returns a {@link Sequence} of {@link ResultRow} objects that are not guaranteed to be in any particular - * order, and may not even be fully grouped. It is expected that a downstream {@link GroupByMergingQueryRunner} will - * finish grouping these results. + * Contains logic to process a groupBy query on a single {@link org.apache.druid.segment.CursorFactory} in a + * non-vectorized manner. Processing returns a {@link Sequence} of {@link ResultRow} objects that are not guaranteed + * to be in any particular order, and may not even be fully grouped. It is expected that a downstream + * {@link GroupByMergingQueryRunner} will finish grouping these results. *

    - * This code runs on anything that processes {@link StorageAdapter} directly, typically data servers like Historicals. + * This code runs on anything that processes {@link org.apache.druid.segment.CursorFactory} directly, typically data + * servers like Historicals. *

    * Used for non-vectorized processing by {@link GroupingEngine#process}. * @@ -95,7 +96,6 @@ public class GroupByQueryEngine public static Sequence process( final GroupByQuery query, - final StorageAdapter storageAdapter, @Nullable final TimeBoundaryInspector timeBoundaryInspector, final CursorHolder cursorHolder, final CursorBuildSpec buildSpec, @@ -126,7 +126,7 @@ public class GroupByQueryEngine query.getDimensions(), columnSelectorFactory ); - GroupByColumnSelectorPlus[] dims = new GroupByColumnSelectorPlus[selectorPlus.length]; + final GroupByColumnSelectorPlus[] dims = new GroupByColumnSelectorPlus[selectorPlus.length]; int curPos = 0; for (int i = 0; i < dims.length; i++) { dims[i] = new GroupByColumnSelectorPlus( @@ -140,7 +140,8 @@ public class GroupByQueryEngine final int cardinalityForArrayAggregation = GroupingEngine.getCardinalityForArrayAggregation( querySpecificConfig, query, - storageAdapter, + columnSelectorFactory, + Arrays.asList(dims), processingBuffer ); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupingSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupingSelector.java new file mode 100644 index 00000000000..6d90164136f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupingSelector.java @@ -0,0 +1,35 @@ +/* + * 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.query.groupby.epinephelinae; + +import org.apache.druid.segment.DimensionDictionarySelector; + +public interface GroupingSelector +{ + /** + * Returns the value cardinality if it is known (such as when backed by a {@link DimensionDictionarySelector}). If + * there are only null values, this method should return 1. If the cardinality is not known, returns + * {@link DimensionDictionarySelector#CARDINALITY_UNKNOWN}. + */ + default int getValueCardinality() + { + return DimensionDictionarySelector.CARDINALITY_UNKNOWN; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java index 4eb0e9588a8..23ff05474e4 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/GroupByColumnSelectorPlus.java @@ -20,8 +20,12 @@ package org.apache.druid.query.groupby.epinephelinae.column; import org.apache.druid.query.ColumnSelectorPlus; +import org.apache.druid.query.groupby.epinephelinae.GroupingSelector; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionDictionarySelector; -public class GroupByColumnSelectorPlus extends ColumnSelectorPlus +public class GroupByColumnSelectorPlus extends ColumnSelectorPlus implements + GroupingSelector { /** * Indicates the offset of this dimension's value within ResultRows. @@ -58,4 +62,14 @@ public class GroupByColumnSelectorPlus extends ColumnSelectorPlus selector = getSelector(); + if (selector instanceof DimensionDictionarySelector) { + return ((DimensionDictionarySelector) selector).getValueCardinality(); + } + return DimensionDictionarySelector.CARDINALITY_UNKNOWN; + } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java index 774d51c5869..60ccbfadddb 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java @@ -21,6 +21,7 @@ package org.apache.druid.query.groupby.epinephelinae.vector; import org.apache.datasketches.memory.WritableMemory; import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.query.groupby.epinephelinae.GroupingSelector; import org.apache.druid.query.groupby.epinephelinae.collection.MemoryPointer; /** @@ -35,7 +36,7 @@ import org.apache.druid.query.groupby.epinephelinae.collection.MemoryPointer; * @see GroupByVectorColumnProcessorFactory * @see org.apache.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorStrategy the nonvectorized version */ -public interface GroupByVectorColumnSelector +public interface GroupByVectorColumnSelector extends GroupingSelector { /** * Get the size in bytes of the key parts generated by this column. diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/NilGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/NilGroupByVectorColumnSelector.java index d7bbb21e59e..e39653e4b43 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/NilGroupByVectorColumnSelector.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/NilGroupByVectorColumnSelector.java @@ -41,6 +41,12 @@ public class NilGroupByVectorColumnSelector implements GroupByVectorColumnSelect return 0; } + @Override + public int getValueCardinality() + { + return 1; + } + @Override public int writeKeys(WritableMemory keySpace, int keySize, int keyOffset, int startRow, int endRow) { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java index 738fa1cb7b3..57ad933188f 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java @@ -39,6 +39,12 @@ public class SingleValueStringGroupByVectorColumnSelector implements GroupByVect return Integer.BYTES; } + @Override + public int getValueCardinality() + { + return selector.getValueCardinality(); + } + @Override public int writeKeys( final WritableMemory keySpace, diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java index c6f52e0bd21..78fa8e532a3 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java @@ -45,7 +45,6 @@ import org.apache.druid.query.vector.VectorCursorGranularizer; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -63,8 +62,9 @@ import java.util.NoSuchElementException; import java.util.stream.Collectors; /** - * Contains logic to process a groupBy query on a single {@link StorageAdapter} in a vectorized manner. - * This code runs on anything that processes {@link StorageAdapter} directly, typically data servers like Historicals. + * Contains logic to process a groupBy query on a single {@link org.apache.druid.segment.CursorFactory} in a vectorized + * manner. This code runs on anything that processes {@link org.apache.druid.segment.CursorFactory} directly, typically + * data servers like Historicals. *

    * Used for vectorized processing by {@link GroupingEngine#process}. * @@ -79,7 +79,6 @@ public class VectorGroupByEngine public static Sequence process( final GroupByQuery query, - final StorageAdapter storageAdapter, @Nullable TimeBoundaryInspector timeBoundaryInspector, final CursorHolder cursorHolder, final ByteBuffer processingBuffer, @@ -145,7 +144,6 @@ public class VectorGroupByEngine query, config, processingConfig, - storageAdapter, timeBoundaryInspector, cursor, cursorHolder.getTimeOrder(), @@ -209,7 +207,6 @@ public class VectorGroupByEngine private final GroupByQuery query; private final GroupByQueryConfig querySpecificConfig; private final DruidProcessingConfig processingConfig; - private final StorageAdapter storageAdapter; private final VectorCursor cursor; private final List selectors; private final ByteBuffer processingBuffer; @@ -241,7 +238,6 @@ public class VectorGroupByEngine final GroupByQuery query, final GroupByQueryConfig querySpecificConfig, final DruidProcessingConfig processingConfig, - final StorageAdapter storageAdapter, @Nullable TimeBoundaryInspector timeBoundaryInspector, final VectorCursor cursor, final Order timeOrder, @@ -254,7 +250,6 @@ public class VectorGroupByEngine this.query = query; this.querySpecificConfig = querySpecificConfig; this.processingConfig = processingConfig; - this.storageAdapter = storageAdapter; this.cursor = cursor; this.selectors = selectors; this.processingBuffer = processingBuffer; @@ -330,11 +325,13 @@ public class VectorGroupByEngine VectorGrouper makeGrouper() { final VectorGrouper grouper; + final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); final int cardinalityForArrayAggregation = GroupingEngine.getCardinalityForArrayAggregation( querySpecificConfig, query, - storageAdapter, + columnSelectorFactory, + selectors, processingBuffer ); @@ -342,7 +339,7 @@ public class VectorGroupByEngine grouper = new BufferArrayGrouper( Suppliers.ofInstance(processingBuffer), AggregatorAdapters.factorizeVector( - cursor.getColumnSelectorFactory(), + columnSelectorFactory, query.getAggregatorSpecs() ), cardinalityForArrayAggregation diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java index 6d0629c4dce..d0015ba8027 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java @@ -29,11 +29,12 @@ import org.apache.druid.query.metadata.metadata.ColumnAnalysis; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.PhysicalSegmentInspector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; @@ -46,7 +47,6 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.TypeSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetrics; @@ -80,32 +80,31 @@ public class SegmentAnalyzer public long numRows(Segment segment) { - return Preconditions.checkNotNull(segment, "segment").asStorageAdapter().getNumRows(); + return Preconditions.checkNotNull(segment.as(PhysicalSegmentInspector.class), "PhysicalSegmentInspector") + .getNumRows(); } public Map analyze(Segment segment) { Preconditions.checkNotNull(segment, "segment"); + final PhysicalSegmentInspector segmentInspector = segment.as(PhysicalSegmentInspector.class); - // index is null for incremental-index-based segments, but storageAdapter is always available - final QueryableIndex index = segment.asQueryableIndex(); - final StorageAdapter storageAdapter = segment.asStorageAdapter(); + // index is null for incremental-index-based segments, but segmentInspector should always be available + final QueryableIndex index = segment.as(QueryableIndex.class); - // get length and column names from storageAdapter - final int numRows = storageAdapter.getNumRows(); + final int numRows = segmentInspector != null ? segmentInspector.getNumRows() : 0; // Use LinkedHashMap to preserve column order. final Map columns = new LinkedHashMap<>(); - final RowSignature rowSignature = storageAdapter.getRowSignature(); + final RowSignature rowSignature = segment.asCursorFactory().getRowSignature(); for (String columnName : rowSignature.getColumnNames()) { final ColumnCapabilities capabilities; - if (storageAdapter instanceof IncrementalIndexStorageAdapter) { - // See javadocs for getSnapshotColumnCapabilities for a discussion of why we need to do this. - capabilities = ((IncrementalIndexStorageAdapter) storageAdapter).getSnapshotColumnCapabilities(columnName); + if (segmentInspector != null) { + capabilities = segmentInspector.getColumnCapabilities(columnName); } else { - capabilities = storageAdapter.getColumnCapabilities(columnName); + capabilities = null; } if (capabilities == null) { @@ -133,7 +132,7 @@ public class SegmentAnalyzer if (index != null) { analysis = analyzeStringColumn(capabilities, index.getColumnHolder(columnName)); } else { - analysis = analyzeStringColumn(capabilities, storageAdapter, columnName); + analysis = analyzeStringColumn(capabilities, segmentInspector, segment.asCursorFactory(), columnName); } break; case ARRAY: @@ -255,7 +254,8 @@ public class SegmentAnalyzer private ColumnAnalysis analyzeStringColumn( final ColumnCapabilities capabilities, - final StorageAdapter storageAdapter, + @Nullable final PhysicalSegmentInspector analysisInspector, + final CursorFactory cursorFactory, final String columnName ) { @@ -265,12 +265,17 @@ public class SegmentAnalyzer Comparable min = null; Comparable max = null; - if (analyzingCardinality()) { - cardinality = storageAdapter.getDimensionCardinality(columnName); + if (analyzingCardinality() && analysisInspector != null) { + cardinality = analysisInspector.getDimensionCardinality(columnName); + } + + if (analyzingMinMax() && analysisInspector != null) { + min = analysisInspector.getMinValue(columnName); + max = analysisInspector.getMaxValue(columnName); } if (analyzingSize()) { - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); if (cursor != null) { @@ -291,11 +296,6 @@ public class SegmentAnalyzer } } - if (analyzingMinMax()) { - min = storageAdapter.getMinValue(columnName); - max = storageAdapter.getMaxValue(columnName); - } - return ColumnAnalysis.builder() .withCapabilities(capabilities) .withSize(size) diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index 827c3a86b07..a2edb663886 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -47,9 +47,11 @@ import org.apache.druid.query.metadata.metadata.ColumnIncluderator; import org.apache.druid.query.metadata.metadata.SegmentAnalysis; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.segment.Metadata; +import org.apache.druid.segment.PhysicalSegmentInspector; import org.apache.druid.segment.Segment; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; @@ -117,7 +119,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory aggregators; Metadata metadata = null; if (updatedQuery.hasAggregators()) { - metadata = segment.asStorageAdapter().getMetadata(); + metadata = getMetadata(segment); if (metadata != null && metadata.getAggregators() != null) { aggregators = new HashMap<>(); for (AggregatorFactory aggregator : metadata.getAggregators()) { @@ -133,7 +135,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory @@ -50,7 +56,7 @@ public class WindowOperatorQueryQueryToolChest extends QueryToolChest mergeResults(QueryRunner runner) { - return new RowsAndColumnsUnravelingQueryRunner( + return new RowsAndColumnsSerializingQueryRunner( (queryPlus, responseContext) -> { final WindowOperatorQuery query = (WindowOperatorQuery) queryPlus.getQuery(); final List opFactories = query.getOperators(); @@ -61,7 +67,7 @@ public class WindowOperatorQueryQueryToolChest extends QueryToolChest opSupplier = () -> { Operator retVal = new SequenceOperator( runner.run( - queryPlus.withQuery(query.withOperators(new ArrayList())), + queryPlus.withQuery(query.withOperators(new ArrayList<>())), responseContext ) ); @@ -112,16 +118,29 @@ public class WindowOperatorQueryQueryToolChest extends QueryToolChest resultSequence ) { - // Dark magic; see RowsAndColumnsUnravelingQueryRunner. + // Dark magic; see RowsAndColumnsSerializingQueryRunner. return (Sequence) resultSequence; } + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public Optional> resultsAsFrames( + WindowOperatorQuery query, + Sequence resultSequence, + MemoryAllocatorFactory memoryAllocatorFactory, + boolean useNestedForUnknownTypes + ) + { + // see RowsAndColumnsSerializingQueryRunner + return Optional.of((Sequence) resultSequence); + } + /** - * This class exists to unravel the RowsAndColumns that are used in this query and make it the return Sequence - * actually be a Sequence of rows. This is relatively broken in a number of regards, the most obvious of which - * is that it is going to run counter to the stated class on the Generic of the QueryToolChest. That is, the - * code makes it look like you are getting a Sequence of RowsAndColumns, but, by using this, the query will - * actually ultimately produce a Sequence of Object[]. This works because of type Erasure in Java (it's all Object + * This class exists to serialize the RowsAndColumns that are used in this query and make it the return Sequence + * actually be a Sequence of rows or frames, as the query requires. + * This is relatively broken in a number of regards, the most obvious of which is that it is going to run counter to the stated class on the Generic of the QueryToolChest. + * That is, the code makes it look like you are getting a Sequence of RowsAndColumns, but, by using this, the query will + * actually ultimately produce a Sequence of Object[] or Frames. This works because of type Erasure in Java (it's all Object * at the end of the day). *

    * While it might seem like this will break all sorts of things, the Generic type is actually there more as a type @@ -132,12 +151,12 @@ public class WindowOperatorQueryQueryToolChest extends QueryToolChest baseQueryRunner; - private RowsAndColumnsUnravelingQueryRunner( + private RowsAndColumnsSerializingQueryRunner( QueryRunner baseQueryRunner ) { @@ -158,42 +177,77 @@ public class WindowOperatorQueryQueryToolChest extends QueryToolChest { - List results = new ArrayList<>(rac.numRows()); - - ColumnAccessor[] accessors = new ColumnAccessor[rowSignature.size()]; - int index = 0; - for (String columnName : rowSignature.getColumnNames()) { - final Column column = rac.findColumn(columnName); - if (column == null) { - final ColumnType columnType = rowSignature - .getColumnType(columnName) - .orElse(ColumnType.UNKNOWN_COMPLEX); - - accessors[index] = new NullColumn.Accessor(columnType, rac.numRows()); - } else { - accessors[index] = column.toAccessor(); - } - ++index; - } - - for (int i = 0; i < rac.numRows(); ++i) { - Object[] objArr = new Object[accessors.length]; - for (int j = 0; j < accessors.length; j++) { - objArr[j] = accessors[j].getObject(i); - } - results.add(objArr); - } - - return Sequences.simple(results); - } + final ResultSerializationMode serializationMode = query.context().getEnum( + ResultSerializationMode.CTX_SERIALIZATION_PARAMETER, + ResultSerializationMode.class, + ResultSerializationMode.ROWS ); + switch (serializationMode) { + case ROWS: + return asRows(baseSequence, query); + case FRAMES: + return asFrames(baseSequence); + default: + throw DruidException.defensive("Serialization mode[%s] not supported", serializationMode); + } } return baseQueryRunner.run(queryPlus, responseContext); } + + /** + * Translates Sequence of RACs to a Sequence of Object[] + */ + private static Sequence asRows(final Sequence baseSequence, final WindowOperatorQuery query) + { + final RowSignature rowSignature = query.getRowSignature(); + return baseSequence.flatMap( + rac -> { + List results = new ArrayList<>(rac.numRows()); + + ColumnAccessor[] accessors = new ColumnAccessor[rowSignature.size()]; + int index = 0; + for (String columnName : rowSignature.getColumnNames()) { + final Column column = rac.findColumn(columnName); + if (column == null) { + final ColumnType columnType = rowSignature + .getColumnType(columnName) + .orElse(ColumnType.UNKNOWN_COMPLEX); + + accessors[index] = new NullColumn.Accessor(columnType, rac.numRows()); + } else { + accessors[index] = column.toAccessor(); + } + ++index; + } + + for (int i = 0; i < rac.numRows(); ++i) { + Object[] objArr = new Object[accessors.length]; + for (int j = 0; j < accessors.length; j++) { + objArr[j] = accessors[j].getObject(i); + } + results.add(objArr); + } + + return Sequences.simple(results); + } + ); + } + + /** + * Translates a sequence of RACs to a Sequence of Frames + */ + private static Sequence asFrames(final Sequence baseSequence) + { + return baseSequence.map( + rac -> { + FrameMaker frameMaker = FrameMaker.fromRAC(rac); + return new FrameSignaturePair( + frameMaker.toColumnBasedFrame(), + frameMaker.computeSignature() + ); + } + ); + } } } diff --git a/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java index 8d3f741087f..f5c600b96ff 100644 --- a/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java +++ b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java @@ -64,7 +64,7 @@ import java.util.Optional; * for the base or leaf datasources to include additional joins. * * The base datasource is the one that will be considered by the core Druid query stack for scanning via - * {@link org.apache.druid.segment.Segment} and {@link org.apache.druid.segment.StorageAdapter}. The other leaf + * {@link org.apache.druid.segment.Segment} and {@link org.apache.druid.segment.CursorFactory}. The other leaf * datasources must be joinable onto the base data. * * The idea here is to keep things simple and dumb. So we focus only on identifying left-leaning join trees, which map diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/AppendableMapOfColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/AppendableMapOfColumns.java index 61f6855cd01..d83f56c7ba5 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/AppendableMapOfColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/AppendableMapOfColumns.java @@ -79,14 +79,4 @@ public class AppendableMapOfColumns implements AppendableRowsAndColumns } return retVal; } - - @Override - @SuppressWarnings("unchecked") - public T as(Class clazz) - { - if (AppendableRowsAndColumns.class.equals(clazz)) { - return (T) this; - } - return null; - } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java index 883371fec65..dc7a4a35564 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java @@ -25,6 +25,7 @@ import it.unimi.dsi.fastutil.ints.IntComparator; import it.unimi.dsi.fastutil.ints.IntList; import org.apache.druid.common.semantic.SemanticCreator; import org.apache.druid.common.semantic.SemanticUtils; +import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.operator.ColumnWithDirection; @@ -40,9 +41,9 @@ import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner; import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner; import org.apache.druid.query.rowsandcols.semantic.NaiveSortMaker; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.RowAdapter; -import org.apache.druid.segment.RowBasedStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.RowBasedCursorFactory; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -153,11 +154,28 @@ public class ArrayListRowsAndColumns implements AppendableRowsAndColumn return new LimitedColumn(retVal, startOffset, endOffset); } - final Function adapterForValue = rowAdapter.columnFunction(name); final Optional maybeColumnType = rowSignature.getColumnType(name); final ColumnType columnType = maybeColumnType.orElse(ColumnType.UNKNOWN_COMPLEX); final Comparator comparator = Comparator.nullsFirst(columnType.getStrategy()); + final Function adapterForValue; + if (columnType.equals(ColumnType.STRING)) { + // special handling to reject MVDs + adapterForValue = f -> { + Object value = rowAdapter.columnFunction(name).apply(f); + if (value instanceof List) { + throw InvalidInput.exception( + "Encountered a multi value column [%s]. Window processing does not support MVDs. " + + "Consider using UNNEST or MV_TO_ARRAY.", + name + ); + } + return value; + }; + } else { + adapterForValue = rowAdapter.columnFunction(name); + } + return new Column() { @Nonnull @@ -356,9 +374,9 @@ public class ArrayListRowsAndColumns implements AppendableRowsAndColumn @SuppressWarnings("unused") @SemanticCreator - public StorageAdapter toStorageAdapter() + public CursorFactory toCursorFactory() { - return new RowBasedStorageAdapter(Sequences.simple(rows), rowAdapter, rowSignature); + return new RowBasedCursorFactory<>(Sequences.simple(rows), rowAdapter, rowSignature); } private class MyClusteredGroupPartitioner implements ClusteredGroupPartitioner diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/ConcatRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/ConcatRowsAndColumns.java index c6ced60849d..3f70f82a253 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/ConcatRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/ConcatRowsAndColumns.java @@ -141,13 +141,6 @@ public class ConcatRowsAndColumns implements RowsAndColumns } } - @Nullable - @Override - public T as(Class clazz) - { - return null; - } - private class ConcatedidColumn implements Column { diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/CursorFactoryRowsAndColumns.java similarity index 73% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java rename to processing/src/main/java/org/apache/druid/query/rowsandcols/CursorFactoryRowsAndColumns.java index 2d49cc32440..46fda857516 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/CursorFactoryRowsAndColumns.java @@ -19,6 +19,8 @@ package org.apache.druid.query.rowsandcols; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import org.apache.druid.frame.Frame; import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; import org.apache.druid.frame.write.FrameWriter; @@ -30,8 +32,8 @@ import org.apache.druid.segment.CloseableShapeshifter; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nonnull; @@ -39,44 +41,45 @@ import java.util.Collection; import java.util.Collections; /** - * Provides {@link RowsAndColumns} on top of a {@link StorageAdapter}. + * Provides {@link RowsAndColumns} on top of a {@link CursorFactory}. */ -public class StorageAdapterRowsAndColumns implements CloseableShapeshifter, RowsAndColumns +public class CursorFactoryRowsAndColumns implements CloseableShapeshifter, RowsAndColumns { - private final StorageAdapter storageAdapter; - private RowsAndColumns materialized; + private final CursorFactory cursorFactory; + private final Supplier materialized; - public StorageAdapterRowsAndColumns(StorageAdapter storageAdapter) + public CursorFactoryRowsAndColumns(CursorFactory cursorFactory) { - this.storageAdapter = storageAdapter; + this.cursorFactory = cursorFactory; + this.materialized = Suppliers.memoize(() -> materialize(cursorFactory)); } @SuppressWarnings("unchecked") @Override public T as(Class clazz) { - if (StorageAdapter.class == clazz) { - return (T) storageAdapter; + if (CursorFactory.class == clazz) { + return (T) cursorFactory; } - return null; + return RowsAndColumns.super.as(clazz); } @Override public Collection getColumnNames() { - return storageAdapter.getRowSignature().getColumnNames(); + return cursorFactory.getRowSignature().getColumnNames(); } @Override public int numRows() { - return storageAdapter.getNumRows(); + return materialized.get().numRows(); } @Override public Column findColumn(String name) { - return getRealRAC().findColumn(name); + return materialized.get().findColumn(name); } @Override @@ -84,25 +87,16 @@ public class StorageAdapterRowsAndColumns implements CloseableShapeshifter, Rows { } - protected RowsAndColumns getRealRAC() - { - if (materialized == null) { - materialized = materialize(storageAdapter); - } - return materialized; - } - @Nonnull - private static RowsAndColumns materialize(StorageAdapter as) + private static RowsAndColumns materialize(CursorFactory cursorFactory) { - try (final CursorHolder cursorHolder = as.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); + final RowSignature rowSignature = cursorFactory.getRowSignature(); if (cursor == null) { return new EmptyRowsAndColumns(); } - - final RowSignature rowSignature = as.getRowSignature(); final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/EmptyRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/EmptyRowsAndColumns.java index dd0c7dab1cd..56647e0f568 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/EmptyRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/EmptyRowsAndColumns.java @@ -21,7 +21,6 @@ package org.apache.druid.query.rowsandcols; import org.apache.druid.query.rowsandcols.column.Column; -import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collection; @@ -44,11 +43,4 @@ public class EmptyRowsAndColumns implements RowsAndColumns { return null; } - - @Nullable - @Override - public T as(Class clazz) - { - return null; - } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index 6be9500b9fe..bb35f683797 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -39,15 +39,15 @@ import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumns; +import org.apache.druid.query.rowsandcols.concrete.FrameRowsAndColumns; import org.apache.druid.query.rowsandcols.semantic.ColumnSelectorFactoryMaker; import org.apache.druid.query.rowsandcols.semantic.DefaultRowsAndColumnsDecorator; import org.apache.druid.query.rowsandcols.semantic.RowsAndColumnsDecorator; -import org.apache.druid.query.rowsandcols.semantic.WireTransferable; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.RowSignature; @@ -150,16 +150,10 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns @SuppressWarnings("unused") @SemanticCreator - public WireTransferable toWireTransferable() + public FrameRowsAndColumns toFrameRowsAndColumns() { - return () -> { - final Pair materialized = materialize(); - if (materialized == null) { - return new byte[]{}; - } else { - return materialized.lhs; - } - }; + maybeMaterialize(); + return base.as(FrameRowsAndColumns.class); } private void maybeMaterialize() @@ -185,11 +179,11 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns throw new ISE("Cannot reorder[%s] scan data right now", ordering); } - final StorageAdapter as = base.as(StorageAdapter.class); + final CursorFactory as = base.as(CursorFactory.class); if (as == null) { return naiveMaterialize(base); } else { - return materializeStorageAdapter(as); + return materializeCursorFactory(as); } } @@ -205,7 +199,7 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns } @Nullable - private Pair materializeStorageAdapter(StorageAdapter as) + private Pair materializeCursorFactory(CursorFactory cursorFactory) { final Collection cols; if (viewableColumns != null) { @@ -228,7 +222,7 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns if (virtualColumns != null) { builder.setVirtualColumns(virtualColumns); } - try (final CursorHolder cursorHolder = as.makeCursorHolder(builder.build())) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(builder.build())) { final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LimitedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LimitedRowsAndColumns.java index abb3d4649b1..8cfadecb4dd 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LimitedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LimitedRowsAndColumns.java @@ -23,7 +23,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.LimitedColumn; -import javax.annotation.Nullable; import java.util.Collection; public class LimitedRowsAndColumns implements RowsAndColumns @@ -66,12 +65,4 @@ public class LimitedRowsAndColumns implements RowsAndColumns return new LimitedColumn(column, start, end); } - - @Nullable - @Override - public T as(Class clazz) - { - return null; - } - } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java index 29f092f6744..d6bc1026a98 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java @@ -164,7 +164,7 @@ public class MapOfColumnsRowsAndColumns implements RowsAndColumns if (AppendableRowsAndColumns.class.equals(clazz)) { return (T) new AppendableMapOfColumns(this); } - return null; + return RowsAndColumns.super.as(clazz); } public static class Builder diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/RearrangedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/RearrangedRowsAndColumns.java index f1793f8fd0e..e64f086edd7 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/RearrangedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/RearrangedRowsAndColumns.java @@ -164,11 +164,4 @@ public class RearrangedRowsAndColumns implements RowsAndColumns ); } } - - @Nullable - @Override - public T as(Class clazz) - { - return null; - } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/RowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/RowsAndColumns.java index 7b6a1f6215d..a34d0e463c0 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/RowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/RowsAndColumns.java @@ -19,12 +19,30 @@ package org.apache.druid.query.rowsandcols; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.deser.std.StdDeserializer; +import com.fasterxml.jackson.databind.ser.std.StdSerializer; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.channel.ByteTracker; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumns; +import org.apache.druid.query.rowsandcols.concrete.FrameRowsAndColumns; +import org.apache.druid.query.rowsandcols.concrete.RowBasedFrameRowsAndColumns; import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; import org.apache.druid.query.rowsandcols.semantic.FramedOnHeapAggregatable; +import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.channels.Channels; import java.util.Collection; /** @@ -110,6 +128,72 @@ public interface RowsAndColumns * @return A concrete implementation of the interface, or null if there is no meaningful optimization to be had * through a local implementation of the interface. */ + @SuppressWarnings("unchecked") @Nullable - T as(Class clazz); + default T as(Class clazz) + { + if (clazz.isInstance(this)) { + return (T) this; + } + return null; + } + + /** + * Serializer for {@link RowsAndColumns} by converting the instance to {@link FrameRowsAndColumns} + */ + class RowsAndColumnsSerializer extends StdSerializer + { + public RowsAndColumnsSerializer() + { + super(RowsAndColumns.class); + } + + @Override + public void serialize( + RowsAndColumns rac, + JsonGenerator jsonGenerator, + SerializerProvider serializerProvider + ) throws IOException + { + FrameRowsAndColumns frameRAC = rac.as(FrameRowsAndColumns.class); + if (frameRAC == null) { + throw DruidException.defensive("Unable to serialize RAC"); + } + JacksonUtils.writeObjectUsingSerializerProvider(jsonGenerator, serializerProvider, frameRAC.getSignature()); + + Frame frame = frameRAC.getFrame(); + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + frame.writeTo(Channels.newChannel(baos), false, null, ByteTracker.unboundedTracker()); + + jsonGenerator.writeBinary(baos.toByteArray()); + } + } + + /** + * Deserializer for {@link RowsAndColumns} returning as an instance of {@link FrameRowsAndColumns} + */ + class RowsAndColumnsDeserializer extends StdDeserializer + { + public RowsAndColumnsDeserializer() + { + super(RowsAndColumns.class); + } + + @Override + public FrameRowsAndColumns deserialize(JsonParser jsonParser, DeserializationContext deserializationContext) + throws IOException + { + RowSignature sig = jsonParser.readValueAs(RowSignature.class); + jsonParser.nextValue(); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + jsonParser.readBinaryValue(baos); + Frame frame = Frame.wrap(baos.toByteArray()); + if (frame.type() == FrameType.COLUMNAR) { + return new ColumnBasedFrameRowsAndColumns(frame, sig); + } else { + return new RowBasedFrameRowsAndColumns(frame, sig); + } + } + } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/AbstractFrameRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/AbstractFrameRowsAndColumns.java new file mode 100644 index 00000000000..5295326c862 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/AbstractFrameRowsAndColumns.java @@ -0,0 +1,106 @@ +/* + * 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.query.rowsandcols.concrete; + +import com.google.common.base.Objects; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.segment.CloseableShapeshifter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.column.RowSignature; + +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.LinkedHashMap; + +public abstract class AbstractFrameRowsAndColumns implements FrameRowsAndColumns, AutoCloseable, CloseableShapeshifter +{ + final Frame frame; + final RowSignature signature; + final LinkedHashMap colCache = new LinkedHashMap<>(); + + public AbstractFrameRowsAndColumns(Frame frame, RowSignature signature) + { + this.frame = frame; + this.signature = signature; + } + + @Override + public Frame getFrame() + { + return frame; + } + + @Override + public RowSignature getSignature() + { + return signature; + } + + @Override + public Collection getColumnNames() + { + return signature.getColumnNames(); + } + + @Override + public int numRows() + { + return frame.numRows(); + } + + @SuppressWarnings("unchecked") + @Nullable + @Override + public T as(Class clazz) + { + if (CursorFactory.class.equals(clazz)) { + return (T) FrameReader.create(signature).makeCursorFactory(frame); + } + return FrameRowsAndColumns.super.as(clazz); + } + + @Override + public void close() + { + // nothing to close + } + + @Override + public int hashCode() + { + return Objects.hashCode(frame, signature); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (!(o instanceof AbstractFrameRowsAndColumns)) { + return false; + } + AbstractFrameRowsAndColumns otherFrame = (AbstractFrameRowsAndColumns) o; + + return frame.writableMemory().equals(otherFrame.frame.writableMemory()) && signature.equals(otherFrame.signature); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumns.java index 71c2541b387..c4a4577dc1a 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumns.java @@ -19,46 +19,21 @@ package org.apache.druid.query.rowsandcols.concrete; +import org.apache.druid.error.DruidException; import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; -import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.read.columnar.FrameColumnReaders; -import org.apache.druid.frame.segment.FrameStorageAdapter; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; -import org.apache.druid.segment.CloseableShapeshifter; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; -import java.util.Collection; -import java.util.LinkedHashMap; -public class ColumnBasedFrameRowsAndColumns implements RowsAndColumns, AutoCloseable, CloseableShapeshifter +public class ColumnBasedFrameRowsAndColumns extends AbstractFrameRowsAndColumns { - private final Frame frame; - private final RowSignature signature; - private final LinkedHashMap colCache = new LinkedHashMap<>(); - public ColumnBasedFrameRowsAndColumns(Frame frame, RowSignature signature) { - this.frame = FrameType.COLUMNAR.ensureType(frame); - this.signature = signature; - } - - @Override - public Collection getColumnNames() - { - return signature.getColumnNames(); - } - - @Override - public int numRows() - { - return frame.numRows(); + super(FrameType.COLUMNAR.ensureType(frame), signature); } @Nullable @@ -73,28 +48,17 @@ public class ColumnBasedFrameRowsAndColumns implements RowsAndColumns, AutoClose } else { final ColumnType columnType = signature .getColumnType(columnIndex) - .orElseThrow(() -> new ISE("just got the id, why is columnType not there?")); + .orElseThrow( + () -> DruidException.defensive( + "just got the id [%s][%s], why is columnType not there?", + columnIndex, + name + ) + ); colCache.put(name, FrameColumnReaders.create(name, columnIndex, columnType).readRACColumn(frame)); } } return colCache.get(name); } - - @SuppressWarnings("unchecked") - @Nullable - @Override - public T as(Class clazz) - { - if (StorageAdapter.class.equals(clazz)) { - return (T) new FrameStorageAdapter(frame, FrameReader.create(signature), Intervals.ETERNITY); - } - return null; - } - - @Override - public void close() - { - // nothing to close - } } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorHolderFactory.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java similarity index 72% rename from processing/src/main/java/org/apache/druid/segment/CursorHolderFactory.java rename to processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java index e191f033f59..022a0f91ac1 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorHolderFactory.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java @@ -17,9 +17,15 @@ * under the License. */ -package org.apache.druid.segment; +package org.apache.druid.query.rowsandcols.concrete; -public interface CursorHolderFactory +import org.apache.druid.frame.Frame; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.segment.column.RowSignature; + +public interface FrameRowsAndColumns extends RowsAndColumns { - CursorHolder makeCursorHolder(CursorBuildSpec spec); + Frame getFrame(); + + RowSignature getSignature(); } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java index 73fc72a1ee4..35f42f6b9e4 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java @@ -26,9 +26,9 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.segment.CloseableShapeshifter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ColumnHolder; import javax.annotation.Nonnull; @@ -109,9 +109,9 @@ public class QueryableIndexRowsAndColumns implements RowsAndColumns, AutoCloseab @SuppressWarnings("unused") @SemanticCreator - public StorageAdapter toStorageAdapter() + public CursorFactory toCursorFactory() { - return new QueryableIndexStorageAdapter(index); + return new QueryableIndexCursorFactory(index); } @SuppressWarnings("unused") diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/RowBasedFrameRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/RowBasedFrameRowsAndColumns.java index fa17984e9ba..c702c210775 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/RowBasedFrameRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/RowBasedFrameRowsAndColumns.java @@ -24,42 +24,17 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.field.FieldReader; import org.apache.druid.frame.field.FieldReaders; -import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.frame.segment.FrameStorageAdapter; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; -import org.apache.druid.segment.CloseableShapeshifter; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; -import java.util.Collection; -import java.util.LinkedHashMap; -public class RowBasedFrameRowsAndColumns implements RowsAndColumns, AutoCloseable, CloseableShapeshifter +public class RowBasedFrameRowsAndColumns extends AbstractFrameRowsAndColumns { - private final Frame frame; - private final RowSignature signature; - private final LinkedHashMap colCache = new LinkedHashMap<>(); - public RowBasedFrameRowsAndColumns(Frame frame, RowSignature signature) { - this.frame = FrameType.ROW_BASED.ensureType(frame); - this.signature = signature; - } - - @Override - public Collection getColumnNames() - { - return signature.getColumnNames(); - } - - @Override - public int numRows() - { - return frame.numRows(); + super(FrameType.ROW_BASED.ensureType(frame), signature); } @Nullable @@ -88,21 +63,4 @@ public class RowBasedFrameRowsAndColumns implements RowsAndColumns, AutoCloseabl } return colCache.get(name); } - - @SuppressWarnings("unchecked") - @Nullable - @Override - public T as(Class clazz) - { - if (StorageAdapter.class.equals(clazz)) { - return (T) new FrameStorageAdapter(frame, FrameReader.create(signature), Intervals.ETERNITY); - } - return null; - } - - @Override - public void close() - { - // nothing to close - } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index 9dd72d15fa4..a24dc78468b 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -38,10 +38,10 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; @@ -75,18 +75,18 @@ public class ScanQueryEngine if (numScannedRows != null && numScannedRows >= query.getScanRowsLimit() && query.getTimeOrder().equals(Order.NONE)) { return Sequences.empty(); } - final boolean hasTimeout = query.context().hasTimeout(); - final Long timeoutAt = responseContext.getTimeoutTime(); - final StorageAdapter adapter = segment.asStorageAdapter(); - - if (adapter == null) { - throw new ISE( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." - ); + if (segment.isTombstone()) { + return Sequences.empty(); } - if (adapter.isFromTombstone()) { - return Sequences.empty(); + final boolean hasTimeout = query.context().hasTimeout(); + final Long timeoutAt = responseContext.getTimeoutTime(); + final CursorFactory cursorFactory = segment.asCursorFactory(); + + if (cursorFactory == null) { + throw new ISE( + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." + ); } final List allColumns = new ArrayList<>(); @@ -99,7 +99,7 @@ public class ScanQueryEngine } else { final Set availableColumns = Sets.newLinkedHashSet( Iterables.concat( - adapter.getRowSignature().getColumnNames(), + cursorFactory.getRowSignature().getColumnNames(), Iterables.transform( Arrays.asList(query.getVirtualColumns().getVirtualColumns()), VirtualColumn::getOutputName @@ -118,7 +118,7 @@ public class ScanQueryEngine // If the row count is not set, set it to 0, else do nothing. responseContext.addRowScanCount(0); final long limit = calculateRemainingScanRowsLimit(query, responseContext); - final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, queryMetrics)); + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(query, queryMetrics)); if (Order.NONE != query.getTimeOrder() && Cursors.getTimeOrdering(cursorHolder.getOrdering()) != query.getTimeOrder()) { final String failureReason = StringUtils.format( diff --git a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java index b6d0cb5f11a..01ad04e7aa8 100644 --- a/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/CursorOnlyStrategy.java @@ -26,10 +26,10 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.search.SearchQueryRunner.SearchColumnSelectorStrategy; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.filter.Filters; import java.util.List; @@ -51,13 +51,14 @@ public class CursorOnlyStrategy extends SearchStrategy @Override public List getExecutionPlan(SearchQuery query, Segment segment) { - final StorageAdapter adapter = segment.asStorageAdapter(); - final List dimensionSpecs = getDimsToSearch(adapter.getAvailableDimensions(), query.getDimensions()); - return ImmutableList.of(new CursorBasedExecutor( - query, - segment, - dimensionSpecs - )); + final List dimensionSpecs = getDimsToSearch(segment, query.getDimensions()); + return ImmutableList.of( + new CursorBasedExecutor( + query, + segment, + dimensionSpecs + ) + ); } public static class CursorBasedExecutor extends SearchQueryExecutor @@ -74,7 +75,7 @@ public class CursorOnlyStrategy extends SearchStrategy @Override public Object2IntRBTreeMap execute(final int limit) { - final StorageAdapter adapter = segment.asStorageAdapter(); + final CursorFactory adapter = segment.asCursorFactory(); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(query.getSingleInterval()) .setFilter( diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchStrategy.java b/processing/src/main/java/org/apache/druid/query/search/SearchStrategy.java index 9306301d13e..cbc0a91fb5c 100644 --- a/processing/src/main/java/org/apache/druid/query/search/SearchStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/SearchStrategy.java @@ -23,14 +23,22 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.Druids; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.Filter; +import org.apache.druid.segment.Metadata; +import org.apache.druid.segment.PhysicalSegmentInspector; +import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.Filters; import org.joda.time.Interval; +import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.List; +import java.util.Set; public abstract class SearchStrategy { @@ -49,10 +57,36 @@ public abstract class SearchStrategy public abstract List getExecutionPlan(SearchQuery query, Segment segment); - static List getDimsToSearch(Indexed availableDimensions, List dimensions) + static List getDimsToSearch(Segment segment, List dimensions) { if (dimensions == null || dimensions.isEmpty()) { - return ImmutableList.copyOf(Iterables.transform(availableDimensions, Druids.DIMENSION_IDENTITY)); + final Set dims = new LinkedHashSet<>(); + final QueryableIndex index = segment.as(QueryableIndex.class); + if (index != null) { + for (String dim : index.getAvailableDimensions()) { + dims.add(dim); + } + } else { + // fallback to RowSignature and Metadata if QueryableIndex not available + final PhysicalSegmentInspector segmentInspector = segment.as(PhysicalSegmentInspector.class); + final Metadata metadata = segmentInspector != null ? segmentInspector.getMetadata() : null; + final Set ignore = new HashSet<>(); + ignore.add(ColumnHolder.TIME_COLUMN_NAME); + if (metadata != null && metadata.getAggregators() != null) { + for (AggregatorFactory factory : metadata.getAggregators()) { + ignore.add(factory.getName()); + } + } + final RowSignature rowSignature = segment.asCursorFactory().getRowSignature(); + for (String columnName : rowSignature.getColumnNames()) { + if (!ignore.contains(columnName)) { + dims.add(columnName); + } + } + } + return ImmutableList.copyOf( + Iterables.transform(dims, Druids.DIMENSION_IDENTITY) + ); } else { return dimensions; } diff --git a/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java b/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java index d5af263a1c7..bd33d7f9783 100644 --- a/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java @@ -35,11 +35,11 @@ import org.apache.druid.query.filter.ColumnIndexSelector; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.search.CursorOnlyStrategy.CursorBasedExecutor; import org.apache.druid.segment.ColumnSelectorColumnIndexSelector; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.DeprecatedQueryableIndexColumnSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; @@ -74,14 +74,15 @@ public class UseIndexesStrategy extends SearchStrategy public List getExecutionPlan(SearchQuery query, Segment segment) { final ImmutableList.Builder builder = ImmutableList.builder(); - final QueryableIndex index = segment.asQueryableIndex(); - final StorageAdapter adapter = segment.asStorageAdapter(); - final List searchDims = getDimsToSearch(adapter.getAvailableDimensions(), query.getDimensions()); + final QueryableIndex index = segment.as(QueryableIndex.class); + final CursorFactory cursorFactory = segment.asCursorFactory(); + final List searchDims = getDimsToSearch(segment, query.getDimensions()); if (index != null) { // pair of bitmap dims and non-bitmap dims final Pair, List> pair = partitionDimensionList( - adapter, + segment, + cursorFactory, query.getVirtualColumns(), searchDims ); @@ -131,7 +132,8 @@ public class UseIndexesStrategy extends SearchStrategy * Note that the returned lists are free to modify. */ private static Pair, List> partitionDimensionList( - StorageAdapter adapter, + Segment segment, + CursorFactory cursorFactory, VirtualColumns virtualColumns, List dimensions ) @@ -139,10 +141,10 @@ public class UseIndexesStrategy extends SearchStrategy final List bitmapDims = new ArrayList<>(); final List nonBitmapDims = new ArrayList<>(); final List dimsToSearch = getDimsToSearch( - adapter.getAvailableDimensions(), + segment, dimensions ); - VirtualizedColumnInspector columnInspector = new VirtualizedColumnInspector(adapter, virtualColumns); + VirtualizedColumnInspector columnInspector = new VirtualizedColumnInspector(cursorFactory, virtualColumns); for (DimensionSpec spec : dimsToSearch) { ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(spec.getDimension()); @@ -263,7 +265,7 @@ public class UseIndexesStrategy extends SearchStrategy @Override public Object2IntRBTreeMap execute(int limit) { - final QueryableIndex index = segment.asQueryableIndex(); + final QueryableIndex index = segment.as(QueryableIndex.class); Preconditions.checkArgument(index != null, "Index should not be null"); ColumnSelectorColumnIndexSelector indexSelector = new ColumnSelectorColumnIndexSelector( diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index ec9e27c6f38..f90ff67615e 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -40,11 +40,10 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.CursorHolderFactory; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.filter.Filters; @@ -95,13 +94,15 @@ public class TimeBoundaryQueryRunnerFactory private static class TimeBoundaryQueryRunner implements QueryRunner> { - private final StorageAdapter adapter; + private final CursorFactory cursorFactory; + private final Interval dataInterval; @Nullable private final TimeBoundaryInspector timeBoundaryInspector; public TimeBoundaryQueryRunner(Segment segment) { - this.adapter = segment.asStorageAdapter(); + this.cursorFactory = segment.asCursorFactory(); + this.dataInterval = segment.getDataInterval(); this.timeBoundaryInspector = segment.as(TimeBoundaryInspector.class); } @@ -124,9 +125,9 @@ public class TimeBoundaryQueryRunnerFactory @Override public Iterator> make() { - if (adapter == null) { + if (cursorFactory == null) { throw new ISE( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." ); } @@ -142,13 +143,13 @@ public class TimeBoundaryQueryRunnerFactory maxTime = timeBoundaryInspector.getMaxTime(); } } else { - final Pair timeBoundary = getTimeBoundary(query, adapter); + final Pair timeBoundary = getTimeBoundary(query, cursorFactory); minTime = timeBoundary.left(); maxTime = timeBoundary.right(); } return query.buildResult( - adapter.getInterval().getStart(), + dataInterval.getStart(), minTime, maxTime ).iterator(); @@ -168,7 +169,7 @@ public class TimeBoundaryQueryRunnerFactory * Whether a particular {@link TimeBoundaryQuery} can use {@link TimeBoundaryInspector#getMinTime()} and/or * {@link TimeBoundaryInspector#getMaxTime()}. * - * If false, must use {@link StorageAdapter#makeCursorHolder(CursorBuildSpec)}. + * If false, must use {@link CursorFactory#makeCursorHolder(CursorBuildSpec)}. */ private static boolean canUseTimeBoundaryInspector( final TimeBoundaryQuery query, @@ -211,7 +212,7 @@ public class TimeBoundaryQueryRunnerFactory private static Pair getTimeBoundary( final TimeBoundaryQuery query, - final CursorHolderFactory cursorHolderFactory + final CursorFactory cursorFactory ) { DateTime minTime = null, maxTime = null; @@ -222,7 +223,7 @@ public class TimeBoundaryQueryRunnerFactory .setPreferredOrdering(Cursors.ascendingTimeOrder()) .build(); - try (final CursorHolder cursorHolder = cursorHolderFactory.makeCursorHolder(cursorSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(cursorSpec)) { if (cursorHolder.getTimeOrder() == Order.ASCENDING) { // Time-ordered cursor, use the first timestamp. minTime = getFirstTimestamp(query, cursorHolder); @@ -239,7 +240,7 @@ public class TimeBoundaryQueryRunnerFactory .setPreferredOrdering(Cursors.descendingTimeOrder()) .build(); - try (final CursorHolder cursorHolder = cursorHolderFactory.makeCursorHolder(cursorSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(cursorSpec)) { if (cursorHolder.getTimeOrder() == Order.DESCENDING) { // Time-ordered cursor, use the first timestamp. maxTime = getFirstTimestamp(query, cursorHolder); diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java index 6e2cb62adcf..88d488f85b9 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQuery.java @@ -243,6 +243,11 @@ public class TimeseriesQuery extends BaseQuery> return Druids.TimeseriesQueryBuilder.copy(this).filters(dimFilter).build(); } + public TimeseriesQuery withAggregatorSpecs(List aggregatorSpecs) + { + return Druids.TimeseriesQueryBuilder.copy(this).aggregators(aggregatorSpecs).build(); + } + public TimeseriesQuery withPostAggregatorSpecs(final List postAggregatorSpecs) { return Druids.TimeseriesQueryBuilder.copy(this).postAggregators(postAggregatorSpecs).build(); diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java index 95cba2e647f..dbec221248e 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java @@ -38,14 +38,15 @@ import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorAdapters; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.vector.VectorCursorGranularizer; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.SegmentMissingException; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -86,22 +87,25 @@ public class TimeseriesQueryEngine * scoped down to a single interval before calling this method. */ public Sequence> process( - final TimeseriesQuery query, - final StorageAdapter adapter, + TimeseriesQuery query, + final CursorFactory cursorFactory, @Nullable TimeBoundaryInspector timeBoundaryInspector, @Nullable final TimeseriesQueryMetrics timeseriesQueryMetrics ) { - if (adapter == null) { + if (cursorFactory == null) { throw new SegmentMissingException( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." ); } final Interval interval = Iterables.getOnlyElement(query.getIntervals()); final Granularity gran = query.getGranularity(); - final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics)); + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics)); + if (cursorHolder.isPreAggregated()) { + query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs())); + } try { final Sequence> result; diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java index 788e72015e2..2839f310962 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java @@ -32,8 +32,8 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.QueryWatcher; import org.apache.druid.query.Result; import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import javax.annotation.Nullable; @@ -62,7 +62,7 @@ public class TimeseriesQueryRunnerFactory @Override public QueryRunner> createRunner(final Segment segment) { - return new TimeseriesQueryRunner(engine, segment.asStorageAdapter(), segment.as(TimeBoundaryInspector.class)); + return new TimeseriesQueryRunner(engine, segment.asCursorFactory(), segment.as(TimeBoundaryInspector.class)); } @Override @@ -83,18 +83,18 @@ public class TimeseriesQueryRunnerFactory private static class TimeseriesQueryRunner implements QueryRunner> { private final TimeseriesQueryEngine engine; - private final StorageAdapter adapter; + private final CursorFactory cursorFactory; @Nullable private final TimeBoundaryInspector timeBoundaryInspector; private TimeseriesQueryRunner( TimeseriesQueryEngine engine, - StorageAdapter adapter, + CursorFactory cursorFactory, @Nullable TimeBoundaryInspector timeBoundaryInspector ) { this.engine = engine; - this.adapter = adapter; + this.cursorFactory = cursorFactory; this.timeBoundaryInspector = timeBoundaryInspector; } @@ -111,11 +111,10 @@ public class TimeseriesQueryRunnerFactory return engine.process( (TimeseriesQuery) input, - adapter, + cursorFactory, timeBoundaryInspector, (TimeseriesQueryMetrics) queryPlus.getQueryMetrics() ); } } - } diff --git a/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java index 99759f515dd..a4605c3f265 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java @@ -28,7 +28,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.StorageAdapter; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -47,18 +46,18 @@ import java.util.List; */ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm { - private final StorageAdapter storageAdapter; private final TopNQuery query; + private final TopNCursorInspector cursorInspector; private final NonBlockingPool bufferPool; public AggregateTopNMetricFirstAlgorithm( - StorageAdapter storageAdapter, TopNQuery query, + TopNCursorInspector cursorInspector, NonBlockingPool bufferPool ) { - this.storageAdapter = storageAdapter; this.query = query; + this.cursorInspector = cursorInspector; this.bufferPool = bufferPool; } @@ -90,7 +89,7 @@ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm interval.contains(storageAdapter.getInterval()))) { + topNOptimizationInspector != null && + topNOptimizationInspector.areAllDictionaryIdsPresent() && + query.getIntervals().stream().anyMatch(interval -> interval.contains(cursorInspector.getDataInterval()))) { endIndex = Math.min(endIndex, startIndex + query.getThreshold()); } diff --git a/processing/src/main/java/org/apache/druid/query/topn/HeapBasedTopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/HeapBasedTopNAlgorithm.java index 6f8ed23bc29..c8986c0f342 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/HeapBasedTopNAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/HeapBasedTopNAlgorithm.java @@ -24,7 +24,6 @@ import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.StorageAdapter; /** * Heap based topn algorithm that handles aggregates on dimension extractions and numeric typed dimension columns. @@ -38,11 +37,11 @@ public class HeapBasedTopNAlgorithm private final TopNQuery query; public HeapBasedTopNAlgorithm( - StorageAdapter storageAdapter, - TopNQuery query + TopNQuery query, + TopNCursorInspector cursorInspector ) { - super(storageAdapter); + super(cursorInspector); this.query = query; } @@ -68,7 +67,7 @@ public class HeapBasedTopNAlgorithm throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality"); } ColumnSelectorPlus selectorPlus = params.getSelectorPlus(); - return selectorPlus.getColumnSelectorStrategy().getRowSelector(query, params, storageAdapter); + return selectorPlus.getColumnSelectorStrategy().getRowSelector(query, params, cursorInspector); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/topn/PooledTopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/PooledTopNAlgorithm.java index d9213b94e09..d2ba1674621 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/PooledTopNAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/PooledTopNAlgorithm.java @@ -35,7 +35,6 @@ import org.apache.druid.query.monomorphicprocessing.StringRuntimeShape; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.FilteredOffset; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.Offset; @@ -212,12 +211,12 @@ public class PooledTopNAlgorithm private static final int AGG_UNROLL_COUNT = 8; // Must be able to fit loop below public PooledTopNAlgorithm( - StorageAdapter storageAdapter, TopNQuery query, + TopNCursorInspector cursorInspector, NonBlockingPool bufferPool ) { - super(storageAdapter); + super(cursorInspector); this.query = query; this.bufferPool = bufferPool; } @@ -232,11 +231,7 @@ public class PooledTopNAlgorithm throw new UnsupportedOperationException("Cannot operate on a dimension with no dictionary"); } - final TopNMetricSpecBuilder arrayProvider = new BaseArrayProvider( - dimSelector, - query, - storageAdapter - ) + final TopNMetricSpecBuilder arrayProvider = new BaseArrayProvider(dimSelector, query, cursorInspector) { private final int[] positions = new int[cardinality]; diff --git a/processing/src/main/java/org/apache/druid/query/topn/TimeExtractionTopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/TimeExtractionTopNAlgorithm.java index 7e5d43b96ad..0c79e7c8d31 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TimeExtractionTopNAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TimeExtractionTopNAlgorithm.java @@ -25,7 +25,6 @@ import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import java.util.HashMap; @@ -39,9 +38,9 @@ public class TimeExtractionTopNAlgorithm extends BaseTopNAlgorithm dimensionValueConverter; - public TimeExtractionTopNAlgorithm(StorageAdapter storageAdapter, TopNQuery query) + public TimeExtractionTopNAlgorithm(TopNQuery query, TopNCursorInspector cursorInspector) { - super(storageAdapter); + super(cursorInspector); this.query = query; // This strategy is used for ExtractionFns on the __time column. They always return STRING, so we need to convert diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNCursorInspector.java b/processing/src/main/java/org/apache/druid/query/topn/TopNCursorInspector.java new file mode 100644 index 00000000000..c5d5d117a53 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNCursorInspector.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.query.topn; + +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.TopNOptimizationInspector; +import org.joda.time.Interval; + +import javax.annotation.Nullable; + +public class TopNCursorInspector +{ + private final ColumnInspector columnInspector; + @Nullable + private final TopNOptimizationInspector topNOptimizationInspector; + private final Interval dataInterval; + + private final int dimensionCardinality; + + public TopNCursorInspector( + ColumnInspector columnInspector, + @Nullable TopNOptimizationInspector topNOptimizationInspector, + Interval dataInterval, + int dimensionCardinality + ) + { + this.columnInspector = columnInspector; + this.topNOptimizationInspector = topNOptimizationInspector; + this.dataInterval = dataInterval; + this.dimensionCardinality = dimensionCardinality; + } + + public ColumnInspector getColumnInspector() + { + return columnInspector; + } + + @Nullable + public TopNOptimizationInspector getOptimizationInspector() + { + return topNOptimizationInspector; + } + + public Interval getDataInterval() + { + return dataInterval; + } + + public int getDimensionCardinality() + { + return dimensionCardinality; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNMapFn.java b/processing/src/main/java/org/apache/druid/query/topn/TopNMapFn.java index dafff313afb..5a48d4a0d28 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNMapFn.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNMapFn.java @@ -23,9 +23,7 @@ import org.apache.druid.query.ColumnSelectorPlus; import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.Result; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor; -import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessorFactory; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.DimensionHandlerUtils; import javax.annotation.Nullable; @@ -45,19 +43,13 @@ public class TopNMapFn @SuppressWarnings("unchecked") @Nullable - public Result apply(final Cursor cursor, final CursorGranularizer granularizer, final @Nullable TopNQueryMetrics queryMetrics) + public Result apply( + final Cursor cursor, + ColumnSelectorPlus> selectorPlus, + final CursorGranularizer granularizer, + final @Nullable TopNQueryMetrics queryMetrics + ) { - final ColumnSelectorPlus> selectorPlus = - DimensionHandlerUtils.createColumnSelectorPlus( - new TopNColumnAggregatesProcessorFactory(query.getDimensionSpec().getOutputType()), - query.getDimensionSpec(), - cursor.getColumnSelectorFactory() - ); - - if (selectorPlus.getSelector() == null) { - return null; - } - TopNParams params = null; try { params = topNAlgorithm.makeInitParams(selectorPlus, cursor, granularizer); diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index c6561faeee3..d10d26242e3 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -25,17 +25,26 @@ import org.apache.druid.collections.ResourceHolder; 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.Sequences; +import org.apache.druid.query.ColumnSelectorPlus; import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor; +import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessorFactory; +import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; +import org.apache.druid.segment.DimensionDictionarySelector; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentMissingException; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; +import org.apache.druid.segment.TopNOptimizationInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.Types; @@ -59,32 +68,58 @@ public class TopNQueryEngine } /** - * Do the thing - process a {@link StorageAdapter} into a {@link Sequence} of {@link TopNResultValue}, with one of the + * Do the thing - process a {@link Segment} into a {@link Sequence} of {@link TopNResultValue}, with one of the * fine {@link TopNAlgorithm} available chosen based on the type of column being aggregated. The algorithm provides a * mapping function to process rows from the adapter {@link Cursor} to apply {@link AggregatorFactory} and create or * update {@link TopNResultValue} */ public Sequence> query( - final TopNQuery query, - final StorageAdapter adapter, - @Nullable final TimeBoundaryInspector timeBoundaryInspector, + TopNQuery query, + final Segment segment, @Nullable final TopNQueryMetrics queryMetrics ) { - if (adapter == null) { + final CursorFactory cursorFactory = segment.asCursorFactory(); + if (cursorFactory == null) { throw new SegmentMissingException( - "Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped." + "Null cursor factory found. Probably trying to issue a query against a segment being memory unmapped." ); } - final TopNMapFn mapFn = getMapFn(query, adapter, queryMetrics); - final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, queryMetrics); - final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec); + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec); + if (cursorHolder.isPreAggregated()) { + query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs())); + } final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return Sequences.withBaggage(Sequences.empty(), cursorHolder); } + + final TimeBoundaryInspector timeBoundaryInspector = segment.as(TimeBoundaryInspector.class); + + final ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); + + final ColumnSelectorPlus> selectorPlus = + DimensionHandlerUtils.createColumnSelectorPlus( + new TopNColumnAggregatesProcessorFactory(query.getDimensionSpec().getOutputType()), + query.getDimensionSpec(), + factory + ); + + final int cardinality; + if (selectorPlus.getSelector() instanceof DimensionDictionarySelector) { + cardinality = ((DimensionDictionarySelector) selectorPlus.getSelector()).getValueCardinality(); + } else { + cardinality = DimensionDictionarySelector.CARDINALITY_UNKNOWN; + } + final TopNCursorInspector cursorInspector = new TopNCursorInspector( + factory, + segment.as(TopNOptimizationInspector.class), + segment.getDataInterval(), + cardinality + ); + final CursorGranularizer granularizer = CursorGranularizer.create( cursor, timeBoundaryInspector, @@ -92,18 +127,19 @@ public class TopNQueryEngine query.getGranularity(), buildSpec.getInterval() ); - if (granularizer == null) { + if (granularizer == null || selectorPlus.getSelector() == null) { return Sequences.withBaggage(Sequences.empty(), cursorHolder); } if (queryMetrics != null) { queryMetrics.cursor(cursor); } + final TopNMapFn mapFn = getMapFn(query, cursorInspector, queryMetrics); return Sequences.filter( Sequences.simple(granularizer.getBucketIterable()) .map(bucketInterval -> { granularizer.advanceToBucket(bucketInterval); - return mapFn.apply(cursor, granularizer, queryMetrics); + return mapFn.apply(cursor, selectorPlus, granularizer, queryMetrics); }), Predicates.notNull() ).withBaggage(cursorHolder); @@ -114,14 +150,15 @@ public class TopNQueryEngine */ private TopNMapFn getMapFn( final TopNQuery query, - final StorageAdapter adapter, + final TopNCursorInspector cursorInspector, final @Nullable TopNQueryMetrics queryMetrics ) { final String dimension = query.getDimensionSpec().getDimension(); - final int cardinality = adapter.getDimensionCardinality(dimension); + + if (queryMetrics != null) { - queryMetrics.dimensionCardinality(cardinality); + queryMetrics.dimensionCardinality(cursorInspector.getDimensionCardinality()); } int numBytesPerRecord = 0; @@ -129,27 +166,43 @@ public class TopNQueryEngine numBytesPerRecord += aggregatorFactory.getMaxIntermediateSizeWithNulls(); } - final TopNAlgorithmSelector selector = new TopNAlgorithmSelector(cardinality, numBytesPerRecord); + final TopNAlgorithmSelector selector = new TopNAlgorithmSelector( + cursorInspector.getDimensionCardinality(), + numBytesPerRecord + ); query.initTopNAlgorithmSelector(selector); - final ColumnCapabilities columnCapabilities = query.getVirtualColumns() - .getColumnCapabilitiesWithFallback(adapter, dimension); - + final ColumnCapabilities columnCapabilities = query.getVirtualColumns().getColumnCapabilitiesWithFallback( + cursorInspector.getColumnInspector(), + dimension + ); final TopNAlgorithm topNAlgorithm; - if (canUsePooledAlgorithm(selector, query, columnCapabilities, bufferPool, cardinality, numBytesPerRecord)) { + if (canUsePooledAlgorithm(selector, query, columnCapabilities, bufferPool, cursorInspector.getDimensionCardinality(), numBytesPerRecord)) { // pool based algorithm selection, if we can if (selector.isAggregateAllMetrics()) { // if sorted by dimension we should aggregate all metrics in a single pass, use the regular pooled algorithm for // this - topNAlgorithm = new PooledTopNAlgorithm(adapter, query, bufferPool); + topNAlgorithm = new PooledTopNAlgorithm( + query, + cursorInspector, + bufferPool + ); } else if (shouldUseAggregateMetricFirstAlgorithm(query, selector)) { // for high cardinality dimensions with larger result sets we aggregate with only the ordering aggregation to // compute the first 'n' values, and then for the rest of the metrics but for only the 'n' values - topNAlgorithm = new AggregateTopNMetricFirstAlgorithm(adapter, query, bufferPool); + topNAlgorithm = new AggregateTopNMetricFirstAlgorithm( + query, + cursorInspector, + bufferPool + ); } else { // anything else, use the regular pooled algorithm - topNAlgorithm = new PooledTopNAlgorithm(adapter, query, bufferPool); + topNAlgorithm = new PooledTopNAlgorithm( + query, + cursorInspector, + bufferPool + ); } } else { // heap based algorithm selection, if we must @@ -161,9 +214,9 @@ public class TopNQueryEngine // A special TimeExtractionTopNAlgorithm is required since HeapBasedTopNAlgorithm // currently relies on the dimension cardinality to support lexicographic sorting - topNAlgorithm = new TimeExtractionTopNAlgorithm(adapter, query); + topNAlgorithm = new TimeExtractionTopNAlgorithm(query, cursorInspector); } else { - topNAlgorithm = new HeapBasedTopNAlgorithm(adapter, query); + topNAlgorithm = new HeapBasedTopNAlgorithm(query, cursorInspector); } } if (queryMetrics != null) { diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java index a81d9258c86..0229bdbcc59 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java @@ -34,7 +34,6 @@ import org.apache.druid.query.QueryWatcher; import org.apache.druid.query.Result; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.TimeBoundaryInspector; import java.nio.ByteBuffer; @@ -78,8 +77,7 @@ public class TopNQueryRunnerFactory implements QueryRunnerFactory extends Column * * A dimension type that does not have integer values should return null. * - * @param query The TopN query being served - * @param params Parameters for the TopN query being served - * @param storageAdapter Column storage adapter, to provide information about the column that can be used for - * query optimization, e.g. whether dimension values are sorted or not + * @param query The TopN query being served + * @param params Parameters for the TopN query being served + * @param cursorInspector provide information about the cursor that can be used for query optimization, e.g. whether + * dimension values are sorted or not * * @return an Aggregator[][] for integer-valued dimensions, null otherwise */ @Nullable - Aggregator[][] getRowSelector(TopNQuery query, TopNParams params, StorageAdapter storageAdapter); + Aggregator[][] getRowSelector(TopNQuery query, TopNParams params, TopNCursorInspector cursorInspector); /** * Used by {@link HeapBasedTopNAlgorithm}. The contract of this method requires calling {@link #initAggregateStore()} diff --git a/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java b/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java index 4fa8d3b45b3..19491403522 100644 --- a/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/ArrayListSegment.java @@ -19,39 +19,34 @@ package org.apache.druid.segment; -import com.google.common.base.Preconditions; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.rowsandcols.ArrayListRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.timeline.SegmentId; -import org.joda.time.Interval; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.ArrayList; /** * A {@link Segment} that is based on a stream of objects. */ -public class ArrayListSegment implements Segment +public class ArrayListSegment extends RowBasedSegment { - private final SegmentId segmentId; private final ArrayList rows; - private final RowAdapter rowAdapter; + private final RowAdapter adapter; private final RowSignature rowSignature; /** * Create a list-based segment. *

    * The provided List must be in time-order according to the provided {@link RowAdapter#timestampFunction()}. - * The cursor returned by {@link RowBasedStorageAdapter#makeCursorHolder(CursorBuildSpec)} makes no attempt to verify + * The cursor returned by {@link RowBasedCursorFactory#makeCursorHolder(CursorBuildSpec)} makes no attempt to verify * this, and callers will expect it. *

    * The provided "rowSignature" will be used for reporting available columns and their capabilities to users of - * {@link #asStorageAdapter()}. Note that the {@link ColumnSelectorFactory} implementation returned by this segment's - * storage adapter will allow creation of selectors on any field, using the {@link RowAdapter#columnFunction} for that + * {@link #asCursorFactory()}. Note that the {@link ColumnSelectorFactory} implementation returned by this segment's + * cursor factory will allow creation of selectors on any field, using the {@link RowAdapter#columnFunction} for that * field, even if it doesn't appear in "rowSignature". * * @param segmentId segment identifier; will be returned by {@link #getId()} @@ -67,40 +62,12 @@ public class ArrayListSegment implements Segment final RowSignature rowSignature ) { - this.segmentId = Preconditions.checkNotNull(segmentId, "segmentId"); + super(segmentId, Sequences.simple(rows), rowAdapter, rowSignature); this.rows = rows; - this.rowAdapter = rowAdapter; + this.adapter = rowAdapter; this.rowSignature = rowSignature; } - @Override - @Nonnull - public SegmentId getId() - { - return segmentId; - } - - @Override - @Nonnull - public Interval getDataInterval() - { - return Intervals.ETERNITY; - } - - @Nullable - @Override - public QueryableIndex asQueryableIndex() - { - return null; - } - - @Override - @Nonnull - public StorageAdapter asStorageAdapter() - { - return new RowBasedStorageAdapter<>(Sequences.simple(rows), rowAdapter, rowSignature); - } - @Nullable @Override @SuppressWarnings("unchecked") @@ -109,7 +76,7 @@ public class ArrayListSegment implements Segment if (CloseableShapeshifter.class.equals(clazz)) { return (T) new MyCloseableShapeshifter(); } - return Segment.super.as(clazz); + return super.as(clazz); } @Override @@ -120,7 +87,7 @@ public class ArrayListSegment implements Segment private RowsAndColumns asRowsAndColumns() { - return new ArrayListRowsAndColumns<>(rows, rowAdapter, rowSignature); + return new ArrayListRowsAndColumns<>(rows, adapter, rowSignature); } private class MyCloseableShapeshifter implements CloseableShapeshifter diff --git a/processing/src/main/java/org/apache/druid/segment/CompleteSegment.java b/processing/src/main/java/org/apache/druid/segment/CompleteSegment.java new file mode 100644 index 00000000000..d44781774ae --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/CompleteSegment.java @@ -0,0 +1,81 @@ +/* + * 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.segment; + +import org.apache.druid.timeline.DataSegment; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; +import java.util.Objects; + +/** + * Contains the {@link DataSegment} and {@link Segment}. The datasegment could be null if the segment is a dummy, such + * as those created by {@link org.apache.druid.msq.input.inline.InlineInputSliceReader}. + */ +public class CompleteSegment implements Closeable +{ + @Nullable + private final DataSegment dataSegment; + private final Segment segment; + + public CompleteSegment(@Nullable DataSegment dataSegment, Segment segment) + { + this.dataSegment = dataSegment; + this.segment = segment; + } + + @Nullable + @SuppressWarnings("unused") + public DataSegment getDataSegment() + { + return dataSegment; + } + + public Segment getSegment() + { + return segment; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CompleteSegment that = (CompleteSegment) o; + return Objects.equals(dataSegment, that.dataSegment) && Objects.equals(segment, that.segment); + } + + @Override + public int hashCode() + { + return Objects.hash(dataSegment, segment); + } + + @Override + public void close() throws IOException + { + segment.close(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java index a1fc14c1054..eb7c72b09e9 100644 --- a/processing/src/main/java/org/apache/druid/segment/Cursor.java +++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java @@ -24,7 +24,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexCursorHolder; /** * Cursor is an interface for iteration over a range of data points, used during query execution. Cursors are available - * from {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} via {@link CursorHolder#asCursor()}. + * from {@link CursorFactory#makeCursorHolder(CursorBuildSpec)} via {@link CursorHolder#asCursor()}. *

    * A typical usage pattern might look something like this: *

    @@ -69,7 +69,8 @@ public interface Cursor
     
       /**
        * Advance to the cursor to the next position. Callers should check {@link #isDone()} or
    -   * {@link #isDoneOrInterrupted()} before getting the next value from a selector.
    +   * {@link #isDoneOrInterrupted()} before getting the next value from a selector. However, underlying
    +   * implementation may still check for thread interruption if advancing the cursor is a long-running operation.
        */
       void advanceUninterruptibly();
     
    diff --git a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java
    index 551430a00e3..b2547ebe38c 100644
    --- a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java
    +++ b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java
    @@ -19,100 +19,36 @@
     
     package org.apache.druid.segment;
     
    -import org.apache.druid.error.DruidException;
    -import org.apache.druid.java.util.common.granularity.Granularity;
    -import org.apache.druid.java.util.common.guava.Sequence;
    -import org.apache.druid.query.QueryMetrics;
    -import org.apache.druid.query.filter.Filter;
    -import org.apache.druid.segment.vector.VectorCursor;
    -import org.joda.time.Interval;
    +import org.apache.druid.segment.column.ColumnCapabilities;
    +import org.apache.druid.segment.column.RowSignature;
     
     import javax.annotation.Nullable;
     
    -/**
    - * Interface extended by {@link StorageAdapter}, which gives them the power to create cursors.
    - *
    - * @see StorageAdapter
    - *
    - * @deprecated This interface is deprecated and no longer implemented by any built-in {@link StorageAdapter}. Callers
    - * should use {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. Implementors should implement
    - * {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead.
    - */
    -@Deprecated
    -public interface CursorFactory
    +public interface CursorFactory extends ColumnInspector
     {
    -  /**
    -   * Returns true if the provided combination of parameters can be handled by "makeVectorCursor".
    -   *
    -   * Query engines should use this before running in vectorized mode, and be prepared to fall back to non-vectorized
    -   * mode if this method returns false.
    -   *
    -   * @deprecated Callers should use {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} and call
    -   * {@link CursorHolder#canVectorize()}.
    -   * Implementors should implement {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead.  This method is
    -   * no longer implemented by any built-in factories.
    -   */
    -  @Deprecated
    -  default boolean canVectorize(
    -      @Nullable Filter filter,
    -      VirtualColumns virtualColumns,
    -      boolean descending
    -  )
    -  {
    -    throw DruidException.defensive(
    -        "CursorFactory.canVectorize is no longer supported, use CursorHolderFactory.makeCursorHolder instead"
    -    );
    -  }
    +  CursorHolder makeCursorHolder(CursorBuildSpec spec);
     
       /**
    -   * Creates a sequence of Cursors, one for each time-granular bucket (based on the provided Granularity).
    -   *
    -   * @deprecated Callers should use {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} and call
    -   * {@link CursorHolder#asCursor()}.
    -   * Implementors should implement {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead.
    -   * This method is no longer implemented by any built-in factories.
    +   * Returns the {@link RowSignature} of the data available from this cursor factory. For mutable segments, even though
    +   * the signature may evolve over time, any particular object returned by this method is an immutable snapshot.
        */
    -  @Deprecated
    -  default Sequence makeCursors(
    -      @Nullable Filter filter,
    -      Interval interval,
    -      VirtualColumns virtualColumns,
    -      Granularity gran,
    -      boolean descending,
    -      @Nullable QueryMetrics queryMetrics
    -  )
    -  {
    -    throw DruidException.defensive(
    -        "CursorFactory.makeCursors is no longer supported, use CursorHolderFactory.makeCursorHolder instead"
    -    );
    -  }
    +  RowSignature getRowSignature();
     
       /**
    -   * Creates a VectorCursor. Unlike the Cursor returned by "makeCursor", there is just one of these. Hence, this method
    -   * does not take a "granularity" parameter. Before calling this method, check "canVectorize" to see if the call you
    -   * are about to make will throw an error or not.
    +   * Returns capabilities of a particular column, if known. May be null if the column doesn't exist, or if
    +   * the column does exist but the capabilities are unknown. The latter is possible with dynamically discovered
    +   * columns.
        *
    -   * Returns null if there is no data to walk over (for example, if the "interval" does not overlap the data interval
    -   * of this segment).
    +   * Note that CursorFactory are representations of "real" segments, so they are not aware of any virtual columns
    +   * that may be involved in a query. In general, query engines should instead use the method
    +   * {@link ColumnSelectorFactory#getColumnCapabilities(String)}, which returns capabilities for virtual columns as
    +   * well.
        *
    -   * @deprecated Callers should use {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} and call
    -   * {@link CursorHolder#asVectorCursor()}. Implementors should implement
    -   * {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} instead. This method is no longer implemented by any
    -   * built-in factories.
    +   * @param column column name
    +   *
    +   * @return capabilities, or null
        */
    -  @Deprecated
    +  @Override
       @Nullable
    -  default VectorCursor makeVectorCursor(
    -      @Nullable Filter filter,
    -      Interval interval,
    -      VirtualColumns virtualColumns,
    -      boolean descending,
    -      int vectorSize,
    -      @Nullable QueryMetrics queryMetrics
    -  )
    -  {
    -    throw DruidException.defensive(
    -        "CursorFactory.makeVectorCursor is no longer supported, use CursorHolderFactory.makeCursorHolder instead"
    -    );
    -  }
    +  ColumnCapabilities getColumnCapabilities(String column);
     }
    diff --git a/processing/src/main/java/org/apache/druid/segment/CursorHolder.java b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java
    index a70fd8757e1..79bf2b4e557 100644
    --- a/processing/src/main/java/org/apache/druid/segment/CursorHolder.java
    +++ b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java
    @@ -22,6 +22,7 @@ package org.apache.druid.segment;
     import org.apache.druid.java.util.common.UOE;
     import org.apache.druid.query.Order;
     import org.apache.druid.query.OrderBy;
    +import org.apache.druid.query.aggregation.AggregatorFactory;
     import org.apache.druid.segment.column.ColumnHolder;
     import org.apache.druid.segment.vector.VectorCursor;
     
    @@ -58,6 +59,22 @@ public interface CursorHolder extends Closeable
         return false;
       }
     
    +  /**
    +   * Returns true if the {@link Cursor} or {@link VectorCursor} contains pre-aggregated columns for all
    +   * {@link AggregatorFactory} specified in {@link CursorBuildSpec#getAggregators()}.
    +   * 

    + * If this method returns true, {@link ColumnSelectorFactory} and + * {@link org.apache.druid.segment.vector.VectorColumnSelectorFactory} created from {@link Cursor} and + * {@link VectorCursor} respectively will provide selectors for {@link AggregatorFactory#getName()}, and engines + * should rewrite the query using {@link AggregatorFactory#getCombiningFactory()}, since the values returned from + * these selectors will be of type {@link AggregatorFactory#getIntermediateType()}, so the cursor becomes a "fold" + * operation rather than a "build" operation. + */ + default boolean isPreAggregated() + { + return false; + } + /** * Returns cursor ordering, which may or may not match {@link CursorBuildSpec#getPreferredOrdering()}. If returns * an empty list then the cursor has no defined ordering. diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/FilteredCursorFactory.java similarity index 51% rename from processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java rename to processing/src/main/java/org/apache/druid/segment/FilteredCursorFactory.java index 46b9f5fe061..b1806f66342 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredCursorFactory.java @@ -24,22 +24,20 @@ import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.filter.AndFilter; -import org.joda.time.Interval; import javax.annotation.Nullable; -public class FilteredStorageAdapter implements StorageAdapter +public class FilteredCursorFactory implements CursorFactory { + private final CursorFactory delegate; @Nullable - private final DimFilter filterOnDataSource; - private final StorageAdapter baseStorageAdapter; + private final DimFilter filter; - public FilteredStorageAdapter(final StorageAdapter adapter, @Nullable final DimFilter filter) + public FilteredCursorFactory(CursorFactory delegate, @Nullable DimFilter filter) { - this.baseStorageAdapter = adapter; - this.filterOnDataSource = filter; + this.delegate = delegate; + this.filter = filter; } @Override @@ -48,89 +46,32 @@ public class FilteredStorageAdapter implements StorageAdapter final CursorBuildSpec.CursorBuildSpecBuilder buildSpecBuilder = CursorBuildSpec.builder(spec); final Filter newFilter; if (spec.getFilter() == null) { - if (filterOnDataSource != null) { - newFilter = filterOnDataSource.toFilter(); + if (filter != null) { + newFilter = filter.toFilter(); } else { newFilter = null; } } else { - if (filterOnDataSource != null) { - newFilter = new AndFilter(ImmutableList.of(spec.getFilter(), filterOnDataSource.toFilter())); + if (filter != null) { + newFilter = new AndFilter(ImmutableList.of(spec.getFilter(), filter.toFilter())); } else { newFilter = spec.getFilter(); } } buildSpecBuilder.setFilter(newFilter); - return baseStorageAdapter.makeCursorHolder(buildSpecBuilder.build()); - } - - @Override - public Interval getInterval() - { - return baseStorageAdapter.getInterval(); + return delegate.makeCursorHolder(buildSpecBuilder.build()); } @Override public RowSignature getRowSignature() { - return baseStorageAdapter.getRowSignature(); - } - - @Override - public Indexed getAvailableDimensions() - { - return baseStorageAdapter.getAvailableDimensions(); - } - - @Override - public Iterable getAvailableMetrics() - { - return baseStorageAdapter.getAvailableMetrics(); - } - - @Override - public int getDimensionCardinality(String column) - { - return baseStorageAdapter.getDimensionCardinality(column); - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - return null; - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - return null; + return delegate.getRowSignature(); } @Nullable @Override public ColumnCapabilities getColumnCapabilities(String column) { - return baseStorageAdapter.getColumnCapabilities(column); - } - - @Override - public int getNumRows() - { - return 0; - } - - @Nullable - @Override - public Metadata getMetadata() - { - return baseStorageAdapter.getMetadata(); - } - - @Override - public boolean isFromTombstone() - { - return baseStorageAdapter.isFromTombstone(); + return delegate.getColumnCapabilities(column); } } diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredSegment.java b/processing/src/main/java/org/apache/druid/segment/FilteredSegment.java new file mode 100644 index 00000000000..145f41d19e7 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/FilteredSegment.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.segment; + +import org.apache.druid.query.filter.DimFilter; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +public class FilteredSegment extends WrappedSegmentReference +{ + @Nullable + private final DimFilter filter; + + public FilteredSegment( + SegmentReference delegate, + @Nullable DimFilter filter + ) + { + super(delegate); + this.filter = filter; + } + + @Override + public CursorFactory asCursorFactory() + { + return new FilteredCursorFactory(delegate.asCursorFactory(), filter); + } + + @Nullable + @Override + public T as(@Nonnull Class clazz) + { + if (TopNOptimizationInspector.class.equals(clazz)) { + return (T) new SimpleTopNOptimizationInspector(filter == null); + } + return super.as(clazz); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java index 62ecda675ac..735f6d1e159 100644 --- a/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java @@ -20,7 +20,8 @@ package org.apache.druid.segment; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; +import org.apache.druid.segment.incremental.IncrementalIndexPhysicalSegmentInspector; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -58,9 +59,9 @@ public class IncrementalIndexSegment implements Segment } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return new IncrementalIndexStorageAdapter(index); + return new IncrementalIndexCursorFactory(index); } @Nullable @@ -71,6 +72,12 @@ public class IncrementalIndexSegment implements Segment return (T) new IncrementalIndexTimeBoundaryInspector(index); } else if (MaxIngestedEventTimeInspector.class.equals(clazz)) { return (T) new IncrementalIndexMaxIngestedEventTimeInspector(index); + } else if (Metadata.class.equals(clazz)) { + return (T) index.getMetadata(); + } else if (PhysicalSegmentInspector.class.equals(clazz)) { + return (T) new IncrementalIndexPhysicalSegmentInspector(index); + } else if (TopNOptimizationInspector.class.equals(clazz)) { + return (T) new SimpleTopNOptimizationInspector(true); } else { return Segment.super.as(clazz); } diff --git a/processing/src/main/java/org/apache/druid/segment/PhysicalSegmentInspector.java b/processing/src/main/java/org/apache/druid/segment/PhysicalSegmentInspector.java new file mode 100644 index 00000000000..6e72e412463 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/PhysicalSegmentInspector.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.segment; + +import javax.annotation.Nullable; + +/** + * Interface for methods describing physical segments such as {@link QueryableIndexSegment} and + * {@link IncrementalIndexSegment} that is not typically used at query time (outside of metadata queries). + */ +public interface PhysicalSegmentInspector extends ColumnInspector +{ + /** + * Returns {@link Metadata} which contains details about how the segment was created + */ + @Nullable + Metadata getMetadata(); + /** + * Returns the minimum value of the provided column, if known through an index, dictionary, or cache. Returns null + * if not known. Does not scan the column to find the minimum value. + */ + @Nullable + Comparable getMinValue(String column); + + /** + * Returns the minimum value of the provided column, if known through an index, dictionary, or cache. Returns null + * if not known. Does not scan the column to find the maximum value. + */ + @Nullable + Comparable getMaxValue(String column); + + /** + * Returns the number of distinct values in a column, if known, or + * {@link DimensionDictionarySelector#CARDINALITY_UNKNOWN} if not.} + */ + int getDimensionCardinality(String column); + + /** + * Returns the number of rows in the segment + */ + int getNumRows(); +} diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java index 3766db4a4b1..309e36d24b8 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java @@ -34,9 +34,9 @@ import java.util.Map; /** * Direct interface to memory mapped segments. Not a public API for extensions; site specific queries should be - * using {@link StorageAdapter}. + * using {@link CursorFactory}. * - * @see QueryableIndexStorageAdapter for query path adapter + * @see QueryableIndexCursorFactory for query path adapter * @see QueryableIndexIndexableAdapter for indexing path adapter */ public interface QueryableIndex extends Closeable, ColumnInspector @@ -48,7 +48,8 @@ public interface QueryableIndex extends Closeable, ColumnInspector */ Indexed getAvailableDimensions(); BitmapFactory getBitmapFactoryForDimensions(); - @Nullable Metadata getMetadata(); + @Nullable + Metadata getMetadata(); /** * Map of column name to {@link DimensionHandler}, whose contents and iteration order matches diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java new file mode 100644 index 00000000000..45892326711 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java @@ -0,0 +1,79 @@ +/* + * 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.segment; + +import org.apache.druid.query.OrderBy; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; + +import javax.annotation.Nullable; +import java.util.LinkedHashSet; + +public class QueryableIndexCursorFactory implements CursorFactory +{ + private final QueryableIndex index; + + public QueryableIndexCursorFactory(QueryableIndex index) + { + this.index = index; + } + + @Override + public CursorHolder makeCursorHolder(CursorBuildSpec spec) + { + return new QueryableIndexCursorHolder(index, CursorBuildSpec.builder(spec).build()); + } + + @Override + public RowSignature getRowSignature() + { + final LinkedHashSet columns = new LinkedHashSet<>(); + + for (final OrderBy orderBy : index.getOrdering()) { + columns.add(orderBy.getColumnName()); + } + + // Add __time after the defined ordering, if __time wasn't part of it. + columns.add(ColumnHolder.TIME_COLUMN_NAME); + columns.addAll(index.getColumnNames()); + + final RowSignature.Builder builder = RowSignature.builder(); + for (final String column : columns) { + final ColumnType columnType = ColumnType.fromCapabilities(index.getColumnCapabilities(column)); + + // index.getOrdering() may include columns that don't exist, such as if they were omitted due to + // being 100% nulls. Don't add those to the row signature. + if (columnType != null) { + builder.add(column, columnType); + } + } + + return builder.build(); + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + return index.getColumnCapabilities(column); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java index 5188b385b36..5aa4dbed8cc 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java @@ -34,6 +34,7 @@ import org.apache.druid.query.Order; import org.apache.druid.query.OrderBy; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContext; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.filter.Filter; @@ -112,6 +113,7 @@ public class QueryableIndexCursorHolder implements CursorHolder Cursors.getTimeOrdering(ordering), interval, filter, + cursorBuildSpec.getQueryContext().getBoolean(QueryContexts.CURSOR_AUTO_ARRANGE_FILTERS, false), metrics ) ); @@ -346,7 +348,6 @@ public class QueryableIndexCursorHolder implements CursorHolder * @param timestamp the timestamp to search for * @param startIndex first index to search, inclusive * @param endIndex last index to search, exclusive - * * @return first index that has a timestamp equal to, or greater, than "timestamp" */ @VisibleForTesting @@ -665,6 +666,7 @@ public class QueryableIndexCursorHolder implements CursorHolder Order timeOrder, Interval interval, @Nullable Filter filter, + boolean cursorAutoArrangeFilters, @Nullable QueryMetrics> metrics ) { @@ -688,6 +690,7 @@ public class QueryableIndexCursorHolder implements CursorHolder interval, filter ), + cursorAutoArrangeFilters, bitmapIndexSelector, numRows, metrics @@ -708,13 +711,14 @@ public class QueryableIndexCursorHolder implements CursorHolder /** * Create a {@link FilterBundle} for a cursor hold instance. - * + *

    * The provided filter must include the query-level interface if needed. To compute this properly, use * {@link #computeFilterWithIntervalIfNeeded}. */ @Nullable private static FilterBundle makeFilterBundle( @Nullable final Filter filter, + boolean cursorAutoArrangeFilters, final ColumnSelectorColumnIndexSelector bitmapIndexSelector, final int numRows, @Nullable final QueryMetrics metrics @@ -732,8 +736,11 @@ public class QueryableIndexCursorHolder implements CursorHolder return null; } final long bitmapConstructionStartNs = System.nanoTime(); - final FilterBundle filterBundle = filter.makeFilterBundle( + final FilterBundle filterBundle = new FilterBundle.Builder( + filter, bitmapIndexSelector, + cursorAutoArrangeFilters + ).build( bitmapResultFactory, numRows, numRows, diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java index 0060b276c4b..d9913036b5a 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java @@ -265,10 +265,10 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter /** * On {@link #moveToNext()} and {@link #mark()}, this class copies all column values into a set of {@link * SettableColumnValueSelector} instances. Alternative approach was to save only offset in column and use the same - * column value selectors as in {@link QueryableIndexStorageAdapter}. The approach with "caching" in {@link + * column value selectors as in {@link QueryableIndexCursorFactory}. The approach with "caching" in {@link * SettableColumnValueSelector}s is chosen for two reasons: * 1) Avoid re-reading column values from serialized format multiple times (because they are accessed multiple times) - * For comparison, it's not a factor for {@link QueryableIndexStorageAdapter} because during query processing, + * For comparison, it's not a factor for {@link QueryableIndexCursorFactory} because during query processing, * column values are usually accessed just once per offset, if aggregator or query runner are written sanely. * Avoiding re-reads is especially important for object columns, because object deserialization is potentially * expensive. diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexPhysicalSegmentInspector.java similarity index 55% rename from processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java rename to processing/src/main/java/org/apache/druid/segment/QueryableIndexPhysicalSegmentInspector.java index a0e23e71f74..44ab4343a6a 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexPhysicalSegmentInspector.java @@ -19,121 +19,30 @@ package org.apache.druid.segment; -import org.apache.druid.query.OrderBy; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; -import org.joda.time.DateTime; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.io.IOException; import java.io.UncheckedIOException; -import java.util.LinkedHashSet; -import java.util.Set; -/** - * - */ -public class QueryableIndexStorageAdapter implements StorageAdapter +public class QueryableIndexPhysicalSegmentInspector implements PhysicalSegmentInspector { - public static final int DEFAULT_VECTOR_SIZE = 512; - private final QueryableIndex index; - public QueryableIndexStorageAdapter(QueryableIndex index) + public QueryableIndexPhysicalSegmentInspector(QueryableIndex index) { this.index = index; } @Override - public Interval getInterval() + public Metadata getMetadata() { - return index.getDataInterval(); - } - - @Override - public RowSignature getRowSignature() - { - final LinkedHashSet columns = new LinkedHashSet<>(); - - for (final OrderBy orderBy : index.getOrdering()) { - columns.add(orderBy.getColumnName()); - } - - // Add __time after the defined ordering, if __time wasn't part of it. - columns.add(ColumnHolder.TIME_COLUMN_NAME); - - for (final String dimension : getAvailableDimensions()) { - columns.add(dimension); - } - - for (final String metric : getAvailableMetrics()) { - columns.add(metric); - } - - final RowSignature.Builder builder = RowSignature.builder(); - for (final String column : columns) { - final ColumnType columnType = ColumnType.fromCapabilities(index.getColumnCapabilities(column)); - - // index.getOrdering() may include columns that don't exist, such as if they were omitted due to - // being 100% nulls. Don't add those to the row signature. - if (columnType != null) { - builder.add(column, columnType); - } - } - - return builder.build(); - } - - @Override - public Indexed getAvailableDimensions() - { - return index.getAvailableDimensions(); - } - - @Override - public Iterable getAvailableMetrics() - { - // Use LinkedHashSet to preserve the original order. - final Set columnNames = new LinkedHashSet<>(index.getColumnNames()); - - for (final String dimension : index.getAvailableDimensions()) { - columnNames.remove(dimension); - } - - return columnNames; - } - - @Override - public int getDimensionCardinality(String dimension) - { - ColumnHolder columnHolder = index.getColumnHolder(dimension); - if (columnHolder == null) { - // NullDimensionSelector has cardinality = 1 (one null, nothing else). - return 1; - } - try (BaseColumn col = columnHolder.getColumn()) { - if (!(col instanceof DictionaryEncodedColumn)) { - return DimensionDictionarySelector.CARDINALITY_UNKNOWN; - } - return ((DictionaryEncodedColumn) col).getCardinality(); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public int getNumRows() - { - return index.getNumRows(); + return index.getMetadata(); } @Override @@ -163,31 +72,34 @@ public class QueryableIndexStorageAdapter implements StorageAdapter } @Override + public int getDimensionCardinality(String column) + { + ColumnHolder columnHolder = index.getColumnHolder(column); + if (columnHolder == null) { + // NullDimensionSelector has cardinality = 1 (one null, nothing else). + return 1; + } + try (BaseColumn col = columnHolder.getColumn()) { + if (!(col instanceof DictionaryEncodedColumn)) { + return DimensionDictionarySelector.CARDINALITY_UNKNOWN; + } + return ((DictionaryEncodedColumn) col).getCardinality(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + @Nullable + @Override public ColumnCapabilities getColumnCapabilities(String column) { return index.getColumnCapabilities(column); } @Override - public DateTime getMaxIngestedEventTime() + public int getNumRows() { - // For immutable indexes, maxIngestedEventTime is maxTime. - return getMaxTime(); - } - - @Override - public CursorHolder makeCursorHolder(CursorBuildSpec spec) - { - return new QueryableIndexCursorHolder( - index, - CursorBuildSpec.builder(spec).build() - ); - } - - @Override - public Metadata getMetadata() - { - return index.getMetadata(); + return index.getNumRows(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java index ccbe0475c33..ece082a0b33 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java @@ -39,14 +39,14 @@ public class QueryableIndexSegment implements Segment .makeAsMap(QueryableIndexSegment.class); private final QueryableIndex index; - private final QueryableIndexStorageAdapter storageAdapter; + private final QueryableIndexCursorFactory cursorFactory; private final TimeBoundaryInspector timeBoundaryInspector; private final SegmentId segmentId; public QueryableIndexSegment(QueryableIndex index, final SegmentId segmentId) { this.index = index; - this.storageAdapter = new QueryableIndexStorageAdapter(index); + this.cursorFactory = new QueryableIndexCursorFactory(index); this.timeBoundaryInspector = QueryableIndexTimeBoundaryInspector.create(index); this.segmentId = segmentId; } @@ -70,9 +70,9 @@ public class QueryableIndexSegment implements Segment } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return storageAdapter; + return cursorFactory; } @Override @@ -97,6 +97,12 @@ public class QueryableIndexSegment implements Segment if (TimeBoundaryInspector.class.equals(clazz)) { return (T) timeBoundaryInspector; + } else if (Metadata.class.equals(clazz)) { + return (T) index.getMetadata(); + } else if (PhysicalSegmentInspector.class.equals(clazz)) { + return (T) new QueryableIndexPhysicalSegmentInspector(index); + } else if (TopNOptimizationInspector.class.equals(clazz)) { + return (T) new SimpleTopNOptimizationInspector(true); } return Segment.super.as(clazz); diff --git a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java index b6ce3860b2f..e2969f10d00 100644 --- a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java @@ -112,11 +112,11 @@ public class ReferenceCountingSegment extends ReferenceCountingCloseableObject implements StorageAdapter +public class RowBasedCursorFactory implements CursorFactory { private final Sequence rowSequence; private final RowAdapter rowAdapter; private final RowSignature rowSignature; - public RowBasedStorageAdapter( - final Sequence rowSequence, - final RowAdapter rowAdapter, - final RowSignature rowSignature + public RowBasedCursorFactory( + Sequence rowSequence, + RowAdapter rowAdapter, + RowSignature rowSignature ) { - this.rowSequence = Preconditions.checkNotNull(rowSequence, "rowSequence"); - this.rowAdapter = Preconditions.checkNotNull(rowAdapter, "rowAdapter"); - this.rowSignature = Preconditions.checkNotNull(rowSignature, "rowSignature"); - } - - /** - * Whether the provided time interval and granularity combination is allowed. - * - * We restrict ETERNITY with non-ALL granularity, because allowing it would involve creating a very high number - * of time grains. This would cause queries to take an excessive amount of time or run out of memory. - */ - public static boolean isQueryGranularityAllowed(final Interval interval, final Granularity granularity) - { - return Granularities.ALL.equals(granularity) || !Intervals.ETERNITY.equals(interval); - } - - @Override - public Interval getInterval() - { - return Intervals.ETERNITY; - } - - @Override - public Indexed getAvailableDimensions() - { - return new ListIndexed<>(new ArrayList<>(rowSignature.getColumnNames())); - } - - @Override - public Iterable getAvailableMetrics() - { - return Collections.emptyList(); - } - - @Override - public RowSignature getRowSignature() - { - return rowSignature; - } - - @Override - public int getDimensionCardinality(String column) - { - return DimensionDictionarySelector.CARDINALITY_UNKNOWN; - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - return null; - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - return null; - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - return RowBasedColumnSelectorFactory.getColumnCapabilities(rowSignature, column); - } - - @Override - public int getNumRows() - { - if (rowSequence instanceof SimpleSequence) { - final Iterable rowIterable = ((SimpleSequence) rowSequence).getIterable(); - - if (rowIterable instanceof Collection) { - return ((Collection) rowIterable).size(); - } - } - - // getNumRows is only used by tests and by segmentMetadataQuery (which would be odd to call on inline datasources) - // so no big deal if it doesn't always work. - throw new UnsupportedOperationException("Cannot retrieve number of rows"); - } - - @Override - public Metadata getMetadata() - { - throw new UnsupportedOperationException("Cannot retrieve metadata"); + this.rowSequence = rowSequence; + this.rowAdapter = rowAdapter; + this.rowSignature = rowSignature; } @Override @@ -199,6 +100,19 @@ public class RowBasedStorageAdapter implements StorageAdapter }; } + @Override + public RowSignature getRowSignature() + { + return rowSignature; + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + return RowBasedColumnSelectorFactory.getColumnCapabilities(rowSignature, column); + } + /** * Reverse a Sequence. * diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java b/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java index 26ff6594bbb..4ba2c98987d 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java @@ -20,6 +20,7 @@ package org.apache.druid.segment; import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.timeline.SegmentId; @@ -34,18 +35,20 @@ import javax.annotation.Nullable; public class RowBasedSegment implements Segment { private final SegmentId segmentId; - private final StorageAdapter storageAdapter; + private final Sequence rowSequence; + private final RowAdapter rowAdapter; + private final RowSignature rowSignature; /** * Create a row-based segment. * * The provided "rowIterable" must be in time-order according to the provided {@link RowAdapter#timestampFunction()}. - * The cursor returned by {@link RowBasedStorageAdapter#makeCursorHolder(CursorBuildSpec)} makes no attempt to verify + * The cursor returned by {@link RowBasedCursorFactory#makeCursorHolder(CursorBuildSpec)} makes no attempt to verify * this, and callers will expect it. * * The provided "rowSignature" will be used for reporting available columns and their capabilities to users of - * {@link #asStorageAdapter()}. Note that the {@link ColumnSelectorFactory} implementation returned by this segment's - * storage adapter will allow creation of selectors on any field, using the {@link RowAdapter#columnFunction} for that + * {@link #asCursorFactory()}. Note that the {@link ColumnSelectorFactory} implementation returned by this segment's + * cursor factory will allow creation of selectors on any field, using the {@link RowAdapter#columnFunction} for that * field, even if it doesn't appear in "rowSignature". * * @param segmentId segment identifier; will be returned by {@link #getId()} @@ -62,11 +65,9 @@ public class RowBasedSegment implements Segment ) { this.segmentId = Preconditions.checkNotNull(segmentId, "segmentId"); - this.storageAdapter = new RowBasedStorageAdapter<>( - rowSequence, - rowAdapter, - rowSignature - ); + this.rowSignature = Preconditions.checkNotNull(rowSignature, "rowSignature"); + this.rowSequence = Preconditions.checkNotNull(rowSequence, "rowSequence"); + this.rowAdapter = Preconditions.checkNotNull(rowAdapter, "rowAdapter"); } @Override @@ -80,7 +81,7 @@ public class RowBasedSegment implements Segment @Nonnull public Interval getDataInterval() { - return storageAdapter.getInterval(); + return Intervals.ETERNITY; } @Nullable @@ -91,10 +92,9 @@ public class RowBasedSegment implements Segment } @Override - @Nonnull - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return storageAdapter; + return new RowBasedCursorFactory<>(rowSequence, rowAdapter, rowSignature); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/RowWalker.java b/processing/src/main/java/org/apache/druid/segment/RowWalker.java index f55245b3bca..a43d1e5f51e 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowWalker.java +++ b/processing/src/main/java/org/apache/druid/segment/RowWalker.java @@ -31,7 +31,7 @@ import java.io.IOException; import java.util.function.ToLongFunction; /** - * Used by {@link RowBasedStorageAdapter} and {@link RowBasedCursor} to walk through rows. It allows multiple + * Used by {@link RowBasedCursorFactory} and {@link RowBasedCursor} to walk through rows. It allows multiple * {@link RowBasedCursor} to share the same underlying Iterable. * * The class creates a yielder from the sequence to iterate over the rows. However, it doesn't call the sequence's close diff --git a/processing/src/main/java/org/apache/druid/segment/Segment.java b/processing/src/main/java/org/apache/druid/segment/Segment.java index 44be82a56ee..135f4d55642 100644 --- a/processing/src/main/java/org/apache/druid/segment/Segment.java +++ b/processing/src/main/java/org/apache/druid/segment/Segment.java @@ -19,6 +19,7 @@ package org.apache.druid.segment; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.query.datasourcemetadata.DataSourceMetadataResultValue; import org.apache.druid.segment.join.table.IndexedTable; @@ -44,18 +45,26 @@ public interface Segment extends Closeable @Nullable QueryableIndex asQueryableIndex(); - StorageAdapter asStorageAdapter(); + @Deprecated + default StorageAdapter asStorageAdapter() + { + throw DruidException.defensive( + "asStorageAdapter is no longer supported, use Segment.asCursorFactory to build cursors, or Segment.as(..) to get various metadata information instead" + ); + } + + CursorFactory asCursorFactory(); /** * Request an implementation of a particular interface. * - * If the passed-in interface is {@link QueryableIndex} or {@link StorageAdapter}, then this method behaves - * identically to {@link #asQueryableIndex()} or {@link #asStorageAdapter()}. Other interfaces are only + * If the passed-in interface is {@link QueryableIndex} or {@link CursorFactory}, then this method behaves + * identically to {@link #asQueryableIndex()} or {@link #asCursorFactory()}. Other interfaces are only * expected to be requested by callers that have specific knowledge of extra features provided by specific * segment types. For example, an extension might provide a custom Segment type that can offer both * StorageAdapter and some new interface. That extension can also offer a Query that uses that new interface. * - * Implementations which accept classes other than {@link QueryableIndex} or {@link StorageAdapter} are limited + * Implementations which accept classes other than {@link QueryableIndex} or {@link CursorFactory} are limited * to using those classes within the extension. This means that one extension cannot rely on the `Segment.as` * behavior of another extension. * @@ -64,18 +73,23 @@ public interface Segment extends Closeable * * @return instance of clazz, or null if the interface is not supported by this segment * - * @see StorageAdapter storage adapter for queries. Never null. + * @see CursorFactory to make cursors to run queries. Never null. * @see QueryableIndex index object, if this is a memory-mapped regular segment. * @see IndexedTable table object, if this is a joinable indexed table. * @see TimeBoundaryInspector inspector for min/max timestamps, if supported by this segment. + * @see PhysicalSegmentInspector inspector for physical segment details, if supported by this segment. * @see MaxIngestedEventTimeInspector inspector for {@link DataSourceMetadataResultValue#getMaxIngestedEventTime()} + * @see TopNOptimizationInspector inspector containing information for topN specific optimizations * @see CloseableShapeshifter stepping stone to {@link org.apache.druid.query.rowsandcols.RowsAndColumns}. + * */ @SuppressWarnings({"unused", "unchecked"}) @Nullable default T as(@Nonnull Class clazz) { - if (clazz.equals(QueryableIndex.class)) { + if (clazz.equals(CursorFactory.class)) { + return (T) asCursorFactory(); + } else if (clazz.equals(QueryableIndex.class)) { return (T) asQueryableIndex(); } else if (clazz.equals(StorageAdapter.class)) { return (T) asStorageAdapter(); @@ -83,6 +97,12 @@ public interface Segment extends Closeable return null; } + default boolean isTombstone() + { + return false; + } + + default String asString() { return getClass().toString(); diff --git a/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java b/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java index 4e39ca00f18..a025113f773 100644 --- a/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java +++ b/processing/src/main/java/org/apache/druid/segment/SegmentWrangler.java @@ -40,7 +40,7 @@ public interface SegmentWrangler * @return Segments that, collectively, contain data for dataSource. May be empty if dataSource does not exist or * has no data in the provided intervals. May contain data outside the provided intervals, so callers should * filter it down further, e.g. through the "interval" parameter of {@link CursorBuildSpec} for - * {@link StorageAdapter#makeCursorHolder(CursorBuildSpec)} + * {@link CursorFactory#makeCursorHolder(CursorBuildSpec)} */ Iterable getSegmentsForIntervals(DataSource dataSource, Iterable intervals); } diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java index 507737a2c5b..3f6c0f5e825 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java @@ -33,6 +33,7 @@ import org.apache.druid.segment.data.Indexed; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -61,11 +62,16 @@ public abstract class SimpleQueryableIndex implements QueryableIndex Preconditions.checkNotNull(columns.get(ColumnHolder.TIME_COLUMN_NAME)); this.dataInterval = Preconditions.checkNotNull(dataInterval, "dataInterval"); ImmutableList.Builder columnNamesBuilder = ImmutableList.builder(); - for (String column : columns.keySet()) { - if (!ColumnHolder.TIME_COLUMN_NAME.equals(column)) { - columnNamesBuilder.add(column); + LinkedHashSet dimsFirst = new LinkedHashSet<>(); + for (String dimName : dimNames) { + dimsFirst.add(dimName); + } + for (String columnName : columns.keySet()) { + if (!ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) { + dimsFirst.add(columnName); } } + columnNamesBuilder.addAll(dimsFirst); this.columnNames = columnNamesBuilder.build(); this.availableDimensions = dimNames; this.bitmapFactory = bitmapFactory; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/WireTransferable.java b/processing/src/main/java/org/apache/druid/segment/SimpleTopNOptimizationInspector.java similarity index 63% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/WireTransferable.java rename to processing/src/main/java/org/apache/druid/segment/SimpleTopNOptimizationInspector.java index a7d55f59929..184f514b695 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/WireTransferable.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleTopNOptimizationInspector.java @@ -17,21 +17,20 @@ * under the License. */ -package org.apache.druid.query.rowsandcols.semantic; +package org.apache.druid.segment; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.query.rowsandcols.RowsAndColumns; - -public interface WireTransferable +public class SimpleTopNOptimizationInspector implements TopNOptimizationInspector { - static WireTransferable fromRAC(RowsAndColumns rac) + private final boolean allDictionaryIdsPresent; + + public SimpleTopNOptimizationInspector(boolean allDictionaryIdsPresent) { - WireTransferable retVal = rac.as(WireTransferable.class); - if (retVal == null) { - throw new ISE("Rac[%s] cannot be transferred over the wire", rac.getClass()); - } - return retVal; + this.allDictionaryIdsPresent = allDictionaryIdsPresent; } - byte[] bytesToTransfer(); + @Override + public boolean areAllDictionaryIdsPresent() + { + return allDictionaryIdsPresent; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java index c0949692e4f..5fa7286b7ae 100644 --- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java @@ -19,14 +19,13 @@ package org.apache.druid.segment; -import com.google.common.collect.Iterables; import org.apache.druid.error.DruidException; import org.apache.druid.guice.annotations.PublicApi; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.OrderBy; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.QueryMetrics; +import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.vector.VectorCursor; @@ -34,185 +33,205 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; -import java.util.List; /** * */ +@Deprecated @PublicApi -public interface StorageAdapter extends CursorFactory, ColumnInspector, CursorHolderFactory +public interface StorageAdapter extends ColumnInspector { - /** - * Build a {@link CursorHolder} which can provide {@link Cursor} and {@link VectorCursor} (if capable) which allows - * scanning segments and creating {@link ColumnSelectorFactory} and - * {@link org.apache.druid.segment.vector.VectorColumnSelectorFactory} respectively to read row values at the cursor - * position. + * @deprecated Use {@link Segment#asCursorFactory()} and then {@link CursorFactory#makeCursorHolder(CursorBuildSpec)} + * and call {@link CursorHolder#canVectorize()} instead. */ - @Override - default CursorHolder makeCursorHolder(CursorBuildSpec spec) + @Deprecated + default boolean canVectorize( + @Nullable Filter filter, + VirtualColumns virtualColumns, + boolean descending + ) { - // For backwards compatibility, the default implementation assumes the underlying rows are sorted by __time. - // Built-in implementations of StorageAdapter must override this method. - final List ordering; - final boolean descending; - if (Cursors.preferDescendingTimeOrdering(spec)) { - ordering = Cursors.descendingTimeOrder(); - descending = true; - } else { - ordering = Cursors.ascendingTimeOrder(); - descending = false; - } - return new CursorHolder() - { - @Override - public boolean canVectorize() - { - return StorageAdapter.this.canVectorize( - spec.getFilter(), - spec.getVirtualColumns(), - descending - ); - } - - @Override - public Cursor asCursor() - { - return Iterables.getOnlyElement( - StorageAdapter.this.makeCursors( - spec.getFilter(), - spec.getInterval(), - spec.getVirtualColumns(), - Granularities.ALL, - descending, - spec.getQueryMetrics() - ).toList() - ); - } - - @Override - public VectorCursor asVectorCursor() - { - return StorageAdapter.this.makeVectorCursor( - spec.getFilter(), - spec.getInterval(), - spec.getVirtualColumns(), - descending, - spec.getQueryContext().getVectorSize(), - spec.getQueryMetrics() - ); - } - - @Nullable - @Override - public List getOrdering() - { - return ordering; - } - - @Override - public void close() - { - // consuming sequences of CursorFactory are expected to close themselves. - } - }; + throw DruidException.defensive( + "canVectorize is no longer supported, use Segment.asCursorFactory().makeCursorHolder(..).canVectorize() instead" + ); } - Interval getInterval(); + /** + * @deprecated Use {@link Segment#asCursorFactory()} and then {@link CursorFactory#makeCursorHolder(CursorBuildSpec)} + * and call {@link CursorHolder#asCursor()} instead. + */ + @Deprecated + default Sequence makeCursors( + @Nullable Filter filter, + Interval interval, + VirtualColumns virtualColumns, + Granularity gran, + boolean descending, + @Nullable QueryMetrics queryMetrics + ) + { + throw DruidException.defensive( + "makeCursors is no longer supported, use Segment.asCursorFactory().makeCursorHolder(..).asCursor() instead" + ); + } /** - * Returns the names of all dimension columns, not including {@link ColumnHolder#TIME_COLUMN_NAME}. + * @deprecated Use {@link Segment#asCursorFactory()} and then {@link CursorFactory#makeCursorHolder(CursorBuildSpec)} + * and call {@link CursorHolder#asVectorCursor()} instead. */ - Indexed getAvailableDimensions(); + @Deprecated + @Nullable + default VectorCursor makeVectorCursor( + @Nullable Filter filter, + Interval interval, + VirtualColumns virtualColumns, + boolean descending, + int vectorSize, + @Nullable QueryMetrics queryMetrics + ) + { + throw DruidException.defensive( + "makeVectorCursor is no longer supported, use Segment.asCursorFactory().makeCursorHolder(..).asVectorCursor() instead" + ); + } /** - * Returns the names of all metric columns. + * @deprecated Callers should use {@link Segment#getDataInterval()} instead. */ - Iterable getAvailableMetrics(); + @Deprecated + default Interval getInterval() + { + throw DruidException.defensive( + "getInterval is no longer supported, use Segment.getDataInterval() instead." + ); + } /** - * Returns the row signature of the data available from this adapter. For mutable adapters, even though the signature - * may evolve over time, any particular object returned by this method is an immutable snapshot. + * @deprecated Callers should use {@link Segment#as(Class)} to construct a {@link Metadata} instead. */ + @Deprecated + default Indexed getAvailableDimensions() + { + throw DruidException.defensive( + "getAvailableDimensions is no longer supported, use Segment.getRowSignature() and or Segment.as(PhysicalSegmentInspector.class) instead" + ); + } + + /** + * @deprecated Callers should use {@link Segment#as(Class)} to construct a {@link Metadata} if available and check + * {@link Metadata#getAggregators()} instead. + */ + @Deprecated + default Iterable getAvailableMetrics() + { + throw DruidException.defensive( + "getAvailableMetrics is no longer supported, use Segment.as(PhysicalSegmentInspector.class) instead" + ); + } + + /** + * @deprecated use {@link Segment#asCursorFactory()} and {@link CursorFactory#getRowSignature()} instead. + */ + @Deprecated default RowSignature getRowSignature() { - final RowSignature.Builder builder = RowSignature.builder(); - builder.addTimeColumn(); - - for (final String column : Iterables.concat(getAvailableDimensions(), getAvailableMetrics())) { - builder.add(column, ColumnType.fromCapabilities(getColumnCapabilities(column))); - } - - return builder.build(); + throw DruidException.defensive( + "getRowSignature is no longer supported, use Segment.asCursorFactory().getRowSignature() instead" + ); } /** - * Returns the number of distinct values for a column, or {@link DimensionDictionarySelector#CARDINALITY_UNKNOWN} - * if unknown. - * - * If the column doesn't exist, returns 1, because a column that doesn't exist is treated as a column of default - * (or null) values. + * @deprecated Callers should use {@link Segment#as(Class)} to construct a {@link PhysicalSegmentInspector} if + * available and call {@link PhysicalSegmentInspector#getDimensionCardinality(String)} instead. */ - int getDimensionCardinality(String column); + @Deprecated + default int getDimensionCardinality(String column) + { + throw DruidException.defensive( + "getDimensionCardinality is no longer supported, use Segment.as(SegmentAnalysisInspector.class) instead" + ); + } /** - * Use {@link TimeBoundaryInspector#getMinTime()} instead. + * @deprecated Use {@link Segment#as(Class)} to get a {@link TimeBoundaryInspector} if available and call + * {@link TimeBoundaryInspector#getMinTime()} instead. */ @Deprecated default DateTime getMinTime() { throw DruidException.defensive( - "getMinTime is no longer supported, use Segment.as(MinMaxValueInspector.class) instead" + "getMinTime is no longer supported, use Segment.as(TimeBoundaryInspector.class) instead" ); } /** - * Use {@link TimeBoundaryInspector#getMaxTime()} instead. + * @deprecated Use {@link Segment#as(Class)} to get a {@link TimeBoundaryInspector} if available and call + * {@link TimeBoundaryInspector#getMaxTime()} instead. */ @Deprecated default DateTime getMaxTime() { throw DruidException.defensive( - "getMaxTime is no longer supported, use Segment.as(MinMaxValueInspector.class) instead" + "getMaxTime is no longer supported, use Segment.as(TimeBoundaryInspector.class) instead" ); } /** - * Returns the minimum value of the provided column, if known through an index, dictionary, or cache. Returns null - * if not known. Does not scan the column to find the minimum value. + * @deprecated Use {@link Segment#as(Class)} to get a {@link PhysicalSegmentInspector} if available and call + * {@link PhysicalSegmentInspector#getMinValue(String)} */ + @Deprecated @Nullable - Comparable getMinValue(String column); + default Comparable getMinValue(String column) + { + throw DruidException.defensive( + "getMinValue is no longer supported, use Segment.as(SegmentAnalysisInspector.class) instead" + ); + } /** - * Returns the minimum value of the provided column, if known through an index, dictionary, or cache. Returns null - * if not known. Does not scan the column to find the maximum value. + * @deprecated Use {@link Segment#as(Class)} to get a {@link PhysicalSegmentInspector} if available and call + * {@link PhysicalSegmentInspector#getMaxValue(String)} */ + @Deprecated @Nullable - Comparable getMaxValue(String column); + default Comparable getMaxValue(String column) + { + throw DruidException.defensive( + "getMaxValue is no longer supported, use Segment.as(SegmentAnalysisInspector.class) instead" + ); + } /** - * Returns capabilities of a particular column, if known. May be null if the column doesn't exist, or if - * the column does exist but the capabilities are unknown. The latter is possible with dynamically discovered - * columns. - * - * Note that StorageAdapters are representations of "real" segments, so they are not aware of any virtual columns - * that may be involved in a query. In general, query engines should instead use the method - * {@link ColumnSelectorFactory#getColumnCapabilities(String)}, which returns capabilities for virtual columns as - * well. - * - * @param column column name - * - * @return capabilities, or null + * @deprecated Use {@link Segment#asCursorFactory()} and then {@link CursorFactory#getColumnCapabilities(String)} + * instead. */ + @Deprecated @Override @Nullable - ColumnCapabilities getColumnCapabilities(String column); - - int getNumRows(); + default ColumnCapabilities getColumnCapabilities(String column) + { + throw DruidException.defensive( + "getColumnCapabilities is no longer supported, use Segment.asCursorFactory().getColumnCapabilities(..) instead" + ); + } /** - * Use {@link MaxIngestedEventTimeInspector#getMaxIngestedEventTime()} instead. + * @deprecated Use {@link Segment#as(Class)} to get a {@link PhysicalSegmentInspector} if available then call + * {@link PhysicalSegmentInspector#getNumRows()} instead. + */ + @Deprecated + default int getNumRows() + { + throw DruidException.defensive( + "getNumRows is no longer supported, use Segment.as(PhysicalSegmentInspector.class) instead" + ); + } + + /** + * @deprecated Use {@link Segment#as(Class)} to get a {@link MaxIngestedEventTimeInspector} if available and call + * {@link MaxIngestedEventTimeInspector#getMaxIngestedEventTime()} instead. */ @Deprecated default DateTime getMaxIngestedEventTime() @@ -222,27 +241,38 @@ public interface StorageAdapter extends CursorFactory, ColumnInspector, CursorHo ); } + /** + * @deprecated Use {@link Segment#as(Class)} to fetch a {@link Metadata} if available + */ + @Deprecated @Nullable - Metadata getMetadata(); + default Metadata getMetadata() + { + throw DruidException.defensive( + "getMetadata is no longer supported, use Segment.as(PhysicalSegmentInspector.class) instead" + ); + } /** - * Returns true if this storage adapter can filter some rows out. The actual column cardinality can be lower than - * what {@link #getDimensionCardinality} returns if this returns true. Dimension selectors for such storage adapter - * can return non-contiguous dictionary IDs because the dictionary IDs in filtered rows will not be returned. - * Note that the number of rows accessible via this storage adapter will not necessarily decrease because of - * the built-in filters. For inner joins, for example, the number of joined rows can be larger than - * the number of rows in the base adapter even though this method returns true. + * @deprecated Use {@link Segment#as(Class)} to get a {@link TopNOptimizationInspector} if available and call + * {@link TopNOptimizationInspector#areAllDictionaryIdsPresent()} instead. */ + @Deprecated default boolean hasBuiltInFilters() { - return false; + throw DruidException.defensive( + "hasBuiltInFilters is no longer supported, use Segment.as(FilteredSegmentInspector.class) instead" + ); } /** - * @return true if this index was created from a tombstone or false otherwise + * @deprecated Use {@link Segment#isTombstone()} */ + @Deprecated default boolean isFromTombstone() { - return false; + throw DruidException.defensive( + "isFromTombstone is no longer supported, use Segment.isTombstone instead" + ); } } diff --git a/processing/src/main/java/org/apache/druid/segment/TopNOptimizationInspector.java b/processing/src/main/java/org/apache/druid/segment/TopNOptimizationInspector.java new file mode 100644 index 00000000000..a5a637ad70d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/TopNOptimizationInspector.java @@ -0,0 +1,37 @@ +/* + * 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.segment; + +/** + * oddly specific {@link Segment} inspector for topN optimizations + */ +public interface TopNOptimizationInspector +{ + /** + * Returns true if all values in the dictionary are present in rows. Returns false such as this segment can filter + * some rows out even if the query does not specify a filter, making the actual column cardinality be lower than + * what {@link DimensionDictionarySelector#getValueCardinality()} returns. Dimension selectors for such cursors can + * return non-contiguous dictionary IDs because the dictionary IDs in filtered rows will not be returned. Note that + * the number of rows accessible via cursors created on these segments will not necessarily decrease because of the + * built-in filters. For inner joins, for example, the number of joined rows can be larger than the number of rows in + * the base segment even though this method returns false. + */ + boolean areAllDictionaryIdsPresent(); +} diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java index a2a920cba1d..57aa5683c8d 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java @@ -203,7 +203,7 @@ public class UnnestColumnValueSelectorCursor implements Cursor public ColumnCapabilities getColumnCapabilities(String column) { if (outputName.equals(column)) { - return UnnestStorageAdapter.computeOutputColumnCapabilities(baseColumnSelectorFactory, unnestColumn); + return UnnestCursorFactory.computeOutputColumnCapabilities(baseColumnSelectorFactory, unnestColumn); } return baseColumnSelectorFactory.getColumnCapabilities(column); diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestCursorFactory.java similarity index 83% rename from processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java rename to processing/src/main/java/org/apache/druid/segment/UnnestCursorFactory.java index 2f9552a1d3c..3337f3b4336 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestCursorFactory.java @@ -22,7 +22,6 @@ package org.apache.druid.segment; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.OrderBy; @@ -39,8 +38,6 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.TypeSignature; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.data.Indexed; -import org.apache.druid.segment.data.ListIndexed; import org.apache.druid.segment.filter.AndFilter; import org.apache.druid.segment.filter.BoundFilter; import org.apache.druid.segment.filter.Filters; @@ -51,45 +48,30 @@ import org.apache.druid.segment.filter.SelectorFilter; import org.apache.druid.segment.join.PostJoinCursor; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.utils.CloseableUtils; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collections; -import java.util.LinkedHashSet; import java.util.List; import java.util.Set; import java.util.function.Supplier; -/** - * This class serves as the Storage Adapter for the Unnest Segment and is responsible for creating the cursors - * If the column is dictionary encoded it creates {@link UnnestDimensionCursor} else {@link UnnestColumnValueSelectorCursor} - * These cursors help navigate the segments for these cases - */ -public class UnnestStorageAdapter implements StorageAdapter +public class UnnestCursorFactory implements CursorFactory { - public StorageAdapter getBaseAdapter() - { - return baseAdapter; - } - - private final StorageAdapter baseAdapter; + private final CursorFactory baseCursorFactory; private final VirtualColumn unnestColumn; - private final String outputColumnName; - @Nullable - private final DimFilter unnestFilter; + private final DimFilter filter; - public UnnestStorageAdapter( - final StorageAdapter baseAdapter, - final VirtualColumn unnestColumn, - @Nullable final DimFilter unnestFilter + public UnnestCursorFactory( + CursorFactory baseCursorFactory, + VirtualColumn unnestColumn, + @Nullable DimFilter filter ) { - this.baseAdapter = baseAdapter; + this.baseCursorFactory = baseCursorFactory; this.unnestColumn = unnestColumn; - this.outputColumnName = unnestColumn.getOutputName(); - this.unnestFilter = unnestFilter; + this.filter = filter; } @Override @@ -98,10 +80,11 @@ public class UnnestStorageAdapter implements StorageAdapter final String input = getUnnestInputIfDirectAccess(unnestColumn); final Pair filterPair = computeBaseAndPostUnnestFilters( spec.getFilter(), - unnestFilter != null ? unnestFilter.toFilter() : null, + filter != null ? filter.toFilter() : null, spec.getVirtualColumns(), input, - input == null ? null : spec.getVirtualColumns().getColumnCapabilitiesWithFallback(baseAdapter, input) + input == null ? null : spec.getVirtualColumns() + .getColumnCapabilitiesWithFallback(baseCursorFactory, input) ); final CursorBuildSpec unnestBuildSpec = CursorBuildSpec.builder(spec) @@ -113,7 +96,7 @@ public class UnnestStorageAdapter implements StorageAdapter { final Closer closer = Closer.create(); final Supplier cursorHolderSupplier = Suppliers.memoize( - () -> closer.register(baseAdapter.makeCursorHolder(unnestBuildSpec)) + () -> closer.register(baseCursorFactory.makeCursorHolder(unnestBuildSpec)) ); @Override @@ -134,14 +117,14 @@ public class UnnestStorageAdapter implements StorageAdapter cursor, cursor.getColumnSelectorFactory(), unnestColumn, - outputColumnName + unnestColumn.getOutputName() ); } else { unnestCursor = new UnnestColumnValueSelectorCursor( cursor, cursor.getColumnSelectorFactory(), unnestColumn, - outputColumnName + unnestColumn.getOutputName() ); } return PostJoinCursor.wrap( @@ -165,118 +148,57 @@ public class UnnestStorageAdapter implements StorageAdapter }; } - @Override - public Interval getInterval() - { - return baseAdapter.getInterval(); - } - @Override public RowSignature getRowSignature() { final RowSignature.Builder builder = RowSignature.builder(); - final RowSignature baseSignature = baseAdapter.getRowSignature(); + final RowSignature baseSignature = baseCursorFactory.getRowSignature(); for (int i = 0; i < baseSignature.size(); i++) { final String column = baseSignature.getColumnName(i); - if (!outputColumnName.equals(column)) { + if (!unnestColumn.getOutputName().equals(column)) { builder.add(column, ColumnType.fromCapabilities(getColumnCapabilities(column))); } } - return builder.add(outputColumnName, ColumnType.fromCapabilities(getColumnCapabilities(outputColumnName))).build(); + return builder.add( + unnestColumn.getOutputName(), + ColumnType.fromCapabilities(getColumnCapabilities(unnestColumn.getOutputName())) + ).build(); } - @Override - public Indexed getAvailableDimensions() - { - final LinkedHashSet availableDimensions = new LinkedHashSet<>(); - - for (String dim : baseAdapter.getAvailableDimensions()) { - availableDimensions.add(dim); - } - availableDimensions.add(outputColumnName); - return new ListIndexed<>(Lists.newArrayList(availableDimensions)); - } - - @Override - public Iterable getAvailableMetrics() - { - return baseAdapter.getAvailableMetrics(); - } - - @Nullable - public Filter getUnnestFilter() - { - return unnestFilter == null ? null : unnestFilter.toFilter(); - } - - @Override - public int getDimensionCardinality(String column) - { - if (!outputColumnName.equals(column)) { - return baseAdapter.getDimensionCardinality(column); - } - return DimensionDictionarySelector.CARDINALITY_UNKNOWN; - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - if (outputColumnName.equals(column)) { - return null; - } - - return baseAdapter.getMinValue(column); - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - if (outputColumnName.equals(column)) { - return null; - } - - return baseAdapter.getMaxValue(column); - } @Nullable @Override public ColumnCapabilities getColumnCapabilities(String column) { - if (outputColumnName.equals(column)) { - return computeOutputColumnCapabilities(baseAdapter, unnestColumn); + if (unnestColumn.getOutputName().equals(column)) { + return computeOutputColumnCapabilities(baseCursorFactory, unnestColumn); } - return baseAdapter.getColumnCapabilities(column); - } - - @Override - public int getNumRows() - { - return 0; - } - - @Nullable - @Override - public Metadata getMetadata() - { - return baseAdapter.getMetadata(); - } - - @Override - public boolean isFromTombstone() - { - return baseAdapter.isFromTombstone(); + return baseCursorFactory.getColumnCapabilities(column); } + @VisibleForTesting public VirtualColumn getUnnestColumn() { return unnestColumn; } + /** + * Returns the input of {@link #unnestColumn}, if it's a direct access; otherwise returns null. + */ + @VisibleForTesting + @Nullable + public String getUnnestInputIfDirectAccess(VirtualColumn unnestColumn) + { + if (unnestColumn instanceof ExpressionVirtualColumn) { + return ((ExpressionVirtualColumn) unnestColumn).getParsedExpression().get().getBindingIfIdentifier(); + } else { + return null; + } + } + /** * Split queryFilter into pre- and post-correlate filters. * @@ -287,6 +209,7 @@ public class UnnestStorageAdapter implements StorageAdapter * @param inputColumnCapabilites input column capabilities if known; otherwise null * @return pair of pre- and post-unnest filters */ + @VisibleForTesting public Pair computeBaseAndPostUnnestFilters( @Nullable final Filter queryFilter, @Nullable final Filter unnestFilter, @@ -333,10 +256,10 @@ public class UnnestStorageAdapter implements StorageAdapter filtersPushedDownToBaseCursor -> null (as the filter cannot be re-written due to presence of virtual columns) filtersForPostUnnestCursor -> d12 IN (a,b) or m1 < 10 */ - final FilterSplitter filterSplitter = new FilterSplitter(inputColumn, inputColumnCapabilites, queryVirtualColumns); + final FilterSplitter filterSplitter = new FilterSplitter(inputColumn, inputColumnCapabilites, unnestColumn, queryVirtualColumns); if (queryFilter != null) { - if (queryFilter.getRequiredColumns().contains(outputColumnName)) { + if (queryFilter.getRequiredColumns().contains(unnestColumn.getOutputName())) { // outside filter contains unnested column // requires check for OR and And filters, disqualify rewrite for non-unnest filters if (queryFilter instanceof BooleanFilter) { @@ -374,11 +297,219 @@ public class UnnestStorageAdapter implements StorageAdapter ); } - class FilterSplitter + /** + * handles the nested rewrite for unnest columns in recursive way, + * it loops through all and/or filters and rewrite only required filters in the child and add it to preFilter if qualified + * or else skip adding it to preFilters. + * RULES: + * 1. Add to preFilters only when top level filter is AND. + * for example: a=1 and (b=2 or c=2) , In this case a=1 can be added as preFilters but we can not add b=2 as preFilters. + * 2. If Top level is OR filter then we can either choose to add entire top level OR filter to preFilter or skip it all together. + * for example: a=1 or (b=2 and c=2) + * 3. Filters on unnest column which is derived from Array or any other Expression can not be pushe down to base. + * for example: a=1 and vc=3 , lets say vc is ExpressionVirtualColumn, and vc=3 can not be push down to base even if top level is AND filter. + * A. Unnesting a single dimension e.g. select * from foo, UNNEST(MV_TO_ARRAY(dim3)) as u(d3) + * B. Unnesting an expression from multiple columns e.g. select * from foo, UNNEST(ARRAY[dim1,dim2]) as u(d12) + * In case A, d3 is a direct reference to dim3 so any filter using d3 can be rewritten using dim3 and added to pre filter + * while in case B, due to presence of the expression virtual column expressionVirtualColumn("j0.unnest", "array(\"dim1\",\"dim2\")", ColumnType.STRING_ARRAY) + * the filters on d12 cannot be pushed to the pre filters + * + * @param queryFilter query filter from {@link CursorBuildSpec} + * @param inputColumn input column to unnest if it's a direct access; otherwise null + * @param inputColumnCapabilites input column capabilities if known; otherwise null + */ + private List recursiveRewriteOnUnnestFilters( + BooleanFilter queryFilter, + final String inputColumn, + final ColumnCapabilities inputColumnCapabilites, + final FilterSplitter filterSplitter + ) { - private String inputColumn; - private ColumnCapabilities inputColumnCapabilites; - private VirtualColumns queryVirtualColumns; + final List preFilterList = new ArrayList<>(); + for (Filter filter : queryFilter.getFilters()) { + if (filter.getRequiredColumns().contains(unnestColumn.getOutputName())) { + if (filter instanceof AndFilter) { + List andChildFilters = recursiveRewriteOnUnnestFilters( + (BooleanFilter) filter, + inputColumn, + inputColumnCapabilites, + filterSplitter + ); + if (!andChildFilters.isEmpty()) { + preFilterList.add(new AndFilter(andChildFilters)); + } + } else if (filter instanceof OrFilter) { + List orChildFilters = recursiveRewriteOnUnnestFilters( + (BooleanFilter) filter, + inputColumn, + inputColumnCapabilites, + filterSplitter + ); + if (orChildFilters.size() == ((OrFilter) filter).getFilters().size()) { + preFilterList.add(new OrFilter(orChildFilters)); + } + } else if (filter instanceof NotFilter) { + // nothing to do here... + continue; + } else { + // can we rewrite + final Filter newFilter = rewriteFilterOnUnnestColumnIfPossible( + filter, + unnestColumn, + inputColumn, + inputColumnCapabilites + ); + if (newFilter != null) { + // this is making sure that we are not pushing the unnest columns filters to base filter without rewriting. + preFilterList.add(newFilter); + filterSplitter.addToPreFilterCount(1); + } + filterSplitter.addToOriginalFilterCount(1); + } + } else { + preFilterList.add(filter); + // for filters on non unnest columns, we still need to count the nested filters if any as we are not traversing it in this method + int filterCount = Filters.countNumberOfFilters(filter); + filterSplitter.addToOriginalFilterCount(filterCount); + filterSplitter.addToPreFilterCount(filterCount); + } + } + return preFilterList; + } + + /** + * Computes ordering of a join {@link CursorHolder} based on the ordering of an underlying {@link CursorHolder}. + */ + private List computeOrdering(final List baseOrdering) + { + // Sorted the same way as the base segment, unless the unnested column shadows one of the base columns. + int limit = 0; + for (; limit < baseOrdering.size(); limit++) { + final String columnName = baseOrdering.get(limit).getColumnName(); + if (columnName.equals(unnestColumn.getOutputName())) { + break; + } + } + + return limit == baseOrdering.size() ? baseOrdering : baseOrdering.subList(0, limit); + } + + /** + * Rewrites a filter on {@link #unnestColumn} to operate on the input column from + * if possible. + */ + @Nullable + private static Filter rewriteFilterOnUnnestColumnIfPossible( + final Filter filter, + final VirtualColumn unnestColumn, + @Nullable final String inputColumn, + @Nullable final ColumnCapabilities inputColumnCapabilities + ) + { + // Only doing this for multi-value strings (not array types) at the moment. + if (inputColumn == null + || inputColumnCapabilities == null + || inputColumnCapabilities.getType() != ValueType.STRING) { + return null; + } + + if (filterMapsOverMultiValueStrings(filter)) { + return filter.rewriteRequiredColumns(ImmutableMap.of(unnestColumn.getOutputName(), inputColumn)); + } else { + return null; + } + } + + /** + * Computes the capabilities of {@link #unnestColumn}, after unnesting. + */ + @Nullable + public static ColumnCapabilities computeOutputColumnCapabilities( + final ColumnInspector baseColumnInspector, + final VirtualColumn unnestColumn + ) + { + final ColumnCapabilities capabilities = unnestColumn.capabilities( + baseColumnInspector, + unnestColumn.getOutputName() + ); + + if (capabilities == null) { + return null; + } else { + // Arrays are unnested as their element type. Anything else is unnested as the same type. + final TypeSignature outputType = + capabilities.isArray() ? capabilities.getElementType() : capabilities.toColumnType(); + + final boolean useDimensionCursor = useDimensionCursor(capabilities); + return ColumnCapabilitiesImpl.createDefault() + .setType(outputType) + .setHasMultipleValues(false) + .setDictionaryEncoded(useDimensionCursor) + .setDictionaryValuesUnique(useDimensionCursor); + } + } + + /** + * Requirement for {@link #rewriteFilterOnUnnestColumnIfPossible}: filter must support rewrites and also must map + * over multi-value strings. (Rather than treat them as arrays.) There isn't a method on the Filter interface that + * tells us this, so resort to instanceof. + */ + private static boolean filterMapsOverMultiValueStrings(final Filter filter) + { + if (filter instanceof BooleanFilter) { + for (Filter child : ((BooleanFilter) filter).getFilters()) { + if (!filterMapsOverMultiValueStrings(child)) { + return false; + } + } + return true; + } else if (filter instanceof NotFilter) { + return false; + } else { + return filter instanceof SelectorFilter + || filter instanceof InDimFilter + || filter instanceof LikeFilter + || filter instanceof BoundFilter + || filter instanceof NullFilter + || filter instanceof EqualityFilter + || filter instanceof RangeFilter; + } + } + + /** + * Array and nested array columns are dictionary encoded, but not correctly for {@link UnnestDimensionCursor} which + * is tailored for scalar logical type values that are {@link ColumnCapabilities#isDictionaryEncoded()} and possibly + * with {@link ColumnCapabilities#hasMultipleValues()} (specifically {@link ValueType#STRING}), so we don't want to + * use this cursor if the capabilities are unknown or if the column type is {@link ValueType#ARRAY}. + */ + private static boolean useDimensionCursor(@Nullable ColumnCapabilities capabilities) + { + if (capabilities == null) { + // capabilities being null here should be indicative of the column not existing or being a virtual column with + // no type information, chances are it is not going to be using a very cool dimension selector and so wont work + // with this, which requires real dictionary ids for the value matcher to work correctly + return false; + } + // the column needs real, unique value dictionary so that the value matcher id lookup works correctly, otherwise + // we must not use the dimension selector + if (capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) { + // if we got here, we only actually want to do this for dictionary encoded strings, since no other dictionary + // encoded column type should ever have multiple values set. nested and array columns are also dictionary encoded, + // but for arrays, the row is always a single dictionary id which maps to the entire array instead of an array + // of ids for each element, so we don't want to ever use the dimension selector cursor for that + return capabilities.is(ValueType.STRING); + } + // wasn't a dictionary encoded string, use the value selector + return false; + } + + private static class FilterSplitter + { + private final String inputColumn; + private final ColumnCapabilities inputColumnCapabilites; + private final VirtualColumn unnestColumn; + private final VirtualColumns queryVirtualColumns; private int originalFilterCount = 0; private int preFilterCount = 0; @@ -386,11 +517,13 @@ public class UnnestStorageAdapter implements StorageAdapter public FilterSplitter( String inputColumn, ColumnCapabilities inputColumnCapabilites, + VirtualColumn unnestColumn, VirtualColumns queryVirtualColumns ) { this.inputColumn = inputColumn; this.inputColumnCapabilites = inputColumnCapabilites; + this.unnestColumn = unnestColumn; this.queryVirtualColumns = queryVirtualColumns; } @@ -403,7 +536,7 @@ public class UnnestStorageAdapter implements StorageAdapter return; } if (!skipPreFilters) { - final Filter newFilter = rewriteFilterOnUnnestColumnIfPossible(filter, inputColumn, inputColumnCapabilites); + final Filter newFilter = rewriteFilterOnUnnestColumnIfPossible(filter, unnestColumn, inputColumn, inputColumnCapabilites); if (newFilter != null) { // Add the rewritten filter pre-unnest, so we get the benefit of any indexes, and so we avoid unnesting // any rows that do not match this filter at all. @@ -460,222 +593,4 @@ public class UnnestStorageAdapter implements StorageAdapter return preFilterCount; } } - - /** - * handles the nested rewrite for unnest columns in recursive way, - * it loops through all and/or filters and rewrite only required filters in the child and add it to preFilter if qualified - * or else skip adding it to preFilters. - * RULES: - * 1. Add to preFilters only when top level filter is AND. - * for example: a=1 and (b=2 or c=2) , In this case a=1 can be added as preFilters but we can not add b=2 as preFilters. - * 2. If Top level is OR filter then we can either choose to add entire top level OR filter to preFilter or skip it all together. - * for example: a=1 or (b=2 and c=2) - * 3. Filters on unnest column which is derived from Array or any other Expression can not be pushe down to base. - * for example: a=1 and vc=3 , lets say vc is ExpressionVirtualColumn, and vc=3 can not be push down to base even if top level is AND filter. - * A. Unnesting a single dimension e.g. select * from foo, UNNEST(MV_TO_ARRAY(dim3)) as u(d3) - * B. Unnesting an expression from multiple columns e.g. select * from foo, UNNEST(ARRAY[dim1,dim2]) as u(d12) - * In case A, d3 is a direct reference to dim3 so any filter using d3 can be rewritten using dim3 and added to pre filter - * while in case B, due to presence of the expression virtual column expressionVirtualColumn("j0.unnest", "array(\"dim1\",\"dim2\")", ColumnType.STRING_ARRAY) - * the filters on d12 cannot be pushed to the pre filters - * - * @param queryFilter query filter from {@link CursorBuildSpec} - * @param inputColumn input column to unnest if it's a direct access; otherwise null - * @param inputColumnCapabilites input column capabilities if known; otherwise null - */ - private List recursiveRewriteOnUnnestFilters( - BooleanFilter queryFilter, - final String inputColumn, - final ColumnCapabilities inputColumnCapabilites, - final FilterSplitter filterSplitter - ) - { - final List preFilterList = new ArrayList<>(); - for (Filter filter : queryFilter.getFilters()) { - if (filter.getRequiredColumns().contains(outputColumnName)) { - if (filter instanceof AndFilter) { - List andChildFilters = recursiveRewriteOnUnnestFilters( - (BooleanFilter) filter, - inputColumn, - inputColumnCapabilites, - filterSplitter - ); - if (!andChildFilters.isEmpty()) { - preFilterList.add(new AndFilter(andChildFilters)); - } - } else if (filter instanceof OrFilter) { - List orChildFilters = recursiveRewriteOnUnnestFilters( - (BooleanFilter) filter, - inputColumn, - inputColumnCapabilites, - filterSplitter - ); - if (orChildFilters.size() == ((OrFilter) filter).getFilters().size()) { - preFilterList.add(new OrFilter(orChildFilters)); - } - } else if (filter instanceof NotFilter) { - // nothing to do here... - continue; - } else { - // can we rewrite - final Filter newFilter = rewriteFilterOnUnnestColumnIfPossible( - filter, - inputColumn, - inputColumnCapabilites - ); - if (newFilter != null) { - // this is making sure that we are not pushing the unnest columns filters to base filter without rewriting. - preFilterList.add(newFilter); - filterSplitter.addToPreFilterCount(1); - } - filterSplitter.addToOriginalFilterCount(1); - } - } else { - preFilterList.add(filter); - // for filters on non unnest columns, we still need to count the nested filters if any as we are not traversing it in this method - int filterCount = Filters.countNumberOfFilters(filter); - filterSplitter.addToOriginalFilterCount(filterCount); - filterSplitter.addToPreFilterCount(filterCount); - } - } - return preFilterList; - } - /** - * Returns the input of {@link #unnestColumn}, if it's a direct access; otherwise returns null. - */ - @Nullable - public String getUnnestInputIfDirectAccess(VirtualColumn unnestColumn) - { - if (unnestColumn instanceof ExpressionVirtualColumn) { - return ((ExpressionVirtualColumn) unnestColumn).getParsedExpression().get().getBindingIfIdentifier(); - } else { - return null; - } - } - - /** - * Rewrites a filter on {@link #outputColumnName} to operate on the input column from - * if possible. - */ - @Nullable - private Filter rewriteFilterOnUnnestColumnIfPossible( - final Filter filter, - @Nullable final String inputColumn, - @Nullable final ColumnCapabilities inputColumnCapabilities - ) - { - // Only doing this for multi-value strings (not array types) at the moment. - if (inputColumn == null - || inputColumnCapabilities == null - || inputColumnCapabilities.getType() != ValueType.STRING) { - return null; - } - - if (filterMapsOverMultiValueStrings(filter)) { - return filter.rewriteRequiredColumns(ImmutableMap.of(outputColumnName, inputColumn)); - } else { - return null; - } - } - - /** - * Computes ordering of a join {@link CursorHolder} based on the ordering of an underlying {@link CursorHolder}. - */ - private List computeOrdering(final List baseOrdering) - { - // Sorted the same way as the base segment, unless the unnested column shadows one of the base columns. - int limit = 0; - for (; limit < baseOrdering.size(); limit++) { - final String columnName = baseOrdering.get(limit).getColumnName(); - if (columnName.equals(outputColumnName) || columnName.equals(unnestColumn.getOutputName())) { - break; - } - } - - return limit == baseOrdering.size() ? baseOrdering : baseOrdering.subList(0, limit); - } - - /** - * Computes the capabilities of {@link #outputColumnName}, after unnesting. - */ - @Nullable - public static ColumnCapabilities computeOutputColumnCapabilities( - final ColumnInspector baseColumnInspector, - final VirtualColumn unnestColumn - ) - { - final ColumnCapabilities capabilities = unnestColumn.capabilities( - baseColumnInspector, - unnestColumn.getOutputName() - ); - - if (capabilities == null) { - return null; - } else { - // Arrays are unnested as their element type. Anything else is unnested as the same type. - final TypeSignature outputType = - capabilities.isArray() ? capabilities.getElementType() : capabilities.toColumnType(); - - final boolean useDimensionCursor = useDimensionCursor(capabilities); - return ColumnCapabilitiesImpl.createDefault() - .setType(outputType) - .setHasMultipleValues(false) - .setDictionaryEncoded(useDimensionCursor) - .setDictionaryValuesUnique(useDimensionCursor); - } - } - - /** - * Requirement for {@link #rewriteFilterOnUnnestColumnIfPossible}: filter must support rewrites and also must map - * over multi-value strings. (Rather than treat them as arrays.) There isn't a method on the Filter interface that - * tells us this, so resort to instanceof. - */ - @VisibleForTesting - static boolean filterMapsOverMultiValueStrings(final Filter filter) - { - if (filter instanceof BooleanFilter) { - for (Filter child : ((BooleanFilter) filter).getFilters()) { - if (!filterMapsOverMultiValueStrings(child)) { - return false; - } - } - return true; - } else if (filter instanceof NotFilter) { - return false; - } else { - return filter instanceof SelectorFilter - || filter instanceof InDimFilter - || filter instanceof LikeFilter - || filter instanceof BoundFilter - || filter instanceof NullFilter - || filter instanceof EqualityFilter - || filter instanceof RangeFilter; - } - } - - /** - * Array and nested array columns are dictionary encoded, but not correctly for {@link UnnestDimensionCursor} which - * is tailored for scalar logical type values that are {@link ColumnCapabilities#isDictionaryEncoded()} and possibly - * with {@link ColumnCapabilities#hasMultipleValues()} (specifically {@link ValueType#STRING}), so we don't want to - * use this cursor if the capabilities are unknown or if the column type is {@link ValueType#ARRAY}. - */ - private static boolean useDimensionCursor(@Nullable ColumnCapabilities capabilities) - { - if (capabilities == null) { - // capabilities being null here should be indicative of the column not existing or being a virtual column with - // no type information, chances are it is not going to be using a very cool dimension selector and so wont work - // with this, which requires real dictionary ids for the value matcher to work correctly - return false; - } - // the column needs real, unique value dictionary so that the value matcher id lookup works correctly, otherwise - // we must not use the dimension selector - if (capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) { - // if we got here, we only actually want to do this for dictionary encoded strings, since no other dictionary - // encoded column type should ever have multiple values set. nested and array columns are also dictionary encoded, - // but for arrays, the row is always a single dictionary id which maps to the entire array instead of an array - // of ids for each element, so we don't want to ever use the dimension selector cursor for that - return capabilities.is(ValueType.STRING); - } - // wasn't a dictionary encoded string, use the value selector - return false; - } } diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java index 398c0a21def..3012d31ff2a 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java @@ -251,7 +251,7 @@ public class UnnestDimensionCursor implements Cursor public ColumnCapabilities getColumnCapabilities(String column) { if (outputName.equals(column)) { - return UnnestStorageAdapter.computeOutputColumnCapabilities(baseColumnSelectorFactory, unnestColumn); + return UnnestCursorFactory.computeOutputColumnCapabilities(baseColumnSelectorFactory, unnestColumn); } return baseColumnSelectorFactory.getColumnCapabilities(column); diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestSegment.java b/processing/src/main/java/org/apache/druid/segment/UnnestSegment.java new file mode 100644 index 00000000000..22b63655f52 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/UnnestSegment.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.segment; + +import org.apache.druid.query.filter.DimFilter; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +public class UnnestSegment extends WrappedSegmentReference +{ + private final VirtualColumn unnestColumn; + @Nullable + private final DimFilter filter; + + public UnnestSegment( + SegmentReference delegate, + VirtualColumn unnestColumn, + @Nullable DimFilter filter + ) + { + super(delegate); + this.unnestColumn = unnestColumn; + this.filter = filter; + } + + @Override + public CursorFactory asCursorFactory() + { + return new UnnestCursorFactory(delegate.asCursorFactory(), unnestColumn, filter); + } + + @Nullable + @Override + public T as(@Nonnull Class clazz) + { + if (TopNOptimizationInspector.class.equals(clazz)) { + return (T) new SimpleTopNOptimizationInspector(filter == null); + } + return super.as(clazz); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java b/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java index c5f2474efdf..8b50315e2f4 100644 --- a/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java @@ -97,8 +97,8 @@ public interface VectorColumnProcessorFactory T makeObjectProcessor(@SuppressWarnings("unused") ColumnCapabilities capabilities, VectorObjectSelector selector); /** - * The processor factory can influence the decision on whether or not to prefer a dictionary encoded column value - * selector over a an object selector by examining the {@link ColumnCapabilities}. + * The processor factory can influence the decision on whether to prefer a dictionary encoded column value selector + * over an object selector by examining the {@link ColumnCapabilities}. * * By default, all processor factories prefer to use a dictionary encoded selector if the column has a dictionary * available ({@link ColumnCapabilities#isDictionaryEncoded()} is true), and there is a unique mapping of dictionary diff --git a/processing/src/main/java/org/apache/druid/segment/VirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/VirtualColumn.java index ca9408d14e4..b7ca00d9e5d 100644 --- a/processing/src/main/java/org/apache/druid/segment/VirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/VirtualColumn.java @@ -21,6 +21,7 @@ package org.apache.druid.segment; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.annotations.SubclassesMustOverrideEqualsAndHashCode; import org.apache.druid.java.util.common.Cacheable; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.ColumnIndexSelector; @@ -336,4 +337,22 @@ public interface VirtualColumn extends Cacheable { return NoIndexesColumnIndexSupplier.getInstance(); } + + /** + * Returns a key used for "equivalence" comparisons, for checking if some virtual column is equivalent to some other + * virtual column, regardless of the output name. If this method returns null, it does not participate in equivalence + * comparisons. + * + * @see VirtualColumns#findEquivalent(VirtualColumn) + */ + @Nullable + default EquivalenceKey getEquivalanceKey() + { + return null; + } + + @SubclassesMustOverrideEqualsAndHashCode + interface EquivalenceKey + { + } } diff --git a/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java index bc2609b4a00..676067a959f 100644 --- a/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java +++ b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java @@ -26,6 +26,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.druid.java.util.common.Cacheable; import org.apache.druid.java.util.common.IAE; @@ -131,10 +132,13 @@ public class VirtualColumns implements Cacheable // For equals, hashCode, toString, and serialization: private final List virtualColumns; private final List virtualColumnNames; + // For equivalence + private final Map equivalence; // For getVirtualColumn: private final Map withDotSupport; private final Map withoutDotSupport; + private final boolean hasNoDotColumns; private VirtualColumns( List virtualColumns, @@ -146,10 +150,15 @@ public class VirtualColumns implements Cacheable this.withDotSupport = withDotSupport; this.withoutDotSupport = withoutDotSupport; this.virtualColumnNames = new ArrayList<>(virtualColumns.size()); - + this.hasNoDotColumns = withDotSupport.isEmpty(); + this.equivalence = Maps.newHashMapWithExpectedSize(virtualColumns.size()); for (VirtualColumn virtualColumn : virtualColumns) { detectCycles(virtualColumn, null); virtualColumnNames.add(virtualColumn.getOutputName()); + VirtualColumn.EquivalenceKey key = virtualColumn.getEquivalanceKey(); + if (key != null) { + equivalence.put(key, virtualColumn); + } } } @@ -172,10 +181,23 @@ public class VirtualColumns implements Cacheable if (vc != null) { return vc; } + if (hasNoDotColumns) { + return null; + } final String baseColumnName = splitColumnName(columnName).lhs; return withDotSupport.get(baseColumnName); } + /** + * Check if a virtual column is already defined which is the same as some other virtual column, ignoring output name, + * returning that virtual column if it exists, or null if there is no equivalent virtual column. + */ + @Nullable + public VirtualColumn findEquivalent(VirtualColumn virtualColumn) + { + return equivalence.get(virtualColumn.getEquivalanceKey()); + } + /** * Get the {@link ColumnIndexSupplier} of the specified virtual column, with the assistance of a * {@link ColumnSelector} to allow reading things from segments. If the column does not have indexes this method diff --git a/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java b/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java index 19db47067b5..97bb1afd4b7 100644 --- a/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java +++ b/processing/src/main/java/org/apache/druid/segment/WrappedSegmentReference.java @@ -19,7 +19,6 @@ package org.apache.druid.segment; -import org.apache.druid.query.FilteredDataSource; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -28,25 +27,20 @@ import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.util.Optional; -import java.util.function.Function; /** - * This class is used as a wrapper for other classes that just want to - * modify the storage adapter for a datasource. Examples include: - * {@link org.apache.druid.query.UnnestDataSource}, {@link FilteredDataSource} + * Simple {@link SegmentReference} implementation for a segment that wraps a base segment such as + * {@link UnnestSegment} or {@link FilteredSegment} */ -public class WrappedSegmentReference implements SegmentReference +public abstract class WrappedSegmentReference implements SegmentReference { - private final SegmentReference delegate; - private final Function storageAdapterWrapperFunction; + protected final SegmentReference delegate; public WrappedSegmentReference( - SegmentReference delegate, - Function storageAdapterWrapperFunction + SegmentReference delegate ) { this.delegate = delegate; - this.storageAdapterWrapperFunction = storageAdapterWrapperFunction; } @Override @@ -74,12 +68,6 @@ public class WrappedSegmentReference implements SegmentReference return delegate.asQueryableIndex(); } - @Override - public StorageAdapter asStorageAdapter() - { - return storageAdapterWrapperFunction.apply(delegate.asStorageAdapter()); - } - @Nullable @Override public T as(@Nonnull Class clazz) @@ -91,10 +79,22 @@ public class WrappedSegmentReference implements SegmentReference } } + @Override + public boolean isTombstone() + { + return delegate.isTombstone(); + } + @Override public void close() throws IOException { delegate.close(); } + + @Override + public String asString() + { + return delegate.asString(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java index dfc618acad8..c4172fc6ce2 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java @@ -72,9 +72,68 @@ public class AndFilter implements BooleanFilter this(new LinkedHashSet<>(filters)); } + public static ValueMatcher makeMatcher(final ValueMatcher[] baseMatchers) + { + Preconditions.checkState(baseMatchers.length > 0); + if (baseMatchers.length == 1) { + return baseMatchers[0]; + } + + return new ValueMatcher() + { + @Override + public boolean matches(boolean includeUnknown) + { + for (ValueMatcher matcher : baseMatchers) { + if (!matcher.matches(includeUnknown)) { + return false; + } + } + return true; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("firstBaseMatcher", baseMatchers[0]); + inspector.visit("secondBaseMatcher", baseMatchers[1]); + // Don't inspect the 3rd and all consequent baseMatchers, cut runtime shape combinations at this point. + // Anyway if the filter is so complex, Hotspot won't inline all calls because of the inline limit. + } + }; + } + + public static VectorValueMatcher makeVectorMatcher(final VectorValueMatcher[] baseMatchers) + { + Preconditions.checkState(baseMatchers.length > 0); + if (baseMatchers.length == 1) { + return baseMatchers[0]; + } + + return new BaseVectorValueMatcher(baseMatchers[0]) + { + @Override + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) + { + ReadableVectorMatch match = mask; + + for (VectorValueMatcher matcher : baseMatchers) { + if (match.isAllFalse()) { + // Short-circuit if the entire vector is false. + break; + } + match = matcher.match(match, includeUnknown); + } + + assert match.isValid(mask); + return match; + } + }; + } + @Override public FilterBundle makeFilterBundle( - ColumnIndexSelector columnIndexSelector, + FilterBundle.Builder filterBundleBuilder, BitmapResultFactory bitmapResultFactory, int applyRowCount, int totalRowCount, @@ -97,20 +156,21 @@ public class AndFilter implements BooleanFilter // a nested AND filter might also partition itself into indexes and bundles, and since it is part of a logical AND // operation, this is valid (and even preferable). final long bitmapConstructionStartNs = System.nanoTime(); - for (Filter subfilter : filters) { - final FilterBundle subBundle = subfilter.makeFilterBundle( - columnIndexSelector, + for (FilterBundle.Builder subFilterBundleBuilder : filterBundleBuilder.getChildBuilders()) { + final FilterBundle subBundle = subFilterBundleBuilder.build( bitmapResultFactory, Math.min(applyRowCount, indexIntersectionSize), totalRowCount, includeUnknown ); - if (subBundle.getIndex() != null) { + if (subBundle.hasIndex()) { if (subBundle.getIndex().getBitmap().isEmpty()) { // if nothing matches for any sub filter, short-circuit, because nothing can possibly match return FilterBundle.allFalse( System.nanoTime() - bitmapConstructionStartNs, - columnIndexSelector.getBitmapFactory().makeEmptyImmutableBitmap() + subFilterBundleBuilder.getColumnIndexSelector() + .getBitmapFactory() + .makeEmptyImmutableBitmap() ); } merged = merged.merge(subBundle.getIndex().getIndexCapabilities()); @@ -122,7 +182,7 @@ public class AndFilter implements BooleanFilter } indexIntersectionSize = index.size(); } - if (subBundle.getMatcherBundle() != null) { + if (subBundle.hasMatcher()) { matcherBundles.add(subBundle.getMatcherBundle()); matcherBundleInfos.add(subBundle.getMatcherBundle().getMatcherInfo()); } @@ -131,11 +191,7 @@ public class AndFilter implements BooleanFilter final FilterBundle.IndexBundle indexBundle; if (index != null) { if (indexBundleInfos.size() == 1) { - indexBundle = new FilterBundle.SimpleIndexBundle( - indexBundleInfos.get(0), - index, - merged - ); + indexBundle = new FilterBundle.SimpleIndexBundle(indexBundleInfos.get(0), index, merged); } else { indexBundle = new FilterBundle.SimpleIndexBundle( new FilterBundle.IndexBundleInfo( @@ -162,11 +218,7 @@ public class AndFilter implements BooleanFilter if (matcherBundles.size() == 1) { return matcherBundleInfos.get(0); } - return new FilterBundle.MatcherBundleInfo( - () -> "AND", - null, - matcherBundleInfos - ); + return new FilterBundle.MatcherBundleInfo(() -> "AND", null, matcherBundleInfos); } @Override @@ -180,7 +232,10 @@ public class AndFilter implements BooleanFilter } @Override - public VectorValueMatcher vectorMatcher(VectorColumnSelectorFactory selectorFactory, ReadableVectorOffset baseOffset) + public VectorValueMatcher vectorMatcher( + VectorColumnSelectorFactory selectorFactory, + ReadableVectorOffset baseOffset + ) { final VectorValueMatcher[] vectorMatchers = new VectorValueMatcher[matcherBundles.size()]; for (int i = 0; i < matcherBundles.size(); i++) { @@ -204,10 +259,7 @@ public class AndFilter implements BooleanFilter matcherBundle = null; } - return new FilterBundle( - indexBundle, - matcherBundle - ); + return new FilterBundle(indexBundle, matcherBundle); } @Nullable @@ -239,6 +291,13 @@ public class AndFilter implements BooleanFilter return finalMerged; } + @Override + public int estimatedComputeCost() + { + // There's no additional cost on AND filter, cost in child filters would be summed. + return 0; + } + @Override public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { @@ -350,65 +409,6 @@ public class AndFilter implements BooleanFilter return StringUtils.format("(%s)", AND_JOINER.join(filters)); } - public static ValueMatcher makeMatcher(final ValueMatcher[] baseMatchers) - { - Preconditions.checkState(baseMatchers.length > 0); - if (baseMatchers.length == 1) { - return baseMatchers[0]; - } - - return new ValueMatcher() - { - @Override - public boolean matches(boolean includeUnknown) - { - for (ValueMatcher matcher : baseMatchers) { - if (!matcher.matches(includeUnknown)) { - return false; - } - } - return true; - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("firstBaseMatcher", baseMatchers[0]); - inspector.visit("secondBaseMatcher", baseMatchers[1]); - // Don't inspect the 3rd and all consequent baseMatchers, cut runtime shape combinations at this point. - // Anyway if the filter is so complex, Hotspot won't inline all calls because of the inline limit. - } - }; - } - - public static VectorValueMatcher makeVectorMatcher(final VectorValueMatcher[] baseMatchers) - { - Preconditions.checkState(baseMatchers.length > 0); - if (baseMatchers.length == 1) { - return baseMatchers[0]; - } - - return new BaseVectorValueMatcher(baseMatchers[0]) - { - @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) - { - ReadableVectorMatch match = mask; - - for (VectorValueMatcher matcher : baseMatchers) { - if (match.isAllFalse()) { - // Short-circuit if the entire vector is false. - break; - } - match = matcher.match(match, includeUnknown); - } - - assert match.isValid(mask); - return match; - } - }; - } - @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java index 700b2fbfa16..e8bdce85c9b 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java @@ -78,330 +78,6 @@ public class OrFilter implements BooleanFilter this(new LinkedHashSet<>(filters)); } - @Override - public FilterBundle makeFilterBundle( - ColumnIndexSelector columnIndexSelector, - BitmapResultFactory bitmapResultFactory, - int applyRowCount, - int totalRowCount, - boolean includeUnknown - ) - { - // for OR filters, we have a few possible outcomes: - // 1 - all clauses are index only bundles. in this case we union the bitmaps together and make an index only bundle - // 2 - some clauses support indexes. in this case, we union the bitmaps of any index only bundles together to form a - // partial index which is constructed into a matcher bundle with convertIndexToMatcherBundle. We translate any - // index AND matcher bundles into a matcher only bundle with convertBundleToMatcherOnlyBundle. Finally, we - // combine these with the remaining matcher only bundles to with makeMatcher/makeVectorMatcher to make a matcher - // only bundle - // 3 - no clauses support indexes. in this case, we make a matcher only bundle using makeMatcher/makeVectorMatcher - - final List indexOnlyBundles = new ArrayList<>(); - final List indexOnlyBundlesInfo = new ArrayList<>(); - final List partialIndexBundles = new ArrayList<>(); - final List matcherOnlyBundles = new ArrayList<>(); - - int indexUnionSize = 0; - ImmutableBitmap index = null; - ColumnIndexCapabilities merged = new SimpleColumnIndexCapabilities(true, true); - int emptyCount = 0; - - final long bitmapConstructionStartNs = System.nanoTime(); - - for (Filter subfilter : filters) { - final FilterBundle bundle = subfilter.makeFilterBundle( - columnIndexSelector, - bitmapResultFactory, - Math.min(applyRowCount, totalRowCount - indexUnionSize), - totalRowCount, - includeUnknown - ); - if (bundle.hasIndex()) { - final ImmutableBitmap bundleIndex = bundle.getIndex().getBitmap(); - if (bundleIndex.isEmpty()) { - // we leave any indexes which are empty out of index, indexOnlyBundles, and partialIndexBundles - // even though we skip them, we still keep track of them to check for the case when we can build the OR into - // an index only bundle. We can count index and matcher bundles here too because the AND operation means that - // an empty index means the matcher can be skipped - emptyCount++; - } else { - if (bundle.hasMatcher()) { - // index and matcher bundles must be handled separately, they will need to be a single value matcher built - // by doing an AND operation between the index and the value matcher - // (a bundle is basically an AND operation between the index and matcher if the matcher is present) - partialIndexBundles.add(convertBundleToMatcherOnlyBundle(bundle, bundleIndex)); - } else { - indexOnlyBundles.add(bundle.getIndex()); - indexOnlyBundlesInfo.add(bundle.getIndex().getIndexInfo()); - merged.merge(bundle.getIndex().getIndexCapabilities()); - // union index only bitmaps together; if all sub-filters are 'index only' bundles we will make an index only - // bundle ourselves, else we will use this index as a single value matcher - if (index == null) { - index = bundle.getIndex().getBitmap(); - } else { - index = index.union(bundle.getIndex().getBitmap()); - } - indexUnionSize = index.size(); - } - } - } else { - matcherOnlyBundles.add(bundle.getMatcherBundle()); - } - } - final long totalBitmapConstructTimeNs = System.nanoTime() - bitmapConstructionStartNs; - - - // if all the filters are 'index only', we can make an index only bundle - if (indexOnlyBundles.size() + emptyCount == filters.size()) { - if (index == null || index.isEmpty()) { - return FilterBundle.allFalse( - totalBitmapConstructTimeNs, - columnIndexSelector.getBitmapFactory().makeEmptyImmutableBitmap() - ); - } - if (indexOnlyBundles.size() == 1) { - return new FilterBundle( - indexOnlyBundles.get(0), - null - ); - } - return new FilterBundle( - new FilterBundle.SimpleIndexBundle( - new FilterBundle.IndexBundleInfo( - () -> "OR", - applyRowCount, - totalBitmapConstructTimeNs, - indexOnlyBundlesInfo - ), - index, - merged - ), - null - ); - } - - // if not the index only outcome, we build a matcher only bundle from all the matchers - final int estimatedSize = (indexOnlyBundles.isEmpty() ? 0 : 1) - + partialIndexBundles.size() - + matcherOnlyBundles.size(); - final List allMatcherBundles = Lists.newArrayListWithCapacity(estimatedSize); - final List allMatcherBundlesInfo = Lists.newArrayListWithCapacity(estimatedSize); - if (!indexOnlyBundles.isEmpty()) { - // translate the indexOnly bundles into a single matcher - final FilterBundle.MatcherBundle matcherBundle = convertIndexToMatcherBundle( - applyRowCount, - indexOnlyBundles, - indexOnlyBundlesInfo, - totalBitmapConstructTimeNs, - index - ); - allMatcherBundles.add(matcherBundle); - allMatcherBundlesInfo.add(matcherBundle.getMatcherInfo()); - } - for (FilterBundle.MatcherBundle bundle : partialIndexBundles) { - allMatcherBundles.add(bundle); - allMatcherBundlesInfo.add(bundle.getMatcherInfo()); - } - for (FilterBundle.MatcherBundle bundle : matcherOnlyBundles) { - allMatcherBundles.add(bundle); - allMatcherBundlesInfo.add(bundle.getMatcherInfo()); - } - - return new FilterBundle( - null, - new FilterBundle.MatcherBundle() - { - @Override - public FilterBundle.MatcherBundleInfo getMatcherInfo() - { - return new FilterBundle.MatcherBundleInfo( - () -> "OR", - null, - allMatcherBundlesInfo - ); - } - - @Override - public ValueMatcher valueMatcher(ColumnSelectorFactory selectorFactory, Offset baseOffset, boolean descending) - { - final ValueMatcher[] matchers = new ValueMatcher[allMatcherBundles.size()]; - for (int i = 0; i < allMatcherBundles.size(); i++) { - matchers[i] = allMatcherBundles.get(i).valueMatcher(selectorFactory, baseOffset, descending); - } - return makeMatcher(matchers); - } - - @Override - public VectorValueMatcher vectorMatcher( - VectorColumnSelectorFactory selectorFactory, - ReadableVectorOffset baseOffset - ) - { - final VectorValueMatcher[] matchers = new VectorValueMatcher[allMatcherBundles.size()]; - for (int i = 0; i < allMatcherBundles.size(); i++) { - matchers[i] = allMatcherBundles.get(i).vectorMatcher(selectorFactory, baseOffset); - } - return makeVectorMatcher(matchers); - } - - @Override - public boolean canVectorize() - { - for (FilterBundle.MatcherBundle bundle : allMatcherBundles) { - if (!bundle.canVectorize()) { - return false; - } - } - return true; - } - } - ); - } - - @Nullable - @Override - public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) - { - if (filters.size() == 1) { - return Iterables.getOnlyElement(filters).getBitmapColumnIndex(selector); - } - - List bitmapColumnIndices = new ArrayList<>(filters.size()); - ColumnIndexCapabilities merged = new SimpleColumnIndexCapabilities(true, true); - for (Filter filter : filters) { - BitmapColumnIndex index = filter.getBitmapColumnIndex(selector); - if (index == null) { - // all or nothing - return null; - } - merged = merged.merge(index.getIndexCapabilities()); - bitmapColumnIndices.add(index); - } - - final ColumnIndexCapabilities finalMerged = merged; - return new BitmapColumnIndex() - { - @Override - public ColumnIndexCapabilities getIndexCapabilities() - { - return finalMerged; - } - - @Override - public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) - { - return bitmapResultFactory.union( - () -> bitmapColumnIndices.stream().map(x -> x.computeBitmapResult(bitmapResultFactory, includeUnknown)).iterator() - ); - } - - @Nullable - @Override - public T computeBitmapResult( - BitmapResultFactory bitmapResultFactory, - int applyRowCount, - int totalRowCount, - boolean includeUnknown - ) - { - List results = Lists.newArrayListWithCapacity(bitmapColumnIndices.size()); - for (BitmapColumnIndex index : bitmapColumnIndices) { - final T r = index.computeBitmapResult(bitmapResultFactory, applyRowCount, totalRowCount, includeUnknown); - if (r == null) { - // all or nothing - return null; - } - results.add(r); - } - return bitmapResultFactory.union(results); - } - }; - } - - @Override - public ValueMatcher makeMatcher(ColumnSelectorFactory factory) - { - final ValueMatcher[] matchers = new ValueMatcher[filters.size()]; - - int i = 0; - for (Filter filter : filters) { - matchers[i++] = filter.makeMatcher(factory); - } - return makeMatcher(matchers); - } - - @Override - public VectorValueMatcher makeVectorMatcher(final VectorColumnSelectorFactory factory) - { - final VectorValueMatcher[] matchers = new VectorValueMatcher[filters.size()]; - - int i = 0; - for (Filter filter : filters) { - matchers[i++] = filter.makeVectorMatcher(factory); - } - return makeVectorMatcher(matchers); - } - - @Override - public boolean canVectorizeMatcher(ColumnInspector inspector) - { - return filters.stream().allMatch(filter -> filter.canVectorizeMatcher(inspector)); - } - - @Override - public LinkedHashSet getFilters() - { - return filters; - } - - @Override - public boolean supportsRequiredColumnRewrite() - { - for (Filter filter : filters) { - if (!filter.supportsRequiredColumnRewrite()) { - return false; - } - } - - return true; - } - - @Override - public Filter rewriteRequiredColumns(Map columnRewrites) - { - final List newFilters = new ArrayList<>(filters.size()); - for (Filter filter : filters) { - newFilters.add(filter.rewriteRequiredColumns(columnRewrites)); - } - return new OrFilter(newFilters); - } - - @Override - public String toString() - { - return StringUtils.format("(%s)", OR_JOINER.join(filters)); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - OrFilter orFilter = (OrFilter) o; - return Objects.equals(getFilters(), orFilter.getFilters()); - } - - @Override - public int hashCode() - { - return Objects.hash(getFilters()); - } - - private static ValueMatcher makeMatcher(final ValueMatcher[] baseMatchers) { Preconditions.checkState(baseMatchers.length > 0); @@ -693,6 +369,7 @@ public class OrFilter implements BooleanFilter { final VectorMatch match = VectorMatch.wrap(new int[vectorOffset.getMaxVectorSize()]); int iterOffset = -1; + @Override public ReadableVectorMatch match(ReadableVectorMatch mask, boolean includeUnknown) { @@ -744,4 +421,334 @@ public class OrFilter implements BooleanFilter } }; } + + @Override + public FilterBundle makeFilterBundle( + FilterBundle.Builder filterBundleBuilder, + BitmapResultFactory bitmapResultFactory, + int applyRowCount, + int totalRowCount, + boolean includeUnknown + ) + { + // for OR filters, we have a few possible outcomes: + // 1 - all clauses are index only bundles. in this case we union the bitmaps together and make an index only bundle + // 2 - some clauses support indexes. in this case, we union the bitmaps of any index only bundles together to form a + // partial index which is constructed into a matcher bundle with convertIndexToMatcherBundle. We translate any + // index AND matcher bundles into a matcher only bundle with convertBundleToMatcherOnlyBundle. Finally, we + // combine these with the remaining matcher only bundles to with makeMatcher/makeVectorMatcher to make a matcher + // only bundle + // 3 - no clauses support indexes. in this case, we make a matcher only bundle using makeMatcher/makeVectorMatcher + + final List indexOnlyBundles = new ArrayList<>(); + final List indexOnlyBundlesInfo = new ArrayList<>(); + final List partialIndexBundles = new ArrayList<>(); + final List matcherOnlyBundles = new ArrayList<>(); + + int indexUnionSize = 0; + ImmutableBitmap index = null; + ColumnIndexCapabilities merged = new SimpleColumnIndexCapabilities(true, true); + int emptyCount = 0; + + final long bitmapConstructionStartNs = System.nanoTime(); + for (FilterBundle.Builder subFilterBundleBuilder : filterBundleBuilder.getChildBuilders()) { + final FilterBundle bundle = subFilterBundleBuilder.build( + bitmapResultFactory, + Math.min(applyRowCount, totalRowCount - indexUnionSize), + totalRowCount, + includeUnknown + ); + if (bundle.hasIndex()) { + final ImmutableBitmap bundleIndex = bundle.getIndex().getBitmap(); + if (bundleIndex.isEmpty()) { + // we leave any indexes which are empty out of index, indexOnlyBundles, and partialIndexBundles + // even though we skip them, we still keep track of them to check for the case when we can build the OR into + // an index only bundle. We can count index and matcher bundles here too because the AND operation means that + // an empty index means the matcher can be skipped + emptyCount++; + } else { + if (bundle.hasMatcher()) { + // index and matcher bundles must be handled separately, they will need to be a single value matcher built + // by doing an AND operation between the index and the value matcher + // (a bundle is basically an AND operation between the index and matcher if the matcher is present) + partialIndexBundles.add(convertBundleToMatcherOnlyBundle(bundle, bundleIndex)); + } else { + indexOnlyBundles.add(bundle.getIndex()); + indexOnlyBundlesInfo.add(bundle.getIndex().getIndexInfo()); + merged.merge(bundle.getIndex().getIndexCapabilities()); + // union index only bitmaps together; if all sub-filters are 'index only' bundles we will make an index only + // bundle ourselves, else we will use this index as a single value matcher + if (index == null) { + index = bundle.getIndex().getBitmap(); + } else { + index = index.union(bundle.getIndex().getBitmap()); + } + indexUnionSize = index.size(); + } + } + } else { + matcherOnlyBundles.add(bundle.getMatcherBundle()); + } + } + final long totalBitmapConstructTimeNs = System.nanoTime() - bitmapConstructionStartNs; + + + // if all the filters are 'index only', we can make an index only bundle + if (indexOnlyBundles.size() + emptyCount == filters.size()) { + if (index == null || index.isEmpty()) { + return FilterBundle.allFalse( + totalBitmapConstructTimeNs, + filterBundleBuilder.getColumnIndexSelector().getBitmapFactory().makeEmptyImmutableBitmap() + ); + } + if (indexOnlyBundles.size() == 1) { + return new FilterBundle( + indexOnlyBundles.get(0), + null + ); + } + return new FilterBundle( + new FilterBundle.SimpleIndexBundle( + new FilterBundle.IndexBundleInfo( + () -> "OR", + applyRowCount, + totalBitmapConstructTimeNs, + indexOnlyBundlesInfo + ), + index, + merged + ), + null + ); + } + + // if not the index only outcome, we build a matcher only bundle from all the matchers + final int estimatedSize = (indexOnlyBundles.isEmpty() ? 0 : 1) + + partialIndexBundles.size() + + matcherOnlyBundles.size(); + final List allMatcherBundles = Lists.newArrayListWithCapacity(estimatedSize); + final List allMatcherBundlesInfo = Lists.newArrayListWithCapacity(estimatedSize); + if (!indexOnlyBundles.isEmpty()) { + // translate the indexOnly bundles into a single matcher + final FilterBundle.MatcherBundle matcherBundle = convertIndexToMatcherBundle( + applyRowCount, + indexOnlyBundles, + indexOnlyBundlesInfo, + totalBitmapConstructTimeNs, + index + ); + allMatcherBundles.add(matcherBundle); + allMatcherBundlesInfo.add(matcherBundle.getMatcherInfo()); + } + for (FilterBundle.MatcherBundle bundle : partialIndexBundles) { + allMatcherBundles.add(bundle); + allMatcherBundlesInfo.add(bundle.getMatcherInfo()); + } + for (FilterBundle.MatcherBundle bundle : matcherOnlyBundles) { + allMatcherBundles.add(bundle); + allMatcherBundlesInfo.add(bundle.getMatcherInfo()); + } + + return new FilterBundle( + null, + new FilterBundle.MatcherBundle() + { + @Override + public FilterBundle.MatcherBundleInfo getMatcherInfo() + { + return new FilterBundle.MatcherBundleInfo( + () -> "OR", + null, + allMatcherBundlesInfo + ); + } + + @Override + public ValueMatcher valueMatcher(ColumnSelectorFactory selectorFactory, Offset baseOffset, boolean descending) + { + final ValueMatcher[] matchers = new ValueMatcher[allMatcherBundles.size()]; + for (int i = 0; i < allMatcherBundles.size(); i++) { + matchers[i] = allMatcherBundles.get(i).valueMatcher(selectorFactory, baseOffset, descending); + } + return makeMatcher(matchers); + } + + @Override + public VectorValueMatcher vectorMatcher( + VectorColumnSelectorFactory selectorFactory, + ReadableVectorOffset baseOffset + ) + { + final VectorValueMatcher[] matchers = new VectorValueMatcher[allMatcherBundles.size()]; + for (int i = 0; i < allMatcherBundles.size(); i++) { + matchers[i] = allMatcherBundles.get(i).vectorMatcher(selectorFactory, baseOffset); + } + return makeVectorMatcher(matchers); + } + + @Override + public boolean canVectorize() + { + for (FilterBundle.MatcherBundle bundle : allMatcherBundles) { + if (!bundle.canVectorize()) { + return false; + } + } + return true; + } + } + ); + } + + @Nullable + @Override + public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) + { + if (filters.size() == 1) { + return Iterables.getOnlyElement(filters).getBitmapColumnIndex(selector); + } + + List bitmapColumnIndices = new ArrayList<>(filters.size()); + ColumnIndexCapabilities merged = new SimpleColumnIndexCapabilities(true, true); + for (Filter filter : filters) { + BitmapColumnIndex index = filter.getBitmapColumnIndex(selector); + if (index == null) { + // all or nothing + return null; + } + merged = merged.merge(index.getIndexCapabilities()); + bitmapColumnIndices.add(index); + } + + final ColumnIndexCapabilities finalMerged = merged; + return new BitmapColumnIndex() + { + @Override + public ColumnIndexCapabilities getIndexCapabilities() + { + return finalMerged; + } + + @Override + public int estimatedComputeCost() + { + // There's no additional cost on OR filter, cost in child filters would be summed. + return 0; + } + + @Override + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) + { + return bitmapResultFactory.union( + () -> bitmapColumnIndices.stream() + .map(x -> x.computeBitmapResult(bitmapResultFactory, includeUnknown)) + .iterator() + ); + } + + @Nullable + @Override + public T computeBitmapResult( + BitmapResultFactory bitmapResultFactory, + int applyRowCount, + int totalRowCount, + boolean includeUnknown + ) + { + List results = Lists.newArrayListWithCapacity(bitmapColumnIndices.size()); + for (BitmapColumnIndex index : bitmapColumnIndices) { + final T r = index.computeBitmapResult(bitmapResultFactory, applyRowCount, totalRowCount, includeUnknown); + if (r == null) { + // all or nothing + return null; + } + results.add(r); + } + return bitmapResultFactory.union(results); + } + }; + } + + @Override + public ValueMatcher makeMatcher(ColumnSelectorFactory factory) + { + final ValueMatcher[] matchers = new ValueMatcher[filters.size()]; + + int i = 0; + for (Filter filter : filters) { + matchers[i++] = filter.makeMatcher(factory); + } + return makeMatcher(matchers); + } + + @Override + public VectorValueMatcher makeVectorMatcher(final VectorColumnSelectorFactory factory) + { + final VectorValueMatcher[] matchers = new VectorValueMatcher[filters.size()]; + + int i = 0; + for (Filter filter : filters) { + matchers[i++] = filter.makeVectorMatcher(factory); + } + return makeVectorMatcher(matchers); + } + + @Override + public boolean canVectorizeMatcher(ColumnInspector inspector) + { + return filters.stream().allMatch(filter -> filter.canVectorizeMatcher(inspector)); + } + + @Override + public LinkedHashSet getFilters() + { + return filters; + } + + @Override + public boolean supportsRequiredColumnRewrite() + { + for (Filter filter : filters) { + if (!filter.supportsRequiredColumnRewrite()) { + return false; + } + } + + return true; + } + + @Override + public Filter rewriteRequiredColumns(Map columnRewrites) + { + final List newFilters = new ArrayList<>(filters.size()); + for (Filter filter : filters) { + newFilters.add(filter.rewriteRequiredColumns(columnRewrites)); + } + return new OrFilter(newFilters); + } + + @Override + public String toString() + { + return StringUtils.format("(%s)", OR_JOINER.join(filters)); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + OrFilter orFilter = (OrFilter) o; + return Objects.equals(getFilters(), orFilter.getFilters()); + } + + @Override + public int hashCode() + { + return Objects.hash(getFilters()); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/FactsHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/FactsHolder.java new file mode 100644 index 00000000000..f7eede10150 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/FactsHolder.java @@ -0,0 +1,82 @@ +/* + * 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.segment.incremental; + +import java.util.Comparator; +import java.util.Iterator; + +/** + * {@link IncrementalIndexRow} storage interface, a mutable data structure for building up a set or rows to eventually + * persist into an immutable segment + * + * @see IncrementalIndex for the data processor which constructs {@link IncrementalIndexRow} to store here + */ +public interface FactsHolder +{ + /** + * @return the previous rowIndex associated with the specified key, or + * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. + */ + int getPriorIndex(IncrementalIndexRow key); + + /** + * Get minimum {@link IncrementalIndexRow#getTimestamp()} present in the facts holder + */ + long getMinTimeMillis(); + + /** + * Get maximum {@link IncrementalIndexRow#getTimestamp()} present in the facts holder + */ + long getMaxTimeMillis(); + + /** + * Get all {@link IncrementalIndex}, depending on the implementation, these rows may or may not be ordered in the same + * order they will be persisted in. Use {@link #persistIterable()} if this is required. + */ + Iterator iterator(boolean descending); + + /** + * Get all {@link IncrementalIndexRow} with {@link IncrementalIndexRow#getTimestamp()} between the start and end + * timestamps specified + */ + Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd); + + /** + * Get all row {@link IncrementalIndexRow} 'keys', which is distinct groups if this is an aggregating facts holder or + * just every row present if not + */ + Iterable keySet(); + + /** + * Get all {@link IncrementalIndexRow} to persist, ordered with {@link Comparator } + */ + Iterable persistIterable(); + + /** + * @return the previous rowIndex associated with the specified key, or + * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. + */ + int putIfAbsent(IncrementalIndexRow key, int rowIndex); + + /** + * Clear all rows present in the facts holder + */ + void clear(); +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java index cfcc99c10e6..8adc47f6533 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java @@ -98,14 +98,14 @@ import java.util.stream.Collectors; /** * In-memory, row-based data structure used to hold data during ingestion. Realtime tasks query this index using - * {@link IncrementalIndexStorageAdapter}. + * {@link IncrementalIndexCursorFactory}. * * Concurrency model: {@link #add(InputRow)} and {@link #add(InputRow, boolean)} are not thread-safe, and must be * called from a single thread or externally synchronized. However, the methods that support - * {@link IncrementalIndexStorageAdapter} are thread-safe, and may be called concurrently with each other, and with + * {@link IncrementalIndexCursorFactory} are thread-safe, and may be called concurrently with each other, and with * the "add" methods. This concurrency model supports real-time queries of the data in the index. */ -public abstract class IncrementalIndex implements Iterable, Closeable, ColumnInspector +public abstract class IncrementalIndex implements IncrementalIndexRowSelector, ColumnInspector, Iterable, Closeable { /** * Column selector used at ingestion time for inputs to aggregators. @@ -255,8 +255,9 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu private final boolean useSchemaDiscovery; - private final InputRowHolder inputRowHolder = new InputRowHolder(); + protected final InputRowHolder inputRowHolder = new InputRowHolder(); + @Nullable private volatile DateTime maxIngestedEventTime; /** @@ -366,8 +367,6 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu ); } - public abstract FactsHolder getFacts(); - public abstract boolean canAppendRow(); public abstract String getOutOfRowsReason(); @@ -384,100 +383,11 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu boolean skipMaxRowsInMemoryCheck ) throws IndexSizeExceededException; - public abstract int getLastRowIndex(); - protected abstract float getMetricFloatValue(int rowOffset, int aggOffset); - - protected abstract long getMetricLongValue(int rowOffset, int aggOffset); - - protected abstract Object getMetricObjectValue(int rowOffset, int aggOffset); - - protected abstract double getMetricDoubleValue(int rowOffset, int aggOffset); - - protected abstract boolean isNull(int rowOffset, int aggOffset); - - static class IncrementalIndexRowResult - { - private final IncrementalIndexRow incrementalIndexRow; - private final List parseExceptionMessages; - - IncrementalIndexRowResult(IncrementalIndexRow incrementalIndexRow, List parseExceptionMessages) - { - this.incrementalIndexRow = incrementalIndexRow; - this.parseExceptionMessages = parseExceptionMessages; - } - - IncrementalIndexRow getIncrementalIndexRow() - { - return incrementalIndexRow; - } - - List getParseExceptionMessages() - { - return parseExceptionMessages; - } - } - - static class AddToFactsResult - { - private final int rowCount; - private final long bytesInMemory; - private final List parseExceptionMessages; - - public AddToFactsResult( - int rowCount, - long bytesInMemory, - List parseExceptionMessages - ) - { - this.rowCount = rowCount; - this.bytesInMemory = bytesInMemory; - this.parseExceptionMessages = parseExceptionMessages; - } - - int getRowCount() - { - return rowCount; - } - - public long getBytesInMemory() - { - return bytesInMemory; - } - - public List getParseExceptionMessages() - { - return parseExceptionMessages; - } - } - - public static class InputRowHolder - { - @Nullable - private InputRow row; - private long rowId = -1; - - public void set(final InputRow row) - { - this.row = row; - this.rowId++; - } - - public void unset() - { - this.row = null; - } - - public InputRow getRow() - { - return Preconditions.checkNotNull(row, "row"); - } - - public long getRowId() - { - return rowId; - } - } + public abstract Iterable iterableWithPostAggregations( + @Nullable List postAggs, + boolean descending + ); public boolean isRollup() { @@ -746,23 +656,6 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu ); } - private static String getSimplifiedEventStringFromRow(InputRow inputRow) - { - if (inputRow instanceof MapBasedInputRow) { - return ((MapBasedInputRow) inputRow).getEvent().toString(); - } - - if (inputRow instanceof ListBasedInputRow) { - return ((ListBasedInputRow) inputRow).asMap().toString(); - } - - if (inputRow instanceof TransformedInputRow) { - InputRow innerRow = ((TransformedInputRow) inputRow).getBaseRow(); - return getSimplifiedEventStringFromRow(innerRow); - } - - return inputRow.toString(); - } private synchronized void updateMaxIngestedTime(DateTime eventTime) { @@ -771,6 +664,7 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu } } + @Override public boolean isEmpty() { return numEntries.get() == 0; @@ -861,6 +755,7 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu /** * Returns the descriptor for a particular dimension. */ + @Override @Nullable public DimensionDesc getDimension(String dimension) { @@ -869,22 +764,39 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu } } - public ColumnValueSelector makeMetricColumnValueSelector(String metric, IncrementalIndexRowHolder currEntry) + @Override + @Nullable + public MetricDesc getMetric(String metric) { - MetricDesc metricDesc = metricDescs.get(metric); + return metricDescs.get(metric); + } + + @Override + public List getOrdering() + { + return metadata.getOrdering(); + } + + public static ColumnValueSelector makeMetricColumnValueSelector( + IncrementalIndexRowSelector rowSelector, + IncrementalIndexRowHolder currEntry, + String metric + ) + { + final MetricDesc metricDesc = rowSelector.getMetric(metric); if (metricDesc == null) { return NilColumnValueSelector.instance(); } int metricIndex = metricDesc.getIndex(); switch (metricDesc.getCapabilities().getType()) { case COMPLEX: - return new ObjectMetricColumnSelector(metricDesc, currEntry, metricIndex); + return new ObjectMetricColumnSelector(rowSelector, currEntry, metricDesc); case LONG: - return new LongMetricColumnSelector(currEntry, metricIndex); + return new LongMetricColumnSelector(rowSelector, currEntry, metricIndex); case FLOAT: - return new FloatMetricColumnSelector(currEntry, metricIndex); + return new FloatMetricColumnSelector(rowSelector, currEntry, metricIndex); case DOUBLE: - return new DoubleMetricColumnSelector(currEntry, metricIndex); + return new DoubleMetricColumnSelector(rowSelector, currEntry, metricIndex); case STRING: throw new IllegalStateException("String is not a metric column type"); default: @@ -910,13 +822,6 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu return isEmpty() ? null : DateTimes.utc(getMaxTimeMillis()); } - @Nullable - public Integer getDimensionIndex(String dimension) - { - DimensionDesc dimSpec = getDimension(dimension); - return dimSpec == null ? null : dimSpec.getIndex(); - } - /** * Returns names of time and dimension columns, in persist sort order. Includes {@link ColumnHolder#TIME_COLUMN_NAME}. */ @@ -1003,6 +908,49 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu return metadata; } + @Override + public Iterator iterator() + { + return iterableWithPostAggregations(null, false).iterator(); + } + + public DateTime getMaxIngestedEventTime() + { + return maxIngestedEventTime; + } + + protected ColumnSelectorFactory makeColumnSelectorFactory( + @Nullable final AggregatorFactory agg, + final InputRowHolder in + ) + { + return makeColumnSelectorFactory(virtualColumns, in, agg); + } + + protected final Comparator dimsComparator() + { + return new IncrementalIndexRowComparator(timePosition, dimensionDescsList); + } + + + private static String getSimplifiedEventStringFromRow(InputRow inputRow) + { + if (inputRow instanceof MapBasedInputRow) { + return ((MapBasedInputRow) inputRow).getEvent().toString(); + } + + if (inputRow instanceof ListBasedInputRow) { + return ((ListBasedInputRow) inputRow).asMap().toString(); + } + + if (inputRow instanceof TransformedInputRow) { + InputRow innerRow = ((TransformedInputRow) inputRow).getBaseRow(); + return getSimplifiedEventStringFromRow(innerRow); + } + + return inputRow.toString(); + } + private static AggregatorFactory[] getCombiningAggregators(AggregatorFactory[] aggregators) { AggregatorFactory[] combiningAggregators = new AggregatorFactory[aggregators.length]; @@ -1012,30 +960,24 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu return combiningAggregators; } - @Override - public Iterator iterator() + private static boolean allNull(Object[] dims, int startPosition) { - return iterableWithPostAggregations(null, false).iterator(); - } - - public abstract Iterable iterableWithPostAggregations( - @Nullable List postAggs, - boolean descending - ); - - public DateTime getMaxIngestedEventTime() - { - return maxIngestedEventTime; + for (int i = startPosition; i < dims.length; i++) { + if (dims[i] != null) { + return false; + } + } + return true; } public static final class DimensionDesc { private final int index; private final String name; - private final DimensionHandler handler; - private final DimensionIndexer indexer; + private final DimensionHandler handler; + private final DimensionIndexer indexer; - public DimensionDesc(int index, String name, DimensionHandler handler, boolean useMaxMemoryEstimates) + public DimensionDesc(int index, String name, DimensionHandler handler, boolean useMaxMemoryEstimates) { this.index = index; this.name = name; @@ -1058,12 +1000,12 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu return indexer.getColumnCapabilities(); } - public DimensionHandler getHandler() + public DimensionHandler getHandler() { return handler; } - public DimensionIndexer getIndexer() + public DimensionIndexer getIndexer() { return indexer; } @@ -1124,19 +1066,90 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu } } - protected ColumnSelectorFactory makeColumnSelectorFactory( - @Nullable final AggregatorFactory agg, - final InputRowHolder in - ) + public static class AddToFactsResult { - return makeColumnSelectorFactory(virtualColumns, in, agg); + private final int rowCount; + private final long bytesInMemory; + private final List parseExceptionMessages; + + public AddToFactsResult( + int rowCount, + long bytesInMemory, + List parseExceptionMessages + ) + { + this.rowCount = rowCount; + this.bytesInMemory = bytesInMemory; + this.parseExceptionMessages = parseExceptionMessages; + } + + int getRowCount() + { + return rowCount; + } + + public long getBytesInMemory() + { + return bytesInMemory; + } + + public List getParseExceptionMessages() + { + return parseExceptionMessages; + } } - protected final Comparator dimsComparator() + public static class InputRowHolder { - return new IncrementalIndexRowComparator(timePosition, dimensionDescsList); + @Nullable + private InputRow row; + private long rowId = -1; + + public void set(final InputRow row) + { + this.row = row; + this.rowId++; + } + + public void unset() + { + this.row = null; + } + + public InputRow getRow() + { + return Preconditions.checkNotNull(row, "row"); + } + + public long getRowId() + { + return rowId; + } } + static class IncrementalIndexRowResult + { + private final IncrementalIndexRow incrementalIndexRow; + private final List parseExceptionMessages; + + IncrementalIndexRowResult(IncrementalIndexRow incrementalIndexRow, List parseExceptionMessages) + { + this.incrementalIndexRow = incrementalIndexRow; + this.parseExceptionMessages = parseExceptionMessages; + } + + IncrementalIndexRow getIncrementalIndexRow() + { + return incrementalIndexRow; + } + + List getParseExceptionMessages() + { + return parseExceptionMessages; + } + } + + @VisibleForTesting static final class IncrementalIndexRowComparator implements Comparator { @@ -1207,57 +1220,19 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu } } - private static boolean allNull(Object[] dims, int startPosition) - { - for (int i = startPosition; i < dims.length; i++) { - if (dims[i] != null) { - return false; - } - } - return true; - } - - public interface FactsHolder - { - /** - * @return the previous rowIndex associated with the specified key, or - * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. - */ - int getPriorIndex(IncrementalIndexRow key); - - long getMinTimeMillis(); - - long getMaxTimeMillis(); - - Iterator iterator(boolean descending); - - Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd); - - Iterable keySet(); - - /** - * Get all {@link IncrementalIndexRow} to persist, ordered with {@link Comparator} - * - * @return - */ - Iterable persistIterable(); - - /** - * @return the previous rowIndex associated with the specified key, or - * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. - */ - int putIfAbsent(IncrementalIndexRow key, int rowIndex); - - void clear(); - } - - private final class LongMetricColumnSelector implements LongColumnSelector + private static final class LongMetricColumnSelector implements LongColumnSelector { + private final IncrementalIndexRowSelector rowSelector; private final IncrementalIndexRowHolder currEntry; private final int metricIndex; - public LongMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) + public LongMetricColumnSelector( + IncrementalIndexRowSelector rowSelector, + IncrementalIndexRowHolder currEntry, + int metricIndex + ) { + this.rowSelector = rowSelector; this.currEntry = currEntry; this.metricIndex = metricIndex; } @@ -1265,99 +1240,72 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu @Override public long getLong() { - assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricLongValue(currEntry.get().getRowIndex(), metricIndex); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("index", IncrementalIndex.this); + return rowSelector.getMetricLongValue(currEntry.get().getRowIndex(), metricIndex); } @Override public boolean isNull() { - return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); - } - } - - private final class ObjectMetricColumnSelector extends ObjectColumnSelector - { - private final IncrementalIndexRowHolder currEntry; - private final int metricIndex; - private Class classOfObject; - - public ObjectMetricColumnSelector( - MetricDesc metricDesc, - IncrementalIndexRowHolder currEntry, - int metricIndex - ) - { - this.currEntry = currEntry; - this.metricIndex = metricIndex; - classOfObject = ComplexMetrics.getSerdeForType(metricDesc.getType()).getObjectStrategy().getClazz(); - } - - @Nullable - @Override - public Object getObject() - { - return getMetricObjectValue(currEntry.get().getRowIndex(), metricIndex); - } - - @Override - public Class classOfObject() - { - return classOfObject; + return rowSelector.isNull(currEntry.get().getRowIndex(), metricIndex); } @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - inspector.visit("index", IncrementalIndex.this); + inspector.visit("index", rowSelector); } } - private final class FloatMetricColumnSelector implements FloatColumnSelector + private static final class FloatMetricColumnSelector implements FloatColumnSelector { + private final IncrementalIndexRowSelector rowSelector; private final IncrementalIndexRowHolder currEntry; private final int metricIndex; - public FloatMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) + public FloatMetricColumnSelector( + IncrementalIndexRowSelector rowSelector, + IncrementalIndexRowHolder currEntry, + int metricIndex + ) { this.currEntry = currEntry; + this.rowSelector = rowSelector; this.metricIndex = metricIndex; } @Override public float getFloat() { - assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricFloatValue(currEntry.get().getRowIndex(), metricIndex); + return rowSelector.getMetricFloatValue(currEntry.get().getRowIndex(), metricIndex); } @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - inspector.visit("index", IncrementalIndex.this); + inspector.visit("index", rowSelector); } @Override public boolean isNull() { - return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); + return rowSelector.isNull(currEntry.get().getRowIndex(), metricIndex); } } - private final class DoubleMetricColumnSelector implements DoubleColumnSelector + private static final class DoubleMetricColumnSelector implements DoubleColumnSelector { + private final IncrementalIndexRowSelector rowSelector; private final IncrementalIndexRowHolder currEntry; private final int metricIndex; - public DoubleMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) + public DoubleMetricColumnSelector( + IncrementalIndexRowSelector rowSelector, + IncrementalIndexRowHolder currEntry, + int metricIndex + ) { this.currEntry = currEntry; + this.rowSelector = rowSelector; this.metricIndex = metricIndex; } @@ -1365,19 +1313,58 @@ public abstract class IncrementalIndex implements Iterable, Closeable, Colu public double getDouble() { assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricDoubleValue(currEntry.get().getRowIndex(), metricIndex); + return rowSelector.getMetricDoubleValue(currEntry.get().getRowIndex(), metricIndex); } @Override public boolean isNull() { - return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); + return rowSelector.isNull(currEntry.get().getRowIndex(), metricIndex); } @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - inspector.visit("index", IncrementalIndex.this); + inspector.visit("index", rowSelector); + } + } + + private static final class ObjectMetricColumnSelector extends ObjectColumnSelector + { + private final IncrementalIndexRowSelector rowSelector; + private final IncrementalIndexRowHolder currEntry; + private final int metricIndex; + private final Class classOfObject; + + public ObjectMetricColumnSelector( + IncrementalIndexRowSelector rowSelector, + IncrementalIndexRowHolder currEntry, + MetricDesc metricDesc + ) + { + this.currEntry = currEntry; + this.rowSelector = rowSelector; + this.metricIndex = metricDesc.getIndex(); + this.classOfObject = ComplexMetrics.getSerdeForType(metricDesc.getType()).getObjectStrategy().getClazz(); + } + + @Nullable + @Override + public Object getObject() + { + return rowSelector.getMetricObjectValue(currEntry.get().getRowIndex(), metricIndex); + } + + @Override + public Class classOfObject() + { + return classOfObject; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("index", rowSelector); } } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java index f43b81c40ad..9d60edef044 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java @@ -22,6 +22,7 @@ package org.apache.druid.segment.incremental; import org.apache.druid.query.Order; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionIndexer; @@ -41,24 +42,32 @@ import javax.annotation.Nullable; */ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, RowIdSupplier { - private final IncrementalIndexStorageAdapter adapter; - private final IncrementalIndex index; + private final ColumnInspector snapshotColumnInspector; private final VirtualColumns virtualColumns; private final Order timeOrder; private final IncrementalIndexRowHolder rowHolder; + private final IncrementalIndexRowSelector rowSelector; IncrementalIndexColumnSelectorFactory( - IncrementalIndexStorageAdapter adapter, + IncrementalIndexRowSelector rowSelector, VirtualColumns virtualColumns, Order timeOrder, IncrementalIndexRowHolder rowHolder ) { - this.adapter = adapter; - this.index = adapter.index; this.virtualColumns = virtualColumns; this.timeOrder = timeOrder; this.rowHolder = rowHolder; + this.rowSelector = rowSelector; + this.snapshotColumnInspector = new ColumnInspector() + { + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + return IncrementalIndexCursorFactory.snapshotColumnCapabilities(rowSelector, column); + } + }; } @Override @@ -78,13 +87,13 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME) && timeOrder != Order.NONE) { return new SingleScanTimeDimensionSelector( - makeColumnValueSelector(dimension), + makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), extractionFn, timeOrder ); } - final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimensionSpec.getDimension()); + final IncrementalIndex.DimensionDesc dimensionDesc = rowSelector.getDimension(dimensionSpec.getDimension()); if (dimensionDesc == null) { // not a dimension, column may be a metric ColumnCapabilities capabilities = getColumnCapabilities(dimension); @@ -113,27 +122,25 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro if (virtualColumns.exists(columnName)) { return virtualColumns.makeColumnValueSelector(columnName, this); } - - if (columnName.equals(ColumnHolder.TIME_COLUMN_NAME)) { + if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) { return rowHolder; } - final Integer dimIndex = index.getDimensionIndex(columnName); - if (dimIndex != null) { - final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(columnName); + final IncrementalIndex.DimensionDesc dimensionDesc = rowSelector.getDimension(columnName); + if (dimensionDesc != null) { final DimensionIndexer indexer = dimensionDesc.getIndexer(); return indexer.makeColumnValueSelector(rowHolder, dimensionDesc); } - return index.makeMetricColumnValueSelector(columnName, rowHolder); + return IncrementalIndex.makeMetricColumnValueSelector(rowSelector, rowHolder, columnName); } @Override @Nullable public ColumnCapabilities getColumnCapabilities(String columnName) { - // Use adapter.getColumnCapabilities instead of index.getCapabilities (see note in IncrementalIndexStorageAdapater) - return virtualColumns.getColumnCapabilitiesWithFallback(adapter, columnName); + // Use snapshotColumnInspector instead of index.getCapabilities (see note in IncrementalIndexStorageAdapater) + return virtualColumns.getColumnCapabilitiesWithFallback(snapshotColumnInspector, columnName); } @Nullable diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java similarity index 50% rename from processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java rename to processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java index c9a6d209697..b73a7b682a3 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java @@ -21,29 +21,19 @@ package org.apache.druid.segment.incremental; import com.google.common.collect.Iterables; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.DimensionDictionarySelector; -import org.apache.druid.segment.DimensionIndexer; -import org.apache.druid.segment.Metadata; import org.apache.druid.segment.NestedDataColumnIndexerV4; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.data.Indexed; -import org.apache.druid.segment.data.ListIndexed; -import org.joda.time.Interval; import javax.annotation.Nullable; -/** - * - */ -public class IncrementalIndexStorageAdapter implements StorageAdapter +public class IncrementalIndexCursorFactory implements CursorFactory { - private static final ColumnCapabilities.CoercionLogic STORAGE_ADAPTER_CAPABILITIES_COERCE_LOGIC = + private static final ColumnCapabilities.CoercionLogic COERCE_LOGIC = new ColumnCapabilities.CoercionLogic() { @Override @@ -77,51 +67,17 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter } }; - private static final ColumnCapabilities.CoercionLogic SNAPSHOT_STORAGE_ADAPTER_CAPABILITIES_COERCE_LOGIC = - new ColumnCapabilities.CoercionLogic() - { - @Override - public boolean dictionaryEncoded() - { - return true; - } + private final IncrementalIndex index; - @Override - public boolean dictionaryValuesSorted() - { - return true; - } - - @Override - public boolean dictionaryValuesUnique() - { - return true; - } - - @Override - public boolean multipleValues() - { - return false; - } - - @Override - public boolean hasNulls() - { - return false; - } - }; - - final IncrementalIndex index; - - public IncrementalIndexStorageAdapter(IncrementalIndex index) + public IncrementalIndexCursorFactory(IncrementalIndex index) { this.index = index; } @Override - public Interval getInterval() + public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - return index.getInterval(); + return new IncrementalIndexCursorHolder(index, spec); } @Override @@ -136,69 +92,16 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter return builder.build(); } - @Override - public Indexed getAvailableDimensions() - { - return new ListIndexed<>(index.getDimensionNames(false)); - } - - @Override - public Iterable getAvailableMetrics() - { - return index.getMetricNames(); - } - - @Override - public int getDimensionCardinality(String dimension) - { - if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME)) { - return DimensionDictionarySelector.CARDINALITY_UNKNOWN; - } - - IncrementalIndex.DimensionDesc desc = index.getDimension(dimension); - if (desc == null) { - return 0; - } - - return desc.getIndexer().getCardinality(); - } - - @Override - public int getNumRows() - { - return index.size(); - } - @Nullable - @Override - public Comparable getMinValue(String column) - { - IncrementalIndex.DimensionDesc desc = index.getDimension(column); - if (desc == null) { - return null; - } - - DimensionIndexer indexer = desc.getIndexer(); - return indexer.getMinValue(); - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - IncrementalIndex.DimensionDesc desc = index.getDimension(column); - if (desc == null) { - return null; - } - - DimensionIndexer indexer = desc.getIndexer(); - return indexer.getMaxValue(); - } - @Override public ColumnCapabilities getColumnCapabilities(String column) { - IncrementalIndex.DimensionDesc desc = index.getDimension(column); + return snapshotColumnCapabilities(index, column); + } + + static ColumnCapabilities snapshotColumnCapabilities(IncrementalIndexRowSelector selector, String column) + { + IncrementalIndex.DimensionDesc desc = selector.getDimension(column); // nested column indexer is a liar, and behaves like any type if it only processes unnested literals of a single // type, so force it to use nested column type if (desc != null && desc.getIndexer() instanceof NestedDataColumnIndexerV4) { @@ -211,7 +114,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter // // We don't want to represent this as having-multiple-values in index.getCapabilities, because that's used // at index-persisting time to determine if we need a multi-value column or not. However, that means we - // need to tweak the capabilities here in the StorageAdapter (a query-time construct), so at query time + // need to tweak the capabilities here in the CursorFactory (a query-time construct), so at query time // they appear multi-valued. // // Note that this could be improved if we snapshot the capabilities at cursor creation time and feed those through @@ -219,34 +122,8 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter // multi-valuedness at cursor creation time, instead of the latest state, and getSnapshotColumnCapabilities could // be removed. return ColumnCapabilitiesImpl.snapshot( - index.getColumnCapabilities(column), - STORAGE_ADAPTER_CAPABILITIES_COERCE_LOGIC + selector.getColumnCapabilities(column), + COERCE_LOGIC ); } - - /** - * Sad workaround for {@link org.apache.druid.query.metadata.SegmentAnalyzer} to deal with the fact that the - * response from {@link #getColumnCapabilities} is not accurate for string columns, in that it reports all string - * columns as having multiple values. This method returns the actual capabilities of the underlying - * {@link IncrementalIndex} at the time this method is called. - */ - public ColumnCapabilities getSnapshotColumnCapabilities(String column) - { - return ColumnCapabilitiesImpl.snapshot( - index.getColumnCapabilities(column), - SNAPSHOT_STORAGE_ADAPTER_CAPABILITIES_COERCE_LOGIC - ); - } - - @Override - public CursorHolder makeCursorHolder(CursorBuildSpec spec) - { - return new IncrementalIndexCursorHolder(this, index, spec); - } - - @Override - public Metadata getMetadata() - { - return index.getMetadata(); - } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java index 33c08d5920c..72ec9116d1f 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java @@ -23,55 +23,46 @@ import com.google.common.collect.Iterators; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.Order; import org.apache.druid.query.OrderBy; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.filter.ValueMatchers; -import org.joda.time.Interval; -import javax.annotation.Nullable; -import java.util.Collections; import java.util.Iterator; import java.util.List; public class IncrementalIndexCursorHolder implements CursorHolder { - private final IncrementalIndexStorageAdapter storageAdapter; - private final IncrementalIndex index; + private final IncrementalIndexRowSelector rowSelector; private final CursorBuildSpec spec; private final List ordering; public IncrementalIndexCursorHolder( - IncrementalIndexStorageAdapter storageAdapter, - IncrementalIndex index, + IncrementalIndexRowSelector rowSelector, CursorBuildSpec spec ) { - this.storageAdapter = storageAdapter; - this.index = index; + this.rowSelector = rowSelector; this.spec = spec; - if (index.timePosition == 0) { + List ordering = rowSelector.getOrdering(); + if (Cursors.getTimeOrdering(ordering) != Order.NONE) { if (Cursors.preferDescendingTimeOrdering(spec)) { this.ordering = Cursors.descendingTimeOrder(); } else { this.ordering = Cursors.ascendingTimeOrder(); } } else { - // In principle, we could report a sort order here for certain types of fact holders; for example the - // RollupFactsHolder would be sorted by dimensions. However, this is left for future work. - this.ordering = Collections.emptyList(); + this.ordering = ordering; } } @Override public Cursor asCursor() { - if (index.isEmpty()) { + if (rowSelector.isEmpty()) { return null; } @@ -79,14 +70,10 @@ public class IncrementalIndexCursorHolder implements CursorHolder spec.getQueryMetrics().vectorized(false); } - return new IncrementalIndexCursor( - storageAdapter, - index, - spec.getVirtualColumns(), - Cursors.getTimeOrdering(ordering), - spec.getFilter(), - spec.getInterval() + rowSelector, + spec, + Cursors.getTimeOrdering(ordering) ); } @@ -98,11 +85,11 @@ public class IncrementalIndexCursorHolder implements CursorHolder static class IncrementalIndexCursor implements Cursor { - private IncrementalIndexRowHolder currEntry; + private final IncrementalIndexRowSelector rowSelector; + private final IncrementalIndexRowHolder currEntry; private final ColumnSelectorFactory columnSelectorFactory; private final ValueMatcher filterMatcher; private final int maxRowIndex; - private final IncrementalIndex.FactsHolder facts; private Iterator baseIter; private Iterable cursorIterable; private boolean emptyRange; @@ -110,31 +97,31 @@ public class IncrementalIndexCursorHolder implements CursorHolder private boolean done; IncrementalIndexCursor( - IncrementalIndexStorageAdapter storageAdapter, - IncrementalIndex index, - VirtualColumns virtualColumns, - Order timeOrder, - @Nullable Filter filter, - Interval actualInterval + IncrementalIndexRowSelector index, + CursorBuildSpec buildSpec, + Order timeOrder ) { currEntry = new IncrementalIndexRowHolder(); + // Set maxRowIndex before creating the filterMatcher. See https://github.com/apache/druid/pull/6340 + maxRowIndex = index.getLastRowIndex(); + numAdvanced = -1; + + rowSelector = index; + cursorIterable = rowSelector.getFacts().timeRangeIterable( + timeOrder == Order.DESCENDING, + buildSpec.getInterval().getStartMillis(), + buildSpec.getInterval().getEndMillis() + ); columnSelectorFactory = new IncrementalIndexColumnSelectorFactory( - storageAdapter, - virtualColumns, + rowSelector, + buildSpec.getVirtualColumns(), timeOrder, currEntry ); - // Set maxRowIndex before creating the filterMatcher. See https://github.com/apache/druid/pull/6340 - maxRowIndex = index.getLastRowIndex(); - filterMatcher = filter == null ? ValueMatchers.allTrue() : filter.makeMatcher(columnSelectorFactory); - numAdvanced = -1; - facts = index.getFacts(); - cursorIterable = facts.timeRangeIterable( - timeOrder == Order.DESCENDING, - actualInterval.getStartMillis(), - actualInterval.getEndMillis() - ); + filterMatcher = buildSpec.getFilter() == null + ? ValueMatchers.allTrue() + : buildSpec.getFilter().makeMatcher(columnSelectorFactory); emptyRange = !cursorIterable.iterator().hasNext(); reset(); @@ -157,7 +144,7 @@ public class IncrementalIndexCursorHolder implements CursorHolder while (baseIter.hasNext()) { BaseQuery.checkInterrupted(); - IncrementalIndexRow entry = baseIter.next(); + final IncrementalIndexRow entry = baseIter.next(); if (beyondMaxRowIndex(entry.getRowIndex())) { continue; } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java new file mode 100644 index 00000000000..c015206a9da --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.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.segment.incremental; + +import org.apache.druid.segment.DimensionDictionarySelector; +import org.apache.druid.segment.DimensionIndexer; +import org.apache.druid.segment.Metadata; +import org.apache.druid.segment.PhysicalSegmentInspector; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnHolder; + +import javax.annotation.Nullable; + +public class IncrementalIndexPhysicalSegmentInspector implements PhysicalSegmentInspector +{ + private static final ColumnCapabilities.CoercionLogic SNAPSHOT_COERCE_LOGIC = + new ColumnCapabilities.CoercionLogic() + { + @Override + public boolean dictionaryEncoded() + { + return true; + } + + @Override + public boolean dictionaryValuesSorted() + { + return true; + } + + @Override + public boolean dictionaryValuesUnique() + { + return true; + } + + @Override + public boolean multipleValues() + { + return false; + } + + @Override + public boolean hasNulls() + { + return false; + } + }; + + private final IncrementalIndex index; + + public IncrementalIndexPhysicalSegmentInspector(IncrementalIndex index) + { + this.index = index; + } + + @Nullable + @Override + public Metadata getMetadata() + { + return index.getMetadata(); + } + + @Nullable + @Override + public Comparable getMinValue(String column) + { + IncrementalIndex.DimensionDesc desc = index.getDimension(column); + if (desc == null) { + return null; + } + + DimensionIndexer indexer = desc.getIndexer(); + return indexer.getMinValue(); + } + + @Nullable + @Override + public Comparable getMaxValue(String column) + { + IncrementalIndex.DimensionDesc desc = index.getDimension(column); + if (desc == null) { + return null; + } + + DimensionIndexer indexer = desc.getIndexer(); + return indexer.getMaxValue(); + } + + @Override + public int getDimensionCardinality(String column) + { + if (column.equals(ColumnHolder.TIME_COLUMN_NAME)) { + return DimensionDictionarySelector.CARDINALITY_UNKNOWN; + } + + IncrementalIndex.DimensionDesc desc = index.getDimension(column); + if (desc == null) { + // non-existent dimension has cardinality = 1 (one null, nothing else). + return 1; + } + + return desc.getIndexer().getCardinality(); + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + return ColumnCapabilitiesImpl.snapshot( + index.getColumnCapabilities(column), + SNAPSHOT_COERCE_LOGIC + ); + } + + @Override + public int getNumRows() + { + return index.size(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java index 89e94961f6b..2e817b993ce 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java @@ -144,6 +144,8 @@ public final class IncrementalIndexRow { if (input == null || (input.getClass().isArray() && Array.getLength(input) == 0)) { return Collections.singletonList("null"); + } else if (input instanceof int[]) { + return Arrays.toString((int[]) input); } return Collections.singletonList(input); } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java index 0e7e82f01c3..15909ac1b52 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java @@ -66,7 +66,7 @@ class IncrementalIndexRowIterator implements TransformableRowIterator { ColumnSelectorFactory columnSelectorFactory = new IncrementalIndexColumnSelectorFactory( - new IncrementalIndexStorageAdapter(incrementalIndex), + incrementalIndex, VirtualColumns.EMPTY, incrementalIndex.timePosition == 0 ? Order.ASCENDING : Order.NONE, rowHolder diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowSelector.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowSelector.java new file mode 100644 index 00000000000..bafa127e881 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowSelector.java @@ -0,0 +1,104 @@ +/* + * 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.segment.incremental; + +import org.apache.druid.query.OrderBy; +import org.apache.druid.segment.ColumnInspector; + +import javax.annotation.Nullable; +import java.util.List; + +/** + * Interface that abstracts selecting data from a {@link FactsHolder} + */ +public interface IncrementalIndexRowSelector extends ColumnInspector +{ + /** + * get {@link IncrementalIndex.DimensionDesc} for the specified column, if available, which provides access to things + * like {@link org.apache.druid.segment.DimensionIndexer} and {@link org.apache.druid.segment.DimensionHandler} as + * well as column capabilities and position within the row + */ + @Nullable + IncrementalIndex.DimensionDesc getDimension(String columnName); + + /** + * Get {@link IncrementalIndex.MetricDesc} which provides column capabilities and position in the aggregators section + * of the row + */ + @Nullable + IncrementalIndex.MetricDesc getMetric(String s); + + /** + * Ordering for the data in the facts table + */ + List getOrdering(); + + /** + * Are there any {@link IncrementalIndexRow} stored in the {@link FactsHolder}? + */ + boolean isEmpty(); + + /** + * Get the {@link FactsHolder} containing all of the {@link IncrementalIndexRow} backing this selector + */ + FactsHolder getFacts(); + + /** + * Highest value {@link IncrementalIndexRow#getRowIndex()} available in this selector. Note that these values do not + * reflect the position of the row in the {@link FactsHolder}, rather just the order in which they were processed + */ + int getLastRowIndex(); + + /** + * @param rowOffset row to get float aggregator value + * @param aggOffset position of the aggregator in the aggregators array of the data schema + * @return float value of the metric + */ + float getMetricFloatValue(int rowOffset, int aggOffset); + + /** + * @param rowOffset row to get long aggregator value + * @param aggOffset position of the aggregator in the aggregators array of the data schema + * @return long value of the aggregator for this row + */ + long getMetricLongValue(int rowOffset, int aggOffset); + + /** + * @param rowOffset row to get double aggregator value + * @param aggOffset position of the aggregator in the aggregators array of the data schema + * @return double value of the aggregator for this row + */ + double getMetricDoubleValue(int rowOffset, int aggOffset); + + /** + * @param rowOffset row to get long aggregator value + * @param aggOffset position of the aggregator in the aggregators array of the data schema + * @return long value of the aggregator for this row + */ + @Nullable + Object getMetricObjectValue(int rowOffset, int aggOffset); + + /** + * @param rowOffset row to check for a aggregator value + * @param aggOffset position of the aggregator in the aggregators array of the data schema + * @return is the value null for this row? + */ + boolean isNull(int rowOffset, int aggOffset); +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java index b5e580f44f2..8c554e016fc 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java @@ -155,7 +155,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex } else { this.facts = new PlainNonTimeOrderedFactsHolder(dimsComparator()); } - maxBytesPerRowForAggregators = + this.maxBytesPerRowForAggregators = useMaxMemoryEstimates ? getMaxBytesPerRowForAggregators(incrementalIndexSchema) : 0; this.useMaxMemoryEstimates = useMaxMemoryEstimates; } @@ -252,14 +252,15 @@ public class OnheapIncrementalIndex extends IncrementalIndex ) throws IndexSizeExceededException { final List parseExceptionMessages = new ArrayList<>(); + final AtomicLong totalSizeInBytes = getBytesInMemory(); + final int priorIndex = facts.getPriorIndex(key); Aggregator[] aggs; final AggregatorFactory[] metrics = getMetrics(); final AtomicInteger numEntries = getNumEntries(); - final AtomicLong totalSizeInBytes = getBytesInMemory(); if (IncrementalIndexRow.EMPTY_ROW_INDEX != priorIndex) { - aggs = concurrentGet(priorIndex); + aggs = aggregators.get(priorIndex); long aggSizeDelta = doAggregate(metrics, aggs, inputRowHolder, parseExceptionMessages); totalSizeInBytes.addAndGet(useMaxMemoryEstimates ? 0 : aggSizeDelta); } else { @@ -272,7 +273,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex aggSizeForRow += doAggregate(metrics, aggs, inputRowHolder, parseExceptionMessages); final int rowIndex = indexIncrement.getAndIncrement(); - concurrentSet(rowIndex, aggs); + aggregators.put(rowIndex, aggs); // Last ditch sanity checks if ((numEntries.get() >= maxRowCount || totalSizeInBytes.get() >= maxBytesInMemory) @@ -363,6 +364,18 @@ public class OnheapIncrementalIndex extends IncrementalIndex InputRowHolder inputRowHolder, List parseExceptionsHolder ) + { + return doAggregate(metrics, aggs, inputRowHolder, parseExceptionsHolder, useMaxMemoryEstimates, preserveExistingMetrics); + } + + private static long doAggregate( + AggregatorFactory[] metrics, + Aggregator[] aggs, + InputRowHolder inputRowHolder, + List parseExceptionsHolder, + boolean useMaxMemoryEstimates, + boolean preserveExistingMetrics + ) { long totalIncrementalBytes = 0L; for (int i = 0; i < metrics.length; i++) { @@ -418,17 +431,6 @@ public class OnheapIncrementalIndex extends IncrementalIndex } } - protected Aggregator[] concurrentGet(int offset) - { - // All get operations should be fine - return aggregators.get(offset); - } - - protected void concurrentSet(int offset, Aggregator[] value) - { - aggregators.put(offset, value); - } - @Override public boolean canAppendRow() { @@ -459,42 +461,53 @@ public class OnheapIncrementalIndex extends IncrementalIndex return outOfRowsReason; } - protected Aggregator[] getAggsForRow(int rowOffset) - { - return concurrentGet(rowOffset); - } - @Override public float getMetricFloatValue(int rowOffset, int aggOffset) { - return ((Number) getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::getFloat)).floatValue(); + return ((Number) getMetricHelper( + getMetricAggs(), + aggregators.get(rowOffset), + aggOffset, + Aggregator::getFloat + )).floatValue(); } @Override public long getMetricLongValue(int rowOffset, int aggOffset) { - return ((Number) getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::getLong)).longValue(); + return ((Number) getMetricHelper( + getMetricAggs(), + aggregators.get(rowOffset), + aggOffset, + Aggregator::getLong + )).longValue(); + } + + @Override + public double getMetricDoubleValue(int rowOffset, int aggOffset) + { + return ((Number) getMetricHelper( + getMetricAggs(), + aggregators.get(rowOffset), + aggOffset, + Aggregator::getDouble + )).doubleValue(); } @Override public Object getMetricObjectValue(int rowOffset, int aggOffset) { - return getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::get); - } - - @Override - protected double getMetricDoubleValue(int rowOffset, int aggOffset) - { - return ((Number) getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::getDouble)).doubleValue(); + return getMetricHelper(getMetricAggs(), aggregators.get(rowOffset), aggOffset, Aggregator::get); } @Override public boolean isNull(int rowOffset, int aggOffset) { + final Aggregator[] aggs = aggregators.get(rowOffset); if (preserveExistingMetrics) { - return concurrentGet(rowOffset)[aggOffset].isNull() && concurrentGet(rowOffset)[aggOffset + getMetricAggs().length].isNull(); + return aggs[aggOffset].isNull() && aggs[aggOffset + getMetricAggs().length].isNull(); } else { - return concurrentGet(rowOffset)[aggOffset].isNull(); + return aggs[aggOffset].isNull(); } } @@ -535,7 +548,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex theVals.put(dimensionName, rowVals); } - Aggregator[] aggs = getAggsForRow(rowOffset); + Aggregator[] aggs = aggregators.get(rowOffset); int aggLength = preserveExistingMetrics ? aggs.length / 2 : aggs.length; for (int i = 0; i < aggLength; ++i) { theVals.put(metrics[i].getName(), getMetricHelper(metrics, aggs, i, Aggregator::get)); @@ -560,11 +573,16 @@ public class OnheapIncrementalIndex extends IncrementalIndex * for aggregating from input into output field and the aggregator for combining already aggregated field, as needed */ @Nullable - private Object getMetricHelper(AggregatorFactory[] metrics, Aggregator[] aggs, int aggOffset, Function getMetricTypeFunction) + private Object getMetricHelper( + AggregatorFactory[] metrics, + Aggregator[] aggs, + int aggOffset, + Function getMetricTypeFunction + ) { if (preserveExistingMetrics) { - // Since the preserveExistingMetrics flag is set, we will have to check and possibly retrieve the aggregated values - // from two aggregators, the aggregator for aggregating from input into output field and the aggregator + // Since the preserveExistingMetrics flag is set, we will have to check and possibly retrieve the aggregated + // values from two aggregators, the aggregator for aggregating from input into output field and the aggregator // for combining already aggregated field if (aggs[aggOffset].isNull()) { // If the aggregator for aggregating from input into output field is null, then we get the value from the @@ -583,8 +601,8 @@ public class OnheapIncrementalIndex extends IncrementalIndex return aggregatorFactory.combine(aggregatedFromSource, aggregatedFromCombined); } } else { - // If preserveExistingMetrics flag is not set then we simply get metrics from the list of Aggregator, aggs, using the - // given aggOffset + // If preserveExistingMetrics flag is not set then we simply get metrics from the list of Aggregator, aggs, + // using the given aggOffset return getMetricTypeFunction.apply(aggs[aggOffset]); } } diff --git a/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java b/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java index 04a5bb8b6b5..f28429969d3 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java @@ -35,6 +35,14 @@ public interface BitmapColumnIndex { ColumnIndexCapabilities getIndexCapabilities(); + /** + * Returns an estimated cost for computing the bitmap result. + */ + default int estimatedComputeCost() + { + return Integer.MAX_VALUE; + } + /** * Compute a bitmap result wrapped with the {@link BitmapResultFactory} representing the rows matched by this index. * If building a cursor, use {@link #computeBitmapResult(BitmapResultFactory, int, int, boolean)} instead. @@ -45,7 +53,6 @@ public interface BitmapColumnIndex * to true, bitmaps returned by this method should include true bits for any rows where * the matching result is 'unknown', such as from the input being null valued. * See {@link NullHandling#useThreeValueLogic()}. - * * @return bitmap result representing rows matched by this index */ T computeBitmapResult( @@ -69,7 +76,6 @@ public interface BitmapColumnIndex * set to true, bitmaps returned by this method should include true bits for any rows where * the matching result is 'unknown', such as from the input being null valued. * See {@link NullHandling#useThreeValueLogic()}. - * * @return bitmap result representing rows matched by this index */ @Nullable diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java index 0cf54b8c148..b6a8da3fbae 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java @@ -23,12 +23,14 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.filter.Filter; -import org.apache.druid.query.rowsandcols.StorageAdapterRowsAndColumns; +import org.apache.druid.query.rowsandcols.CursorFactoryRowsAndColumns; import org.apache.druid.segment.CloseableShapeshifter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.SimpleTopNOptimizationInspector; import org.apache.druid.segment.TimeBoundaryInspector; +import org.apache.druid.segment.TopNOptimizationInspector; import org.apache.druid.segment.WrappedTimeBoundaryInspector; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.timeline.SegmentId; @@ -36,7 +38,6 @@ import org.apache.druid.utils.CloseableUtils; import org.joda.time.Interval; import javax.annotation.Nullable; - import java.io.Closeable; import java.io.IOException; import java.util.List; @@ -104,16 +105,37 @@ public class HashJoinSegment implements SegmentReference } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return new HashJoinSegmentStorageAdapter( - baseSegment.asStorageAdapter(), + return new HashJoinSegmentCursorFactory( + baseSegment.asCursorFactory(), baseFilter, clauses, joinFilterPreAnalysis ); } + @SuppressWarnings("unchecked") + @Override + public T as(Class clazz) + { + if (CloseableShapeshifter.class.equals(clazz)) { + return (T) new CursorFactoryRowsAndColumns(asCursorFactory()); + } else if (TimeBoundaryInspector.class.equals(clazz)) { + return (T) WrappedTimeBoundaryInspector.create(baseSegment.as(TimeBoundaryInspector.class)); + } else if (TopNOptimizationInspector.class.equals(clazz)) { + // if the baseFilter is not null, then rows from underlying cursor can be potentially filtered. + // otherwise, a filtering inner or left join can also filter rows. + return (T) new SimpleTopNOptimizationInspector( + baseFilter == null && clauses.stream().allMatch( + clause -> clause.getJoinType().isLefty() || clause.getCondition().isAlwaysTrue() + ) + ); + } else { + return SegmentReference.super.as(clazz); + } + } + @Override public void close() throws IOException { @@ -134,7 +156,7 @@ public class HashJoinSegment implements SegmentReference if (acquireFailed) { break; } - acquireFailed |= joinClause.acquireReferences().map(closeable -> { + acquireFailed = joinClause.acquireReferences().map(closeable -> { closer.register(closeable); return false; }).orElse(true); @@ -153,17 +175,4 @@ public class HashJoinSegment implements SegmentReference return Optional.empty(); } } - - @SuppressWarnings("unchecked") - @Override - public T as(Class clazz) - { - if (CloseableShapeshifter.class.equals(clazz)) { - return (T) new StorageAdapterRowsAndColumns(this.asStorageAdapter()); - } else if (TimeBoundaryInspector.class.equals(clazz)) { - return (T) WrappedTimeBoundaryInspector.create(baseSegment.as(TimeBoundaryInspector.class)); - } else { - return SegmentReference.super.as(clazz); - } - } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactory.java similarity index 56% rename from processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java rename to processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactory.java index 765ab4fbd91..c359a96fad0 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactory.java @@ -28,22 +28,18 @@ import org.apache.druid.query.OrderBy; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.Metadata; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.data.Indexed; -import org.apache.druid.segment.data.ListIndexed; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey; import org.apache.druid.segment.join.filter.JoinFilterSplit; import org.apache.druid.utils.CloseableUtils; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.Arrays; @@ -51,171 +47,27 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Optional; -public class HashJoinSegmentStorageAdapter implements StorageAdapter +public class HashJoinSegmentCursorFactory implements CursorFactory { - private final StorageAdapter baseAdapter; - + private final CursorFactory baseCursorFactory; @Nullable private final Filter baseFilter; private final List clauses; private final JoinFilterPreAnalysis joinFilterPreAnalysis; - /** - * @param baseAdapter A StorageAdapter for the left-hand side base segment - * @param clauses The right-hand side clauses. The caller is responsible for ensuring that there are no - * duplicate prefixes or prefixes that shadow each other across the clauses - * @param joinFilterPreAnalysis Pre-analysis for the query we expect to run on this storage adapter - */ - HashJoinSegmentStorageAdapter( - final StorageAdapter baseAdapter, - final List clauses, - final JoinFilterPreAnalysis joinFilterPreAnalysis + public HashJoinSegmentCursorFactory( + CursorFactory baseCursorFactory, + @Nullable Filter baseFilter, + List clauses, + JoinFilterPreAnalysis joinFilterPreAnalysis ) { - this(baseAdapter, null, clauses, joinFilterPreAnalysis); - } - - /** - * @param baseAdapter A StorageAdapter for the left-hand side base segment - * @param baseFilter A filter for the left-hand side base segment - * @param clauses The right-hand side clauses. The caller is responsible for ensuring that there are no - * duplicate prefixes or prefixes that shadow each other across the clauses - * @param joinFilterPreAnalysis Pre-analysis for the query we expect to run on this storage adapter - */ - HashJoinSegmentStorageAdapter( - final StorageAdapter baseAdapter, - @Nullable final Filter baseFilter, - final List clauses, - final JoinFilterPreAnalysis joinFilterPreAnalysis - ) - { - this.baseAdapter = baseAdapter; + this.baseCursorFactory = baseCursorFactory; this.baseFilter = baseFilter; this.clauses = clauses; this.joinFilterPreAnalysis = joinFilterPreAnalysis; } - @Override - public Interval getInterval() - { - return baseAdapter.getInterval(); - } - - @Override - public RowSignature getRowSignature() - { - // Use a Set since we may encounter duplicates, if a field from a Joinable shadows one of the base fields. - final LinkedHashSet columns = new LinkedHashSet<>(baseAdapter.getRowSignature().getColumnNames()); - - for (final JoinableClause clause : clauses) { - columns.addAll(clause.getAvailableColumnsPrefixed()); - } - - final RowSignature.Builder builder = RowSignature.builder(); - for (final String column : columns) { - builder.add(column, ColumnType.fromCapabilities(getColumnCapabilities(column))); - } - - return builder.build(); - } - - @Override - public Indexed getAvailableDimensions() - { - // Use a Set since we may encounter duplicates, if a field from a Joinable shadows one of the base fields. - final LinkedHashSet availableDimensions = new LinkedHashSet<>(); - - baseAdapter.getAvailableDimensions().forEach(availableDimensions::add); - - for (JoinableClause clause : clauses) { - availableDimensions.addAll(clause.getAvailableColumnsPrefixed()); - } - - return new ListIndexed<>(Lists.newArrayList(availableDimensions)); - } - - @Override - public Iterable getAvailableMetrics() - { - return baseAdapter.getAvailableMetrics(); - } - - @Override - public int getDimensionCardinality(String column) - { - final Optional maybeClause = getClauseForColumn(column); - - if (maybeClause.isPresent()) { - final JoinableClause clause = maybeClause.get(); - return clause.getJoinable().getCardinality(clause.unprefix(column)); - } else { - return baseAdapter.getDimensionCardinality(column); - } - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - if (isBaseColumn(column)) { - return baseAdapter.getMinValue(column); - } else { - return null; - } - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - if (isBaseColumn(column)) { - return baseAdapter.getMaxValue(column); - } else { - return null; - } - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - final Optional maybeClause = getClauseForColumn(column); - - if (maybeClause.isPresent()) { - final JoinableClause clause = maybeClause.get(); - return clause.getJoinable().getColumnCapabilities(clause.unprefix(column)); - } else { - return baseAdapter.getColumnCapabilities(column); - } - } - - @Override - public int getNumRows() - { - // Cannot determine the number of rows ahead of time for a join segment (rows may be added or removed based - // on the join condition). At the time of this writing, this method is only used by the 'segmentMetadata' query, - // which isn't meant to support join segments anyway. - throw new UnsupportedOperationException("Cannot retrieve number of rows from join segment"); - } - - @Override - public Metadata getMetadata() - { - // Cannot get meaningful Metadata for this segment, since it isn't real. At the time of this writing, this method - // is only used by the 'segmentMetadata' query, which isn't meant to support join segments anyway. - throw new UnsupportedOperationException("Cannot retrieve metadata from join segment"); - } - - @Override - public boolean hasBuiltInFilters() - { - // if the baseFilter is not null, then rows from underlying storage adapter can be potentially filtered. - // otherwise, a filtering inner join can also filter rows. - return baseFilter != null || clauses.stream().anyMatch( - clause -> clause.getJoinType() == JoinType.INNER && !clause.getCondition().isAlwaysTrue() - ); - } - @Override public CursorHolder makeCursorHolder(CursorBuildSpec spec) { @@ -229,7 +81,7 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter // if there are no clauses, we can just use the base cursor directly if we apply the combined filter final CursorBuildSpec newSpec = cursorBuildSpecBuilder.setFilter(combinedFilter) .build(); - return baseAdapter.makeCursorHolder(newSpec); + return baseCursorFactory.makeCursorHolder(newSpec); } return new CursorHolder() @@ -237,19 +89,19 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter final Closer joinablesCloser = Closer.create(); /** - * Typically the same as {@link HashJoinSegmentStorageAdapter#joinFilterPreAnalysis}, but may differ when + * Typically the same as {@link HashJoinSegmentCursorFactory#joinFilterPreAnalysis}, but may differ when * an unnest datasource is layered on top of a join datasource. */ final JoinFilterPreAnalysis actualPreAnalysis; /** * Result of {@link JoinFilterAnalyzer#splitFilter} on {@link #actualPreAnalysis} and - * {@link HashJoinSegmentStorageAdapter#baseFilter}. + * {@link HashJoinSegmentCursorFactory#baseFilter}. */ final JoinFilterSplit joinFilterSplit; /** - * Cursor holder for {@link HashJoinSegmentStorageAdapter#baseAdapter}. + * Cursor holder for {@link HashJoinSegmentCursorFactory#baseCursorFactory}. */ final CursorHolder baseCursorHolder; @@ -297,8 +149,7 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter ); cursorBuildSpecBuilder.setVirtualColumns(preJoinVirtualColumns); - baseCursorHolder = - joinablesCloser.register(baseAdapter.makeCursorHolder(cursorBuildSpecBuilder.build())); + baseCursorHolder = joinablesCloser.register(baseCursorFactory.makeCursorHolder(cursorBuildSpecBuilder.build())); } @Override @@ -338,35 +189,53 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter } @Override - public boolean isFromTombstone() + public RowSignature getRowSignature() { - return baseAdapter.isFromTombstone(); - } + // Use a Set since we may encounter duplicates, if a field from a Joinable shadows one of the base fields. + final RowSignature baseSignature = baseCursorFactory.getRowSignature(); - /** - * Returns whether "column" will be selected from "baseAdapter". This is true if it is not shadowed by any joinables - * (i.e. if it does not start with any of their prefixes). - */ - public boolean isBaseColumn(final String column) - { - return !getClauseForColumn(column).isPresent(); - } + final LinkedHashSet columns = new LinkedHashSet<>(baseSignature.getColumnNames()); + for (final JoinableClause clause : clauses) { + columns.addAll(clause.getAvailableColumnsPrefixed()); + } - /** - * Returns the JoinableClause corresponding to a particular column, based on the clauses' prefixes. - * - * @param column column name - * - * @return the clause, or absent if the column does not correspond to any clause - */ - private Optional getClauseForColumn(final String column) - { + final RowSignature.Builder builder = RowSignature.builder(); // Check clauses in reverse, since "makeCursorHolder" creates the cursor in such a way that the last clause // gets first dibs to claim a column. - return Lists.reverse(clauses) - .stream() - .filter(clause -> clause.includesColumn(column)) - .findFirst(); + LinkedHashSet reverseClauses = new LinkedHashSet<>(Lists.reverse(clauses)); + for (final String column : columns) { + final Optional maybeClause = reverseClauses.stream() + .filter(c -> c.includesColumn(column)) + .findFirst(); + if (maybeClause.isPresent()) { + final JoinableClause clause = maybeClause.get(); + builder.add( + column, + ColumnType.fromCapabilities(clause.getJoinable().getColumnCapabilities(clause.unprefix(column))) + ); + } else { + builder.add(column, baseSignature.getColumnType(column).get()); + } + } + + return builder.build(); + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + final Optional maybeClause = Lists.reverse(clauses) + .stream() + .filter(x -> x.includesColumn(column)) + .findFirst(); + + if (maybeClause.isPresent()) { + final JoinableClause clause = maybeClause.get(); + return clause.getJoinable().getColumnCapabilities(clause.unprefix(column)); + } else { + return baseCursorFactory.getColumnCapabilities(column); + } } @Nullable @@ -383,7 +252,7 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter // Sorted the same way as the base segment, unless a joined-in column shadows one of the base columns. int limit = 0; for (; limit < baseOrdering.size(); limit++) { - if (!isBaseColumn(baseOrdering.get(limit).getColumnName())) { + if (!baseCursorFactory.getRowSignature().contains(baseOrdering.get(limit).getColumnName())) { break; } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java index b8ef88996ff..571806d50ee 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java @@ -30,7 +30,7 @@ import org.apache.druid.segment.VirtualColumns; import javax.annotation.Nullable; /** - * A Cursor decorator used by {@link HashJoinSegmentStorageAdapter#makeCursorHolder(CursorBuildSpec)} to add post-join + * A Cursor decorator used by {@link HashJoinSegmentCursorFactory#makeCursorHolder(CursorBuildSpec)} to add post-join * virtual columns and filters. */ public class PostJoinCursor implements Cursor @@ -84,21 +84,6 @@ public class PostJoinCursor implements Cursor } } - /** - * Matches tuples coming out of a join to a post-join condition uninterruptibly, and hence can be a long-running call. - * For this reason, {@link PostJoinCursor#advance()} instead calls {@link PostJoinCursor#advanceToMatch()} (unlike - * other cursors) that allows interruptions, thereby resolving issues where the - * CPU thread running PostJoinCursor cannot be terminated - */ - private void advanceToMatchUninterruptibly() - { - if (valueMatcher != null) { - while (!isDone() && !valueMatcher.matches(false)) { - baseCursor.advanceUninterruptibly(); - } - } - } - @Override public ColumnSelectorFactory getColumnSelectorFactory() { @@ -120,11 +105,17 @@ public class PostJoinCursor implements Cursor advanceToMatch(); } + + /** + * Advancing the post-join requires evaluating the join on whole segment and advancing without interruption can take + * a long time if there are no matches but the join itself is big. This can leave the thread running well after + * the timeout elapses. One such issue is described in + * CPU thread running PostJoinCursor cannot be terminated + */ @Override public void advanceUninterruptibly() { - baseCursor.advanceUninterruptibly(); - advanceToMatchUninterruptibly(); + advance(); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java index 5fe6d8a698a..9a56ec430cb 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java @@ -29,13 +29,13 @@ import org.apache.druid.segment.ColumnCache; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexColumnSelectorFactory; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.BaseColumn; @@ -60,7 +60,7 @@ public class BroadcastSegmentIndexedTable implements IndexedTable private static final byte CACHE_PREFIX = 0x01; private final QueryableIndexSegment segment; - private final QueryableIndexStorageAdapter adapter; + private final CursorFactory cursorFactory; private final QueryableIndex queryableIndex; private final Set keyColumns; private final RowSignature rowSignature; @@ -76,18 +76,18 @@ public class BroadcastSegmentIndexedTable implements IndexedTable this.keyColumns = keyColumns; this.version = version; this.segment = Preconditions.checkNotNull(theSegment, "Segment must not be null"); - this.adapter = Preconditions.checkNotNull( - (QueryableIndexStorageAdapter) segment.asStorageAdapter(), - "Segment[%s] must have a QueryableIndexStorageAdapter", + this.cursorFactory = Preconditions.checkNotNull( + segment.asCursorFactory(), + "Segment[%s] must have a cursor factory", segment.getId() ); this.queryableIndex = Preconditions.checkNotNull( - segment.asQueryableIndex(), - "Segment[%s] must have a QueryableIndexSegment", + segment.as(QueryableIndex.class), + "Segment[%s] must have a QueryableIndex", segment.getId() ); - this.rowSignature = adapter.getRowSignature(); + this.rowSignature = cursorFactory.getRowSignature(); // initialize keycolumn index builders final ArrayList indexBuilders = new ArrayList<>(rowSignature.size()); @@ -107,7 +107,7 @@ public class BroadcastSegmentIndexedTable implements IndexedTable indexBuilders.add(m); } - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { this.keyColumnsIndexes = Collections.emptyList(); @@ -123,7 +123,7 @@ public class BroadcastSegmentIndexedTable implements IndexedTable .stream() .map(columnName -> { // multi-value dimensions are not currently supported - if (adapter.getColumnCapabilities(columnName).hasMultipleValues().isMaybeTrue()) { + if (columnSelectorFactory.getColumnCapabilities(columnName).hasMultipleValues().isMaybeTrue()) { return NilColumnValueSelector.instance(); } return columnSelectorFactory.makeColumnValueSelector(columnName); @@ -178,7 +178,7 @@ public class BroadcastSegmentIndexedTable implements IndexedTable @Override public int numRows() { - return adapter.getNumRows(); + return queryableIndex.getNumRows(); } @Override @@ -193,7 +193,7 @@ public class BroadcastSegmentIndexedTable implements IndexedTable if (!rowSignature.contains(column)) { throw new IAE("Column[%d] is not a valid column for segment[%s]", column, segment.getId()); } - final SimpleAscendingOffset offset = new SimpleAscendingOffset(adapter.getNumRows()); + final SimpleAscendingOffset offset = new SimpleAscendingOffset(queryableIndex.getNumRows()); final BaseColumn baseColumn = queryableIndex.getColumnHolder(rowSignature.getColumnName(column)).getColumn(); final BaseObjectColumnValueSelector selector = baseColumn.makeColumnValueSelector(offset); 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 ceaffe846e2..68807c8c284 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 @@ -25,10 +25,8 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.read.columnar.FrameColumnReader; import org.apache.druid.frame.read.columnar.FrameColumnReaders; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.segment.columnar.FrameQueryableIndex; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -39,6 +37,7 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.QueryableIndex; @@ -114,22 +113,22 @@ public class FrameBasedIndexedTable implements IndexedTable indexBuilders.add(m); } - final Sequence storageAdapters = Sequences.simple( + final Sequence cursorFactories = Sequences.simple( frameBasedInlineDataSource .getFrames() .stream() .map(frameSignaturePair -> { Frame frame = frameSignaturePair.getFrame(); RowSignature rowSignature = frameSignaturePair.getRowSignature(); - return new FrameStorageAdapter(frame, FrameReader.create(rowSignature), Intervals.ETERNITY); + return FrameReader.create(rowSignature).makeCursorFactory(frame); }) .collect(Collectors.toList()) ); final Sequence sequence = Sequences.map( - storageAdapters, - storageAdapter -> { - try (final CursorHolder holder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + cursorFactories, + cursorFactory -> { + try (final CursorHolder holder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = holder.asCursor(); if (cursor == null) { return 0; diff --git a/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java b/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java index bb6ad8aaca9..f07c039ea4e 100644 --- a/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactory.java @@ -23,14 +23,18 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.query.OrderBy; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.Indexed; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -129,16 +133,7 @@ public class TombstoneSegmentizerFactory implements SegmentizerFactory }; - final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(queryableIndex) - { - @Override - public boolean isFromTombstone() - { - return true; - } - }; - - Segment segmentObject = new Segment() + final Segment segmentObject = new Segment() { @Override public SegmentId getId() @@ -149,7 +144,7 @@ public class TombstoneSegmentizerFactory implements SegmentizerFactory @Override public Interval getDataInterval() { - return asQueryableIndex().getDataInterval(); + return tombstone.getInterval(); } @Nullable @@ -160,9 +155,43 @@ public class TombstoneSegmentizerFactory implements SegmentizerFactory } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return storageAdapter; + return new CursorFactory() + { + @Override + public CursorHolder makeCursorHolder(CursorBuildSpec spec) + { + return new CursorHolder() + { + @Nullable + @Override + public Cursor asCursor() + { + return null; + } + }; + } + + @Override + public RowSignature getRowSignature() + { + return RowSignature.empty(); + } + + @Override + @Nullable + public ColumnCapabilities getColumnCapabilities(String column) + { + return null; + } + }; + } + + @Override + public boolean isTombstone() + { + return true; } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/vector/NilVectorSelector.java b/processing/src/main/java/org/apache/druid/segment/vector/NilVectorSelector.java index 762f2e73fc7..7362b38a1c6 100644 --- a/processing/src/main/java/org/apache/druid/segment/vector/NilVectorSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/vector/NilVectorSelector.java @@ -20,8 +20,8 @@ package org.apache.druid.segment.vector; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.IdLookup; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import javax.annotation.Nullable; import java.util.Arrays; @@ -29,12 +29,12 @@ import java.util.Arrays; public class NilVectorSelector implements VectorValueSelector, VectorObjectSelector, SingleValueDimensionVectorSelector, IdLookup { - private static final boolean[] DEFAULT_NULLS_VECTOR = new boolean[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE]; - private static final int[] DEFAULT_INT_VECTOR = new int[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE]; - private static final long[] DEFAULT_LONG_VECTOR = new long[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE]; - private static final float[] DEFAULT_FLOAT_VECTOR = new float[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE]; - private static final double[] DEFAULT_DOUBLE_VECTOR = new double[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE]; - private static final Object[] DEFAULT_OBJECT_VECTOR = new Object[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE]; + private static final boolean[] DEFAULT_NULLS_VECTOR = new boolean[QueryContexts.DEFAULT_VECTOR_SIZE]; + private static final int[] DEFAULT_INT_VECTOR = new int[QueryContexts.DEFAULT_VECTOR_SIZE]; + private static final long[] DEFAULT_LONG_VECTOR = new long[QueryContexts.DEFAULT_VECTOR_SIZE]; + private static final float[] DEFAULT_FLOAT_VECTOR = new float[QueryContexts.DEFAULT_VECTOR_SIZE]; + private static final double[] DEFAULT_DOUBLE_VECTOR = new double[QueryContexts.DEFAULT_VECTOR_SIZE]; + private static final Object[] DEFAULT_OBJECT_VECTOR = new Object[QueryContexts.DEFAULT_VECTOR_SIZE]; static { Arrays.fill(DEFAULT_NULLS_VECTOR, NullHandling.sqlCompatible()); @@ -69,7 +69,7 @@ public class NilVectorSelector public static NilVectorSelector create(final VectorSizeInspector vectorSizeInspector) { - if (vectorSizeInspector.getMaxVectorSize() <= QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE) { + if (vectorSizeInspector.getMaxVectorSize() <= QueryContexts.DEFAULT_VECTOR_SIZE) { // Reuse static vars when possible. return new NilVectorSelector( vectorSizeInspector, diff --git a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java index 103d769fc47..4771605583b 100644 --- a/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/vector/VectorCursor.java @@ -20,11 +20,11 @@ package org.apache.druid.segment.vector; import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorHolderFactory; +import org.apache.druid.segment.CursorFactory; /** * Vectorized cursor used during query execution. VectorCursors are available from - * {@link CursorHolderFactory#makeCursorHolder(CursorBuildSpec)} via + * {@link CursorFactory#makeCursorHolder(CursorBuildSpec)} via * {@link org.apache.druid.segment.CursorHolder#asVectorCursor()}. *

    * See {@link org.apache.druid.query.vector.VectorCursorGranularizer} for a helper that makes it easier for query diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionDeferredGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionDeferredGroupByVectorColumnSelector.java index 82e8a1f8eb8..1f42002758f 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionDeferredGroupByVectorColumnSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionDeferredGroupByVectorColumnSelector.java @@ -26,6 +26,7 @@ import org.apache.druid.math.expr.InputBindings; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.epinephelinae.collection.MemoryPointer; import org.apache.druid.query.groupby.epinephelinae.vector.GroupByVectorColumnSelector; +import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.column.RowSignature; import java.util.HashMap; @@ -90,6 +91,15 @@ public class ExpressionDeferredGroupByVectorColumnSelector implements GroupByVec return exprKeyBytes; } + @Override + public int getValueCardinality() + { + if (subSelectors.size() == 1) { + return subSelectors.get(0).getValueCardinality(); + } + return DimensionDictionarySelector.CARDINALITY_UNKNOWN; + } + @Override public int writeKeys(WritableMemory keySpace, int keySize, int keyOffset, int startRow, int endRow) { diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java index 42a723907b9..e6f4d57e1d5 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java @@ -61,9 +61,7 @@ public class ExpressionVirtualColumn implements VirtualColumn private static final Logger log = new Logger(ExpressionVirtualColumn.class); private final String name; - private final String expression; - @Nullable - private final ColumnType outputType; + private final Expression expression; private final Supplier parsedExpression; private final Supplier cacheKey; @@ -126,8 +124,7 @@ public class ExpressionVirtualColumn implements VirtualColumn ) { this.name = Preconditions.checkNotNull(name, "name"); - this.expression = Preconditions.checkNotNull(expression, "expression"); - this.outputType = outputType; + this.expression = new Expression(Preconditions.checkNotNull(expression, "expression"), outputType); this.parsedExpression = parsedExpression; this.cacheKey = makeCacheKeySupplier(); } @@ -142,14 +139,14 @@ public class ExpressionVirtualColumn implements VirtualColumn @JsonProperty public String getExpression() { - return expression; + return expression.expressionString; } @Nullable @JsonProperty public ColumnType getOutputType() { - return outputType; + return expression.outputType; } @JsonIgnore @@ -273,7 +270,7 @@ public class ExpressionVirtualColumn implements VirtualColumn ColumnIndexSelector columnIndexSelector ) { - return getParsedExpression().get().asColumnIndexSupplier(columnIndexSelector, outputType); + return getParsedExpression().get().asColumnIndexSupplier(columnIndexSelector, expression.outputType); } @Override @@ -283,7 +280,7 @@ public class ExpressionVirtualColumn implements VirtualColumn // are unable to compute the output type of the expression, either due to incomplete type information of the // inputs or because of unimplemented methods on expression implementations themselves, or, because a // ColumnInspector is not available - + final ColumnType outputType = expression.outputType; if (ExpressionProcessing.processArraysAsMultiValueStrings() && outputType != null && outputType.isArray()) { return new ColumnCapabilitiesImpl().setType(ColumnType.STRING).setHasMultipleValues(true); } @@ -299,6 +296,8 @@ public class ExpressionVirtualColumn implements VirtualColumn return inspector.getColumnCapabilities(parsedExpression.get().getBindingIfIdentifier()); } + final ColumnType outputType = expression.outputType; + final ExpressionPlan plan = ExpressionPlanner.plan(inspector, parsedExpression.get()); final ColumnCapabilities inferred = plan.inferColumnCapabilities(outputType); // if we can infer the column capabilities from the expression plan, then use that @@ -311,14 +310,14 @@ public class ExpressionVirtualColumn implements VirtualColumn log.warn( "Projected output type %s of expression %s does not match provided type %s", inferred.asTypeString(), - expression, + expression.expressionString, outputType ); } else { log.debug( "Projected output type %s of expression %s does not match provided type %s", inferred.asTypeString(), - expression, + expression.expressionString, outputType ); } @@ -348,6 +347,13 @@ public class ExpressionVirtualColumn implements VirtualColumn return cacheKey.get(); } + @Nullable + @Override + public EquivalenceKey getEquivalanceKey() + { + return expression; + } + @Override public boolean equals(final Object o) { @@ -359,14 +365,13 @@ public class ExpressionVirtualColumn implements VirtualColumn } final ExpressionVirtualColumn that = (ExpressionVirtualColumn) o; return Objects.equals(name, that.name) && - Objects.equals(expression, that.expression) && - Objects.equals(outputType, that.outputType); + Objects.equals(expression, that.expression); } @Override public int hashCode() { - return Objects.hash(name, expression, outputType); + return Objects.hash(name, expression); } @Override @@ -374,8 +379,7 @@ public class ExpressionVirtualColumn implements VirtualColumn { return "ExpressionVirtualColumn{" + "name='" + name + '\'' + - ", expression='" + expression + '\'' + - ", outputType=" + outputType + + ", expression=" + expression + '}'; } @@ -389,10 +393,10 @@ public class ExpressionVirtualColumn implements VirtualColumn final ColumnCapabilities baseCapabilities = inspector.getColumnCapabilities(parsedExpression.get().getBindingIfIdentifier()); - if (outputType == null) { + if (expression.outputType == null) { // No desired output type. Anything from the source is fine. return true; - } else if (baseCapabilities != null && outputType.equals(baseCapabilities.toColumnType())) { + } else if (baseCapabilities != null && expression.outputType.equals(baseCapabilities.toColumnType())) { // Desired output type matches the type from the source. return true; } @@ -408,10 +412,57 @@ public class ExpressionVirtualColumn implements VirtualColumn .appendString(name) .appendCacheable(parsedExpression.get()); - if (outputType != null) { - builder.appendString(outputType.toString()); + if (expression.outputType != null) { + builder.appendString(expression.outputType.toString()); } return builder.build(); }); } + + /** + * {@link VirtualColumn.EquivalenceKey} for expressions. Note that this does not check true equivalence of + * expressions, for example it will not currently consider something like 'a + b' equivalent to 'b + a'. This is ok + * for current uses of this functionality, but in theory we could push down equivalence to the parsed expression + * instead of checking for an identical string expression, it would just be a lot more expensive. + */ + private static final class Expression implements EquivalenceKey + { + private final String expressionString; + @Nullable + private final ColumnType outputType; + + private Expression(String expression, @Nullable ColumnType outputType) + { + this.expressionString = expression; + this.outputType = outputType; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Expression that = (Expression) o; + return Objects.equals(expressionString, that.expressionString) && Objects.equals(outputType, that.outputType); + } + + @Override + public int hashCode() + { + return Objects.hash(expressionString, outputType); + } + + @Override + public String toString() + { + return "Expression{" + + "expression='" + expressionString + '\'' + + ", outputType=" + outputType + + '}'; + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java index aa7dd8b88b7..5ec5958f586 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java @@ -93,27 +93,23 @@ import java.util.Set; * nested fields ({@link NestedFieldDictionaryEncodedColumn}) including using * their indexes. *

    - * This virtual column is used for the SQL operators JSON_VALUE (if {@link #processFromRaw} is set to false) or + * This virtual column is used for the SQL operators JSON_VALUE (if {@link #isProcessFromRaw()} is set to false) or * JSON_QUERY (if it is true), and accepts 'JSONPath' or 'jq' syntax string representations of paths, or a parsed * list of {@link NestedPathPart} in order to determine what should be selected from the column. *

    * Type information for nested fields is completely absent in the SQL planner, so it guesses the best it can to set - * {@link #expectedType} from the context of how something is being used, e.g. an aggregators default type or an + * {@link #getExpectedType()} from the context of how something is being used, e.g. an aggregators default type or an * explicit cast, or, if using the 'RETURNING' syntax which explicitly specifies type. This might not be the same as * if it had actual type information, but, we try to stick with whatever we chose there to do the best we can for now. *

    - * Since {@link #capabilities(ColumnInspector, String)} is determined by the {@link #expectedType}, the results will - * be best effor cast to the expected type if the column is not natively the expected type so that this column can + * Since {@link #capabilities(ColumnInspector, String)} is determined by the {@link #getExpectedType()}, the results + * will be best effor cast to the expected type if the column is not natively the expected type so that this column can * fulfill the contract of the type of selector that is likely to be created to read this column. */ public class NestedFieldVirtualColumn implements VirtualColumn { - private final String columnName; private final String outputName; - @Nullable - private final ColumnType expectedType; - private final List parts; - private final boolean processFromRaw; + private final NestedFieldSpec fieldSpec; private final boolean hasNegativeArrayIndex; @@ -128,22 +124,21 @@ public class NestedFieldVirtualColumn implements VirtualColumn @JsonProperty("useJqSyntax") @Nullable Boolean useJqSyntax ) { - this.columnName = columnName; this.outputName = outputName; if (path != null) { Preconditions.checkArgument(parts == null, "Cannot define both 'path' and 'pathParts'"); } else if (parts == null) { throw new IllegalArgumentException("Must define exactly one of 'path' or 'pathParts'"); } - + final List pathParts; if (parts != null) { - this.parts = parts; + pathParts = parts; } else { boolean isInputJq = useJqSyntax != null && useJqSyntax; - this.parts = isInputJq ? NestedPathFinder.parseJqPath(path) : NestedPathFinder.parseJsonPath(path); + pathParts = isInputJq ? NestedPathFinder.parseJqPath(path) : NestedPathFinder.parseJsonPath(path); } boolean hasNegative = false; - for (NestedPathPart part : this.parts) { + for (NestedPathPart part : pathParts) { if (part instanceof NestedPathArrayElement) { NestedPathArrayElement elementPart = (NestedPathArrayElement) part; if (elementPart.getIndex() < 0) { @@ -153,8 +148,12 @@ public class NestedFieldVirtualColumn implements VirtualColumn } } this.hasNegativeArrayIndex = hasNegative; - this.expectedType = expectedType; - this.processFromRaw = processFromRaw == null ? false : processFromRaw; + this.fieldSpec = new NestedFieldSpec( + columnName, + expectedType, + pathParts, + processFromRaw != null && processFromRaw + ); } @VisibleForTesting @@ -181,12 +180,12 @@ public class NestedFieldVirtualColumn implements VirtualColumn @Override public byte[] getCacheKey() { - final String partsString = NestedPathFinder.toNormalizedJsonPath(parts); + final String partsString = NestedPathFinder.toNormalizedJsonPath(fieldSpec.parts); return new CacheKeyBuilder(VirtualColumnCacheHelper.CACHE_TYPE_ID_USER_DEFINED).appendString("nested-field") .appendString(outputName) - .appendString(columnName) + .appendString(fieldSpec.columnName) .appendString(partsString) - .appendBoolean(processFromRaw) + .appendBoolean(fieldSpec.processFromRaw) .build(); } @@ -200,25 +199,25 @@ public class NestedFieldVirtualColumn implements VirtualColumn @JsonProperty public String getColumnName() { - return columnName; + return fieldSpec.columnName; } @JsonProperty("pathParts") public List getPathParts() { - return parts; + return fieldSpec.parts; } @JsonProperty public ColumnType getExpectedType() { - return expectedType; + return fieldSpec.expectedType; } @JsonProperty public boolean isProcessFromRaw() { - return processFromRaw; + return fieldSpec.processFromRaw; } @Override @@ -241,13 +240,13 @@ public class NestedFieldVirtualColumn implements VirtualColumn ) { // this column value selector is used for realtime queries, so we always process StructuredData - final ColumnValueSelector baseSelector = factory.makeColumnValueSelector(this.columnName); + final ColumnValueSelector baseSelector = factory.makeColumnValueSelector(fieldSpec.columnName); // processFromRaw is true that means JSON_QUERY, which can return partial results, otherwise this virtual column // is JSON_VALUE which only returns literals, so use the literal value selector instead - return processFromRaw - ? new RawFieldColumnSelector(baseSelector, parts) - : new RawFieldLiteralColumnValueSelector(baseSelector, parts); + return fieldSpec.processFromRaw + ? new RawFieldColumnSelector(baseSelector, fieldSpec.parts) + : new RawFieldLiteralColumnValueSelector(baseSelector, fieldSpec.parts); } @Nullable @@ -258,7 +257,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn ReadableOffset offset ) { - ColumnHolder holder = columnSelector.getColumnHolder(columnName); + ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName); if (holder == null) { // column doesn't exist return dimensionSpec.decorate(DimensionSelector.constant(null, dimensionSpec.getExtractionFn())); @@ -283,11 +282,11 @@ public class NestedFieldVirtualColumn implements VirtualColumn BaseColumn theColumn = holder.getColumn(); if (theColumn instanceof NestedDataComplexColumn) { final NestedDataComplexColumn column = (NestedDataComplexColumn) theColumn; - return column.makeDimensionSelector(parts, offset, extractionFn); + return column.makeDimensionSelector(fieldSpec.parts, offset, extractionFn); } // not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts - if (parts.isEmpty()) { + if (fieldSpec.parts.isEmpty()) { // dictionary encoded columns do not typically implement the value selector methods (getLong, getDouble, getFloat) // nothing *should* be using a dimension selector to call the numeric getters, but just in case... wrap their // selector in a "best effort" casting selector to implement them @@ -303,10 +302,10 @@ public class NestedFieldVirtualColumn implements VirtualColumn ); } - if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) { + if (fieldSpec.parts.size() == 1 && fieldSpec.parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) { final VariantColumn arrayColumn = (VariantColumn) theColumn; ColumnValueSelector arraySelector = arrayColumn.makeColumnValueSelector(offset); - final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex(); + final int elementNumber = ((NestedPathArrayElement) fieldSpec.parts.get(0)).getIndex(); if (elementNumber < 0) { throw new IAE("Cannot make array element selector, negative array index not supported"); } @@ -351,13 +350,13 @@ public class NestedFieldVirtualColumn implements VirtualColumn ReadableOffset offset ) { - ColumnHolder holder = columnSelector.getColumnHolder(this.columnName); + ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName); if (holder == null) { return NilColumnValueSelector.instance(); } BaseColumn theColumn = holder.getColumn(); - if (processFromRaw || hasNegativeArrayIndex) { + if (fieldSpec.processFromRaw || hasNegativeArrayIndex) { // if the path has negative array elements, or has set the flag to process 'raw' values explicitly (JSON_QUERY), // then we use the 'raw' processing of the RawFieldColumnSelector/RawFieldLiteralColumnValueSelector created // with the column selector factory instead of using the optimized nested field column @@ -367,11 +366,11 @@ public class NestedFieldVirtualColumn implements VirtualColumn // "JSON_VALUE", which only returns literals, on a NestedDataComplexColumn, so we can use the fields value selector if (theColumn instanceof NestedDataComplexColumn) { final NestedDataComplexColumn column = (NestedDataComplexColumn) theColumn; - return column.makeColumnValueSelector(parts, offset); + return column.makeColumnValueSelector(fieldSpec.parts, offset); } // not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts - if (parts.isEmpty()) { + if (fieldSpec.parts.isEmpty()) { // dictionary encoded columns do not typically implement the value selector methods (getLong, getDouble, getFloat) // so we want to wrap their selector in a "best effort" casting selector to implement them if (theColumn instanceof DictionaryEncodedColumn && !(theColumn instanceof VariantColumn)) { @@ -383,10 +382,10 @@ public class NestedFieldVirtualColumn implements VirtualColumn return theColumn.makeColumnValueSelector(offset); } - if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) { + if (fieldSpec.parts.size() == 1 && fieldSpec.parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) { final VariantColumn arrayColumn = (VariantColumn) theColumn; ColumnValueSelector arraySelector = arrayColumn.makeColumnValueSelector(offset); - final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex(); + final int elementNumber = ((NestedPathArrayElement) fieldSpec.parts.get(0)).getIndex(); if (elementNumber < 0) { throw new IAE("Cannot make array element selector, negative array index not supported"); } @@ -466,7 +465,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn ReadableVectorOffset offset ) { - ColumnHolder holder = columnSelector.getColumnHolder(columnName); + ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName); if (holder == null) { return dimensionSpec.decorate(NilVectorSelector.create(offset)); } @@ -482,11 +481,11 @@ public class NestedFieldVirtualColumn implements VirtualColumn BaseColumn theColumn = holder.getColumn(); if (theColumn instanceof NestedDataComplexColumn) { final NestedDataComplexColumn column = (NestedDataComplexColumn) theColumn; - return column.makeSingleValueDimensionVectorSelector(parts, offset); + return column.makeSingleValueDimensionVectorSelector(fieldSpec.parts, offset); } // not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts - if (parts.isEmpty()) { + if (fieldSpec.parts.isEmpty()) { // we will not end up here unless underlying column capabilities lied about something being dictionary encoded... // so no need for magic casting like nonvectorized engine return ((DictionaryEncodedColumn) theColumn).makeSingleValueDimensionVectorSelector(offset); @@ -505,7 +504,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn ReadableVectorOffset offset ) { - ColumnHolder holder = columnSelector.getColumnHolder(this.columnName); + ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName); if (holder == null) { return NilVectorSelector.create(offset); } @@ -514,80 +513,80 @@ public class NestedFieldVirtualColumn implements VirtualColumn if (column instanceof NestedDataComplexColumn) { final NestedDataComplexColumn complexColumn = (NestedDataComplexColumn) column; - if (processFromRaw) { + if (fieldSpec.processFromRaw) { // processFromRaw is true, that means JSON_QUERY, which can return partial results, otherwise this virtual column // is JSON_VALUE which only returns literals, so we can use the nested columns value selector - return new RawFieldVectorObjectSelector(complexColumn.makeVectorObjectSelector(offset), parts); + return new RawFieldVectorObjectSelector(complexColumn.makeVectorObjectSelector(offset), fieldSpec.parts); } - Set types = complexColumn.getColumnTypes(parts); + Set types = complexColumn.getColumnTypes(fieldSpec.parts); ColumnType leastRestrictiveType = null; if (types != null) { for (ColumnType type : types) { leastRestrictiveType = ColumnType.leastRestrictiveType(leastRestrictiveType, type); } } - if (leastRestrictiveType != null && leastRestrictiveType.isNumeric() && !Types.isNumeric(expectedType)) { + if (leastRestrictiveType != null && leastRestrictiveType.isNumeric() && !Types.isNumeric(fieldSpec.expectedType)) { return ExpressionVectorSelectors.castValueSelectorToObject( offset, columnName, - complexColumn.makeVectorValueSelector(parts, offset), + complexColumn.makeVectorValueSelector(fieldSpec.parts, offset), leastRestrictiveType, - expectedType == null ? ColumnType.STRING : expectedType + fieldSpec.expectedType == null ? ColumnType.STRING : fieldSpec.expectedType ); } - final VectorObjectSelector objectSelector = complexColumn.makeVectorObjectSelector(parts, offset); + final VectorObjectSelector objectSelector = complexColumn.makeVectorObjectSelector(fieldSpec.parts, offset); if (leastRestrictiveType != null && leastRestrictiveType.isArray() && - expectedType != null && - !expectedType.isArray() + fieldSpec.expectedType != null && + !fieldSpec.expectedType.isArray() ) { final ExpressionType elementType = ExpressionType.fromColumnTypeStrict(leastRestrictiveType.getElementType()); - final ExpressionType castTo = ExpressionType.fromColumnTypeStrict(expectedType); + final ExpressionType castTo = ExpressionType.fromColumnTypeStrict(fieldSpec.expectedType); return makeVectorArrayToScalarObjectSelector(offset, objectSelector, elementType, castTo); } return objectSelector; } // not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts - if (parts.isEmpty()) { + if (fieldSpec.parts.isEmpty()) { ColumnCapabilities capabilities = holder.getCapabilities(); // expectedType shouldn't possibly be null if we are being asked for an object selector and the underlying column // is numeric, else we would have been asked for a value selector Preconditions.checkArgument( - expectedType != null, + fieldSpec.expectedType != null, "Asked for a VectorObjectSelector on a numeric column, 'expectedType' must not be null" ); if (capabilities.isNumeric()) { return ExpressionVectorSelectors.castValueSelectorToObject( offset, - this.columnName, + fieldSpec.columnName, column.makeVectorValueSelector(offset), capabilities.toColumnType(), - expectedType + fieldSpec.expectedType ); } // if the underlying column is array typed, the vector object selector it spits out will homogenize stuff to // make all of the objects a consistent type, which is typically a good thing, but if we are doing mixed type // stuff and expect the output type to be scalar typed, then we should coerce things to only extract the scalars - if (capabilities.isArray() && !expectedType.isArray()) { + if (capabilities.isArray() && !fieldSpec.expectedType.isArray()) { final VectorObjectSelector delegate = column.makeVectorObjectSelector(offset); final ExpressionType elementType = ExpressionType.fromColumnTypeStrict(capabilities.getElementType()); - final ExpressionType castTo = ExpressionType.fromColumnTypeStrict(expectedType); + final ExpressionType castTo = ExpressionType.fromColumnTypeStrict(fieldSpec.expectedType); return makeVectorArrayToScalarObjectSelector(offset, delegate, elementType, castTo); } return column.makeVectorObjectSelector(offset); } - if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && column instanceof VariantColumn) { + if (fieldSpec.parts.size() == 1 && fieldSpec.parts.get(0) instanceof NestedPathArrayElement && column instanceof VariantColumn) { final VariantColumn arrayColumn = (VariantColumn) column; final ExpressionType elementType = ExpressionType.fromColumnTypeStrict( arrayColumn.getLogicalType().isArray() ? arrayColumn.getLogicalType().getElementType() : arrayColumn.getLogicalType() ); - final ExpressionType castTo = expectedType == null + final ExpressionType castTo = fieldSpec.expectedType == null ? ExpressionType.STRING - : ExpressionType.fromColumnTypeStrict(expectedType); + : ExpressionType.fromColumnTypeStrict(fieldSpec.expectedType); VectorObjectSelector arraySelector = arrayColumn.makeVectorObjectSelector(offset); - final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex(); + final int elementNumber = ((NestedPathArrayElement) fieldSpec.parts.get(0)).getIndex(); if (elementNumber < 0) { throw new IAE("Cannot make array element selector, negative array index not supported"); } @@ -646,17 +645,17 @@ public class NestedFieldVirtualColumn implements VirtualColumn ReadableVectorOffset offset ) { - ColumnHolder holder = columnSelector.getColumnHolder(this.columnName); + ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName); if (holder == null) { return NilVectorSelector.create(offset); } BaseColumn theColumn = holder.getColumn(); if (!(theColumn instanceof NestedDataComplexColumn)) { - if (parts.isEmpty()) { + if (fieldSpec.parts.isEmpty()) { if (theColumn instanceof DictionaryEncodedColumn) { final VectorObjectSelector delegate = theColumn.makeVectorObjectSelector(offset); - if (expectedType != null && expectedType.is(ValueType.LONG)) { + if (fieldSpec.expectedType != null && fieldSpec.expectedType.is(ValueType.LONG)) { return new BaseLongVectorValueSelector(offset) { private int currentOffsetId = ReadableVectorInspector.NULL_ID; @@ -701,7 +700,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn } } }; - } else if (expectedType != null && expectedType.is(ValueType.FLOAT)) { + } else if (fieldSpec.expectedType != null && fieldSpec.expectedType.is(ValueType.FLOAT)) { return new BaseFloatVectorValueSelector(offset) { private int currentOffsetId = ReadableVectorInspector.NULL_ID; @@ -794,15 +793,15 @@ public class NestedFieldVirtualColumn implements VirtualColumn } return theColumn.makeVectorValueSelector(offset); } - if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) { + if (fieldSpec.parts.size() == 1 && fieldSpec.parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) { final VariantColumn arrayColumn = (VariantColumn) theColumn; VectorObjectSelector arraySelector = arrayColumn.makeVectorObjectSelector(offset); - final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex(); + final int elementNumber = ((NestedPathArrayElement) fieldSpec.parts.get(0)).getIndex(); if (elementNumber < 0) { throw new IAE("Cannot make array element selector, negative array index not supported"); } - if (expectedType != null && expectedType.is(ValueType.LONG)) { + if (fieldSpec.expectedType != null && fieldSpec.expectedType.is(ValueType.LONG)) { return new BaseLongVectorValueSelector(offset) { private final long[] longs = new long[offset.getMaxVectorSize()]; @@ -871,7 +870,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn return nulls; } }; - } else if (expectedType != null && expectedType.is(ValueType.FLOAT)) { + } else if (fieldSpec.expectedType != null && fieldSpec.expectedType.is(ValueType.FLOAT)) { return new BaseFloatVectorValueSelector(offset) { private final float[] floats = new float[offset.getMaxVectorSize()]; @@ -1015,12 +1014,12 @@ public class NestedFieldVirtualColumn implements VirtualColumn final NestedDataComplexColumn column = (NestedDataComplexColumn) theColumn; // if column is numeric, it has a vector value selector, so we can directly make a vector value selector // if we are missing an expectedType, then we've got nothing else to work with so try it anyway - if (column.isNumeric(parts) || expectedType == null) { - return column.makeVectorValueSelector(parts, offset); + if (column.isNumeric(fieldSpec.parts) || fieldSpec.expectedType == null) { + return column.makeVectorValueSelector(fieldSpec.parts, offset); } - final VectorObjectSelector objectSelector = column.makeVectorObjectSelector(parts, offset); - if (expectedType.is(ValueType.LONG)) { + final VectorObjectSelector objectSelector = column.makeVectorObjectSelector(fieldSpec.parts, offset); + if (fieldSpec.expectedType.is(ValueType.LONG)) { return new BaseLongVectorValueSelector(offset) { private final long[] longVector = new long[offset.getMaxVectorSize()]; @@ -1162,47 +1161,47 @@ public class NestedFieldVirtualColumn implements VirtualColumn ColumnIndexSelector indexSelector ) { - ColumnHolder holder = indexSelector.getColumnHolder(this.columnName); + ColumnHolder holder = indexSelector.getColumnHolder(fieldSpec.columnName); if (holder == null) { return null; } BaseColumn theColumn = holder.getColumn(); if (theColumn instanceof CompressedNestedDataComplexColumn) { final CompressedNestedDataComplexColumn nestedColumn = (CompressedNestedDataComplexColumn) theColumn; - final ColumnIndexSupplier nestedColumnPathIndexSupplier = nestedColumn.getColumnIndexSupplier(parts); - if (nestedColumnPathIndexSupplier == null && processFromRaw) { + final ColumnIndexSupplier nestedColumnPathIndexSupplier = nestedColumn.getColumnIndexSupplier(fieldSpec.parts); + if (nestedColumnPathIndexSupplier == null && fieldSpec.processFromRaw) { // if processing from raw, a non-exstent path from parts doesn't mean the path doesn't really exist // so fall back to no indexes return NoIndexesColumnIndexSupplier.getInstance(); } - if (expectedType != null) { - final Set types = nestedColumn.getColumnTypes(parts); + if (fieldSpec.expectedType != null) { + final Set types = nestedColumn.getColumnTypes(fieldSpec.parts); // if the expected output type is numeric but not all of the input types are numeric, we might have additional // null values than what the null value bitmap is tracking, fall back to not using indexes - if (expectedType.isNumeric() && (types == null || types.stream().anyMatch(t -> !t.isNumeric()))) { + if (fieldSpec.expectedType.isNumeric() && (types == null || types.stream().anyMatch(t -> !t.isNumeric()))) { return NoIndexesColumnIndexSupplier.getInstance(); } } return nestedColumnPathIndexSupplier; } - if (parts.isEmpty()) { + if (fieldSpec.parts.isEmpty()) { final ColumnIndexSupplier baseIndexSupplier = holder.getIndexSupplier(); - if (expectedType != null) { + if (fieldSpec.expectedType != null) { if (theColumn instanceof NumericColumn) { return baseIndexSupplier; } if (theColumn instanceof NestedCommonFormatColumn) { final NestedCommonFormatColumn commonFormat = (NestedCommonFormatColumn) theColumn; - if (expectedType.isNumeric() && !commonFormat.getLogicalType().isNumeric()) { + if (fieldSpec.expectedType.isNumeric() && !commonFormat.getLogicalType().isNumeric()) { return NoIndexesColumnIndexSupplier.getInstance(); } } else { - return expectedType.isNumeric() ? NoIndexesColumnIndexSupplier.getInstance() : baseIndexSupplier; + return fieldSpec.expectedType.isNumeric() ? NoIndexesColumnIndexSupplier.getInstance() : baseIndexSupplier; } } return baseIndexSupplier; } - if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) { + if (fieldSpec.parts.size() == 1 && fieldSpec.parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) { // cannot use the array column index supplier directly, in the future array columns should expose a function // with a signature like 'getArrayElementIndexSupplier(int index)' to allow getting indexes for specific elements // if we want to support this stuff. Right now VariantArrayColumn doesn't actually retain enough information about @@ -1215,7 +1214,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn @Override public ColumnCapabilities capabilities(String columnName) { - if (processFromRaw) { + if (fieldSpec.processFromRaw) { // JSON_QUERY always returns a StructuredData return ColumnCapabilitiesImpl.createDefault() .setType(ColumnType.NESTED_DATA) @@ -1225,7 +1224,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn // this should only be used for 'realtime' queries, so don't indicate that we are dictionary encoded or have indexes // from here return ColumnCapabilitiesImpl.createDefault() - .setType(expectedType != null ? expectedType : ColumnType.STRING) + .setType(fieldSpec.expectedType != null ? fieldSpec.expectedType : ColumnType.STRING) .setHasNulls(true); } @@ -1233,8 +1232,8 @@ public class NestedFieldVirtualColumn implements VirtualColumn @Override public ColumnCapabilities capabilities(ColumnInspector inspector, String columnName) { - if (processFromRaw) { - if (expectedType != null && expectedType.isArray() && ColumnType.NESTED_DATA.equals(expectedType.getElementType())) { + if (fieldSpec.processFromRaw) { + if (fieldSpec.expectedType != null && fieldSpec.expectedType.isArray() && ColumnType.NESTED_DATA.equals(fieldSpec.expectedType.getElementType())) { // arrays of objects! return ColumnCapabilitiesImpl.createDefault() .setType(ColumnType.ofArray(ColumnType.NESTED_DATA)) @@ -1249,16 +1248,16 @@ public class NestedFieldVirtualColumn implements VirtualColumn } // ColumnInspector isn't really enough... we need the ability to read the complex column itself to examine // the nested fields type information to really be accurate here, so we rely on the expectedType to guide us - final ColumnCapabilities capabilities = inspector.getColumnCapabilities(this.columnName); + final ColumnCapabilities capabilities = inspector.getColumnCapabilities(fieldSpec.columnName); if (capabilities != null) { // if the underlying column is a nested column (and persisted to disk, re: the dictionary encoded check) if (capabilities.is(ValueType.COMPLEX) && capabilities.getComplexTypeName().equals(NestedDataComplexTypeSerde.TYPE_NAME) && capabilities.isDictionaryEncoded().isTrue()) { - final boolean useDictionary = parts.isEmpty() || !(parts.get(parts.size() - 1) instanceof NestedPathArrayElement); + final boolean useDictionary = fieldSpec.parts.isEmpty() || !(fieldSpec.parts.get(fieldSpec.parts.size() - 1) instanceof NestedPathArrayElement); return ColumnCapabilitiesImpl.createDefault() - .setType(expectedType != null ? expectedType : ColumnType.STRING) + .setType(fieldSpec.expectedType != null ? fieldSpec.expectedType : ColumnType.STRING) .setDictionaryEncoded(useDictionary) .setDictionaryValuesSorted(useDictionary) .setDictionaryValuesUnique(useDictionary) @@ -1266,12 +1265,12 @@ public class NestedFieldVirtualColumn implements VirtualColumn .setHasNulls(true); } // column is not nested, use underlying column capabilities, adjusted for expectedType as necessary - if (parts.isEmpty()) { + if (fieldSpec.parts.isEmpty()) { ColumnCapabilitiesImpl copy = ColumnCapabilitiesImpl.copyOf(capabilities); - if (expectedType != null) { - copy.setType(expectedType); + if (fieldSpec.expectedType != null) { + copy.setType(fieldSpec.expectedType); copy.setHasNulls( - copy.hasNulls().or(ColumnCapabilities.Capable.of(expectedType.getType() != capabilities.getType())) + copy.hasNulls().or(ColumnCapabilities.Capable.of(fieldSpec.expectedType.getType() != capabilities.getType())) ); } return copy; @@ -1287,7 +1286,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn @Override public List requiredColumns() { - return Collections.singletonList(columnName); + return Collections.singletonList(fieldSpec.columnName); } @Override @@ -1296,6 +1295,13 @@ public class NestedFieldVirtualColumn implements VirtualColumn return false; } + @Nullable + @Override + public EquivalenceKey getEquivalanceKey() + { + return fieldSpec; + } + @Override public boolean equals(Object o) { @@ -1306,28 +1312,25 @@ public class NestedFieldVirtualColumn implements VirtualColumn return false; } NestedFieldVirtualColumn that = (NestedFieldVirtualColumn) o; - return columnName.equals(that.columnName) && - outputName.equals(that.outputName) && - parts.equals(that.parts) && - Objects.equals(expectedType, that.expectedType) && - processFromRaw == that.processFromRaw; + return outputName.equals(that.outputName) && + fieldSpec.equals(that.fieldSpec); } @Override public int hashCode() { - return Objects.hash(columnName, parts, outputName, expectedType, processFromRaw); + return Objects.hash(outputName, fieldSpec); } @Override public String toString() { return "NestedFieldVirtualColumn{" + - "columnName='" + columnName + '\'' + + "columnName='" + fieldSpec.columnName + '\'' + ", outputName='" + outputName + '\'' + - ", typeHint='" + expectedType + '\'' + - ", pathParts='" + parts + '\'' + - ", allowFallback=" + processFromRaw + + ", typeHint='" + fieldSpec.expectedType + '\'' + + ", pathParts='" + fieldSpec.parts + '\'' + + ", allowFallback=" + fieldSpec.processFromRaw + '}'; } @@ -1386,6 +1389,50 @@ public class NestedFieldVirtualColumn implements VirtualColumn }; } + private static class NestedFieldSpec implements EquivalenceKey + { + private final String columnName; + @Nullable + private final ColumnType expectedType; + private final List parts; + private final boolean processFromRaw; + + private NestedFieldSpec( + String columnName, + @Nullable ColumnType expectedType, + List parts, + boolean processFromRaw + ) + { + this.columnName = columnName; + this.expectedType = expectedType; + this.parts = parts; + this.processFromRaw = processFromRaw; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + NestedFieldSpec that = (NestedFieldSpec) o; + return processFromRaw == that.processFromRaw + && Objects.equals(columnName, that.columnName) + && Objects.equals(expectedType, that.expectedType) + && Objects.equals(parts, that.parts); + } + + @Override + public int hashCode() + { + return Objects.hash(columnName, expectedType, parts, processFromRaw); + } + } + /** * Process the "raw" data to extract non-complex values. Like {@link RawFieldColumnSelector} but does not return * complex nested objects and does not wrap the results in {@link StructuredData}. diff --git a/processing/src/test/java/org/apache/druid/collections/QueueNonBlockingPoolTest.java b/processing/src/test/java/org/apache/druid/collections/QueueNonBlockingPoolTest.java new file mode 100644 index 00000000000..ae20bce41c9 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/collections/QueueNonBlockingPoolTest.java @@ -0,0 +1,76 @@ +/* + * 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.collections; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.NoSuchElementException; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; + +public class QueueNonBlockingPoolTest +{ + @Test + public void testTakeAllTwice() + { + final BlockingQueue queue = new ArrayBlockingQueue<>(2); + queue.add("foo"); + queue.add("bar"); + + final QueueNonBlockingPool pool = new QueueNonBlockingPool<>(queue); + + // Take everything from pool + final ResourceHolder obj1 = pool.take(); + Assert.assertEquals("foo", obj1.get()); + Assert.assertEquals(1, queue.size()); + + final ResourceHolder obj2 = pool.take(); + Assert.assertEquals("bar", obj2.get()); + Assert.assertEquals(0, queue.size()); + + Assert.assertThrows( + NoSuchElementException.class, + pool::take + ); + + // Re-fill pool in reverse order + obj2.close(); + Assert.assertEquals(1, queue.size()); + + obj1.close(); + Assert.assertEquals(2, queue.size()); + + // Re-take everything from pool + + final ResourceHolder obj1b = pool.take(); + Assert.assertEquals("bar", obj1b.get()); + Assert.assertEquals(1, queue.size()); + + final ResourceHolder obj2b = pool.take(); + Assert.assertEquals("foo", obj2b.get()); + Assert.assertEquals(0, queue.size()); + + Assert.assertThrows( + NoSuchElementException.class, + pool::take + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java b/processing/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java index 52be34e78f9..9c261c7935f 100644 --- a/processing/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java +++ b/processing/src/test/java/org/apache/druid/common/guava/FutureUtilsTest.java @@ -243,6 +243,32 @@ public class FutureUtilsTest ); } + @Test + public void test_transformAsync_exceptionInFunction() + { + final ListenableFuture f = FutureUtils.transformAsync( + Futures.immediateFuture("x"), + s -> { + throw new ISE("error!"); + } + ); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + f::get + ); + + MatcherAssert.assertThat( + e, + ThrowableCauseMatcher.hasCause(CoreMatchers.instanceOf(ISE.class)) + ); + + MatcherAssert.assertThat( + e, + ThrowableCauseMatcher.hasCause(ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("error!"))) + ); + } + @Test public void test_coalesce_allOk() throws Exception { diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java index 7768e09d9dc..0a65862579d 100644 --- a/processing/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java +++ b/processing/src/test/java/org/apache/druid/data/input/impl/CsvInputFormatTest.java @@ -44,7 +44,7 @@ public class CsvInputFormatTest extends InitializedNullHandlingTest public void testSerde() throws IOException { final ObjectMapper mapper = new ObjectMapper(); - final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), "|", null, true, 10); + final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), "|", null, true, 10, null); final byte[] bytes = mapper.writeValueAsBytes(format); final CsvInputFormat fromJson = (CsvInputFormat) mapper.readValue(bytes, InputFormat.class); Assert.assertEquals(format, fromJson); @@ -127,12 +127,23 @@ public class CsvInputFormatTest extends InitializedNullHandlingTest ); } + @Test + public void testDeserializeWithTryParseNumbers() throws IOException + { + final ObjectMapper mapper = new ObjectMapper(); + final CsvInputFormat inputFormat = (CsvInputFormat) mapper.readValue( + "{\"type\":\"csv\",\"hasHeaderRow\":true,\"tryParseNumbers\":true}", + InputFormat.class + ); + Assert.assertTrue(inputFormat.shouldTryParseNumbers()); + } + @Test public void testComma() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Column[a,] cannot have the delimiter[,] in its name"); - new CsvInputFormat(Collections.singletonList("a,"), "|", null, false, 0); + new CsvInputFormat(Collections.singletonList("a,"), "|", null, false, 0, null); } @Test @@ -140,20 +151,20 @@ public class CsvInputFormatTest extends InitializedNullHandlingTest { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot have same delimiter and list delimiter of [,]"); - new CsvInputFormat(Collections.singletonList("a\t"), ",", null, false, 0); + new CsvInputFormat(Collections.singletonList("a\t"), ",", null, false, 0, null); } @Test public void testFindColumnsFromHeaderWithColumnsReturningItsValue() { - final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), null, null, true, 0); + final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), null, null, true, 0, null); Assert.assertTrue(format.isFindColumnsFromHeader()); } @Test public void testFindColumnsFromHeaderWithMissingColumnsReturningItsValue() { - final CsvInputFormat format = new CsvInputFormat(null, null, null, true, 0); + final CsvInputFormat format = new CsvInputFormat(null, null, null, true, 0, null); Assert.assertTrue(format.isFindColumnsFromHeader()); } @@ -162,13 +173,13 @@ public class CsvInputFormatTest extends InitializedNullHandlingTest { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Either [columns] or [findColumnsFromHeader] must be set"); - new CsvInputFormat(null, null, null, null, 0); + new CsvInputFormat(null, null, null, null, 0, null); } @Test public void testMissingFindColumnsFromHeaderWithColumnsReturningFalse() { - final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), null, null, null, 0); + final CsvInputFormat format = new CsvInputFormat(Collections.singletonList("a"), null, null, null, 0, null); Assert.assertFalse(format.isFindColumnsFromHeader()); } @@ -177,20 +188,20 @@ public class CsvInputFormatTest extends InitializedNullHandlingTest { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot accept both [findColumnsFromHeader] and [hasHeaderRow]"); - new CsvInputFormat(null, null, true, false, 0); + new CsvInputFormat(null, null, true, false, 0, null); } @Test public void testHasHeaderRowWithMissingColumnsReturningItsValue() { - final CsvInputFormat format = new CsvInputFormat(null, null, true, null, 0); + final CsvInputFormat format = new CsvInputFormat(null, null, true, null, 0, null); Assert.assertTrue(format.isFindColumnsFromHeader()); } @Test public void test_getWeightedSize_withoutCompression() { - final CsvInputFormat format = new CsvInputFormat(null, null, true, null, 0); + final CsvInputFormat format = new CsvInputFormat(null, null, true, null, 0, null); final long unweightedSize = 100L; Assert.assertEquals(unweightedSize, format.getWeightedSize("file.csv", unweightedSize)); } @@ -198,7 +209,7 @@ public class CsvInputFormatTest extends InitializedNullHandlingTest @Test public void test_getWeightedSize_withGzCompression() { - final CsvInputFormat format = new CsvInputFormat(null, null, true, null, 0); + final CsvInputFormat format = new CsvInputFormat(null, null, true, null, 0, null); final long unweightedSize = 100L; Assert.assertEquals( unweightedSize * CompressionUtils.COMPRESSED_TEXT_WEIGHT_FACTOR, diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java index ae8c8709a1d..70d6c7b705d 100644 --- a/processing/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java +++ b/processing/src/test/java/org/apache/druid/data/input/impl/CsvReaderTest.java @@ -61,12 +61,34 @@ public class CsvReaderTest { final ByteEntity source = writeData( ImmutableList.of( - "2019-01-01T00:00:10Z,name_1,5", - "2019-01-01T00:00:20Z,name_2,10", - "2019-01-01T00:00:30Z,name_3,15" + "2019-01-01T00:00:10Z,name_1,5,0.0", + "2019-01-01T00:00:20Z,name_2,10,1.0", + "2019-01-01T00:00:30Z,name_3,15,2.0" ) ); - final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "score"), null, null, false, 0); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "score"), null, null, false, 0, null); + assertResult(source, format); + } + + @Test + public void testParseNumbers() throws IOException + { + final ByteEntity source = writeData( + ImmutableList.of( + "2019-01-01T00:00:10Z,name_1,5,0.0", + "2019-01-01T00:00:20Z,name_2,10,1.0", + "2019-01-01T00:00:30Z,name_3,15,2.0" + ) + ); + final CsvInputFormat format = new CsvInputFormat( + ImmutableList.of("ts", "name", "score", "rating_dbl"), + null, + null, + false, + 0, + true + ); + assertResult(source, format); } @@ -81,7 +103,7 @@ public class CsvReaderTest "2019-01-01T00:00:30Z,name_3,15" ) ); - final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), null, null, true, 0); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), null, null, true, 0, null); assertResult(source, format); } @@ -96,7 +118,7 @@ public class CsvReaderTest "2019-01-01T00:00:30Z,name_3,15" ) ); - final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "score"), null, null, false, 1); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "score"), null, null, false, 1, null); assertResult(source, format); } @@ -112,7 +134,7 @@ public class CsvReaderTest "2019-01-01T00:00:30Z,name_3,15" ) ); - final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), null, null, true, 1); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), null, null, true, 1, null); assertResult(source, format); } @@ -127,7 +149,7 @@ public class CsvReaderTest "2019-01-01T00:00:30Z,name_3,15|3" ) ); - final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), "|", null, true, 0); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of(), "|", null, true, 0, null); final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA, source, null); int numResults = 0; try (CloseableIterator iterator = reader.read()) { @@ -227,7 +249,124 @@ public class CsvReaderTest null, null, false, - 0 + 0, + null + ); + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("Timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("Timestamp"))), + ColumnsFilter.all() + ), + source, + null + ); + + try (CloseableIterator iterator = reader.read()) { + final Iterator expectedRowIterator = expectedResults.iterator(); + while (iterator.hasNext()) { + Assert.assertTrue(expectedRowIterator.hasNext()); + Assert.assertEquals(expectedRowIterator.next(), iterator.next()); + } + } + } + + @Test + public void testDataWithParseNumbersEnabled() throws IOException + { + final ByteEntity source = writeData( + ImmutableList.of( + "3,1.0,2,1|2|3,false,2018-05-05T10:00:00Z", + "34,-2.0,foo,1,true,2018-05-06T10:00:00Z", + "343,3.0,bar,2|3|4,false,2018-05-07T10:00:00Z", + "545,-4.0,7,2,true,2018-05-08T10:00:00Z", + "65,5.0,baz,3|4,false,2018-05-09T10:00:00Z" + ) + ); + final RowSignature signature = + RowSignature.builder() + .add("Value1", null) + .add("Value2", null) + .add("Value3", null) + .add("Value4", null) + .add("Value5", null) + .add("Timestamp", null) + .build(); + + final List expectedResults = ImmutableList.of( + new ListBasedInputRow( + signature, + DateTimes.of("2018-05-05T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableList.of( + 3L, + 1.0, + 2L, + ImmutableList.of(1L, 2L, 3L), + "false", + "2018-05-05T10:00:00Z" + ) + ), + new ListBasedInputRow( + signature, + DateTimes.of("2018-05-06T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableList.of( + 34L, + -2.0, + "foo", + 1L, + "true", + "2018-05-06T10:00:00Z" + ) + ), + new ListBasedInputRow( + signature, + DateTimes.of("2018-05-07T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableList.of( + 343L, + 3.0, + "bar", + ImmutableList.of(2L, 3L, 4L), + "false", + "2018-05-07T10:00:00Z" + ) + ), + new ListBasedInputRow( + signature, + DateTimes.of("2018-05-08T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableList.of( + 545L, + -4.0, + 7L, + 2L, + "true", + "2018-05-08T10:00:00Z" + ) + ), + new ListBasedInputRow( + signature, + DateTimes.of("2018-05-09T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableList.of( + 65L, + 5.0, + "baz", + ImmutableList.of(3L, 4L), + "false", + "2018-05-09T10:00:00Z" + ) + ) + ); + final CsvInputFormat format = new CsvInputFormat( + ImmutableList.of("Value1", "Value2", "Value3", "Value4", "Value5", "Timestamp"), + "|", + null, + false, + 0, + true ); final InputEntityReader reader = format.createReader( new InputRowSchema( @@ -256,7 +395,7 @@ public class CsvReaderTest "2019-01-01T00:00:10Z,name_1,\"Как говорится: \\\"\"всё течет, всё изменяется\\\"\". Украина как всегда обвиняет Россию в собственных проблемах. #ПровокацияКиева\"" ) ); - final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "Comment"), null, null, false, 0); + final CsvInputFormat format = new CsvInputFormat(ImmutableList.of("ts", "name", "Comment"), null, null, false, 0, null); final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA, source, null); try (CloseableIterator iterator = reader.read()) { Assert.assertTrue(iterator.hasNext()); @@ -300,6 +439,17 @@ public class CsvReaderTest StringUtils.format("name_%d", numResults + 1), Iterables.getOnlyElement(row.getDimension("name")) ); + if (format.shouldTryParseNumbers()) { + Assert.assertEquals(((numResults + 1) * 5L), row.getRaw("score")); + if (format.getColumns().contains("rating_dbl")) { + Assert.assertEquals(numResults * 1.0, row.getRaw("rating_dbl")); + } + } else { + Assert.assertEquals(Integer.toString((numResults + 1) * 5), row.getRaw("score")); + if (format.getColumns().contains("rating_dbl")) { + Assert.assertEquals(Double.toString(numResults * 1.0), row.getRaw("rating_dbl")); + } + } Assert.assertEquals( Integer.toString((numResults + 1) * 5), Iterables.getOnlyElement(row.getDimension("score")) diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/DelimitedInputFormatTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/DelimitedInputFormatTest.java index 9cf6db93c3f..8f4c01bcd49 100644 --- a/processing/src/test/java/org/apache/druid/data/input/impl/DelimitedInputFormatTest.java +++ b/processing/src/test/java/org/apache/druid/data/input/impl/DelimitedInputFormatTest.java @@ -45,7 +45,8 @@ public class DelimitedInputFormatTest "delim", null, true, - 10 + 10, + null ); final byte[] bytes = mapper.writeValueAsBytes(format); final DelimitedInputFormat fromJson = (DelimitedInputFormat) mapper.readValue(bytes, InputFormat.class); @@ -57,7 +58,7 @@ public class DelimitedInputFormatTest { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Column[a\t] cannot have the delimiter[\t] in its name"); - new DelimitedInputFormat(Collections.singletonList("a\t"), ",", null, null, false, 0); + new DelimitedInputFormat(Collections.singletonList("a\t"), ",", null, null, false, 0, null); } @Test @@ -65,7 +66,7 @@ public class DelimitedInputFormatTest { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot have same delimiter and list delimiter of [,]"); - new DelimitedInputFormat(Collections.singletonList("a\t"), ",", ",", null, false, 0); + new DelimitedInputFormat(Collections.singletonList("a\t"), ",", ",", null, false, 0, null); } @Test @@ -73,7 +74,7 @@ public class DelimitedInputFormatTest { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Column[a|] cannot have the delimiter[|] in its name"); - new DelimitedInputFormat(Collections.singletonList("a|"), ",", "|", null, false, 0); + new DelimitedInputFormat(Collections.singletonList("a|"), ",", "|", null, false, 0, null); } @Test @@ -85,7 +86,8 @@ public class DelimitedInputFormatTest "delim", null, true, - 0 + 0, + null ); Assert.assertTrue(format.isFindColumnsFromHeader()); } @@ -99,17 +101,44 @@ public class DelimitedInputFormatTest "delim", null, true, - 0 + 0, + null ); Assert.assertTrue(format.isFindColumnsFromHeader()); } + @Test + public void testTryParseNumbers() + { + final DelimitedInputFormat format = new DelimitedInputFormat( + null, + null, + "delim", + null, + true, + 0, + true + ); + Assert.assertTrue(format.shouldTryParseNumbers()); + } + + @Test + public void testDeserializeWithTryParseNumbers() throws IOException + { + final ObjectMapper mapper = new ObjectMapper(); + final DelimitedInputFormat inputFormat = (DelimitedInputFormat) mapper.readValue( + "{\"type\":\"tsv\",\"hasHeaderRow\":true,\"tryParseNumbers\":true}", + InputFormat.class + ); + Assert.assertTrue(inputFormat.shouldTryParseNumbers()); + } + @Test public void testMissingFindColumnsFromHeaderWithMissingColumnsThrowingError() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Either [columns] or [findColumnsFromHeader] must be set"); - new DelimitedInputFormat(null, null, "delim", null, null, 0); + new DelimitedInputFormat(null, null, "delim", null, null, 0, null); } @Test @@ -121,7 +150,8 @@ public class DelimitedInputFormatTest "delim", null, null, - 0 + 0, + null ); Assert.assertFalse(format.isFindColumnsFromHeader()); } @@ -131,19 +161,19 @@ public class DelimitedInputFormatTest { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot accept both [findColumnsFromHeader] and [hasHeaderRow]"); - new DelimitedInputFormat(null, null, "delim", true, false, 0); + new DelimitedInputFormat(null, null, "delim", true, false, 0, null); } @Test public void testHasHeaderRowWithMissingColumnsReturningItsValue() { - final DelimitedInputFormat format = new DelimitedInputFormat(null, null, "delim", true, null, 0); + final DelimitedInputFormat format = new DelimitedInputFormat(null, null, "delim", true, null, 0, null); Assert.assertTrue(format.isFindColumnsFromHeader()); } @Test public void test_getWeightedSize_withoutCompression() { - final DelimitedInputFormat format = new DelimitedInputFormat(null, null, "delim", true, null, 0); + final DelimitedInputFormat format = new DelimitedInputFormat(null, null, "delim", true, null, 0, null); final long unweightedSize = 100L; Assert.assertEquals(unweightedSize, format.getWeightedSize("file.tsv", unweightedSize)); } @@ -151,7 +181,7 @@ public class DelimitedInputFormatTest @Test public void test_getWeightedSize_withGzCompression() { - final DelimitedInputFormat format = new DelimitedInputFormat(null, null, "delim", true, null, 0); + final DelimitedInputFormat format = new DelimitedInputFormat(null, null, "delim", true, null, 0, null); final long unweightedSize = 100L; Assert.assertEquals( unweightedSize * CompressionUtils.COMPRESSED_TEXT_WEIGHT_FACTOR, diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/DelimitedReaderTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/DelimitedReaderTest.java index c98d8fff6a8..aaa6490b298 100644 --- a/processing/src/test/java/org/apache/druid/data/input/impl/DelimitedReaderTest.java +++ b/processing/src/test/java/org/apache/druid/data/input/impl/DelimitedReaderTest.java @@ -26,9 +26,11 @@ import org.apache.druid.data.input.ColumnsFilter; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.ListBasedInputRow; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.segment.column.RowSignature; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -36,6 +38,7 @@ import org.junit.Test; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Arrays; +import java.util.Iterator; import java.util.List; import java.util.stream.Collectors; @@ -58,18 +61,41 @@ public class DelimitedReaderTest { final ByteEntity source = writeData( ImmutableList.of( - "2019-01-01T00:00:10Z\tname_1\t5", - "2019-01-01T00:00:20Z\tname_2\t10", - "2019-01-01T00:00:30Z\tname_3\t15" + "2019-01-01T00:00:10Z\tname_1\t5\t0.0", + "2019-01-01T00:00:20Z\tname_2\t10\t1.0", + "2019-01-01T00:00:30Z\tname_3\t15\t2.0" ) ); final DelimitedInputFormat format = new DelimitedInputFormat( - ImmutableList.of("ts", "name", "score"), + ImmutableList.of("ts", "name", "score", "rating_dbl"), null, null, null, false, - 0 + 0, + null + ); + assertResult(source, format); + } + + @Test + public void testParseNumbers() throws IOException + { + final ByteEntity source = writeData( + ImmutableList.of( + "2019-01-01T00:00:10Z\tname_1\t5\t0.0", + "2019-01-01T00:00:20Z\tname_2\t10\t1.0", + "2019-01-01T00:00:30Z\tname_3\t15\t2.0" + ) + ); + final DelimitedInputFormat format = new DelimitedInputFormat( + ImmutableList.of("ts", "name", "score", "rating_dbl"), + null, + null, + null, + false, + 0, + true ); assertResult(source, format); } @@ -85,7 +111,7 @@ public class DelimitedReaderTest "2019-01-01T00:00:30Z\tname_3\t15" ) ); - final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), null, null, null, true, 0); + final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), null, null, null, true, 0, null); assertResult(source, format); } @@ -106,7 +132,8 @@ public class DelimitedReaderTest null, null, false, - 1 + 1, + null ); assertResult(source, format); } @@ -123,7 +150,7 @@ public class DelimitedReaderTest "2019-01-01T00:00:30Z\tname_3\t15" ) ); - final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), null, null, null, true, 1); + final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), null, null, null, true, 1, null); assertResult(source, format); } @@ -138,7 +165,7 @@ public class DelimitedReaderTest "2019-01-01T00:00:30Z\tname_3\t15|3" ) ); - final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), "|", null, null, true, 0); + final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), "|", null, null, true, 0, null); final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA, source, null); int numResults = 0; try (CloseableIterator iterator = reader.read()) { @@ -162,6 +189,123 @@ public class DelimitedReaderTest } } + @Test + public void testDataWithParseNumbersEnabled() throws IOException + { + final ByteEntity source = writeData( + ImmutableList.of( + "3\t1.0\t2\t1|2|3\tfalse\t2018-05-05T10:00:00Z", + "34\t-2.0\tfoo\t1\ttrue\t2018-05-06T10:00:00Z", + "343\t3.0\tbar\t2|3|4\tfalse\t2018-05-07T10:00:00Z", + "545\t-4.0\t7\t2\ttrue\t2018-05-08T10:00:00Z", + "65\t5.0\tbaz\t3|4\tfalse\t2018-05-09T10:00:00Z" + ) + ); + final RowSignature signature = + RowSignature.builder() + .add("Value1", null) + .add("Value2", null) + .add("Value3", null) + .add("Value4", null) + .add("Value5", null) + .add("Timestamp", null) + .build(); + + final List expectedResults = ImmutableList.of( + new ListBasedInputRow( + signature, + DateTimes.of("2018-05-05T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableList.of( + 3L, + 1.0, + 2L, + ImmutableList.of(1L, 2L, 3L), + "false", + "2018-05-05T10:00:00Z" + ) + ), + new ListBasedInputRow( + signature, + DateTimes.of("2018-05-06T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableList.of( + 34L, + -2.0, + "foo", + 1L, + "true", + "2018-05-06T10:00:00Z" + ) + ), + new ListBasedInputRow( + signature, + DateTimes.of("2018-05-07T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableList.of( + 343L, + 3.0, + "bar", + ImmutableList.of(2L, 3L, 4L), + "false", + "2018-05-07T10:00:00Z" + ) + ), + new ListBasedInputRow( + signature, + DateTimes.of("2018-05-08T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableList.of( + 545L, + -4.0, + 7L, + 2L, + "true", + "2018-05-08T10:00:00Z" + ) + ), + new ListBasedInputRow( + signature, + DateTimes.of("2018-05-09T10:00:00Z"), + ImmutableList.of("Timestamp"), + ImmutableList.of( + 65L, + 5.0, + "baz", + ImmutableList.of(3L, 4L), + "false", + "2018-05-09T10:00:00Z" + ) + ) + ); + final DelimitedInputFormat format = new DelimitedInputFormat( + ImmutableList.of("Value1", "Value2", "Value3", "Value4", "Value5", "Timestamp"), + "|", + null, + false, + null, + 0, + true + ); + final InputEntityReader reader = format.createReader( + new InputRowSchema( + new TimestampSpec("Timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("Timestamp"))), + ColumnsFilter.all() + ), + source, + null + ); + + try (CloseableIterator iterator = reader.read()) { + final Iterator expectedRowIterator = expectedResults.iterator(); + while (iterator.hasNext()) { + Assert.assertTrue(expectedRowIterator.hasNext()); + Assert.assertEquals(expectedRowIterator.next(), iterator.next()); + } + } + } + @Test public void testCustomizeSeparator() throws IOException { @@ -173,7 +317,7 @@ public class DelimitedReaderTest "2019-01-01T00:00:30Z|name_3|15\t3" ) ); - final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), "\t", "|", null, true, 0); + final DelimitedInputFormat format = new DelimitedInputFormat(ImmutableList.of(), "\t", "|", null, true, 0, null); final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA, source, null); int numResults = 0; try (CloseableIterator iterator = reader.read()) { @@ -211,7 +355,8 @@ public class DelimitedReaderTest null, null, false, - 0 + 0, + null ); final InputEntityReader reader = format.createReader(INPUT_ROW_SCHEMA, source, null); try (CloseableIterator iterator = reader.read()) { @@ -256,6 +401,17 @@ public class DelimitedReaderTest StringUtils.format("name_%d", numResults + 1), Iterables.getOnlyElement(row.getDimension("name")) ); + if (format.shouldTryParseNumbers()) { + Assert.assertEquals(((numResults + 1) * 5L), row.getRaw("score")); + if (format.getColumns().contains("rating_dbl")) { + Assert.assertEquals(numResults * 1.0, row.getRaw("rating_dbl")); + } + } else { + Assert.assertEquals(Integer.toString((numResults + 1) * 5), row.getRaw("score")); + if (format.getColumns().contains("rating_dbl")) { + Assert.assertEquals(Double.toString(numResults * 1.0), row.getRaw("rating_dbl")); + } + } Assert.assertEquals( Integer.toString((numResults + 1) * 5), Iterables.getOnlyElement(row.getDimension("score")) diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/HttpEntityTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/HttpEntityTest.java index 59217456dbf..8776cd27573 100644 --- a/processing/src/test/java/org/apache/druid/data/input/impl/HttpEntityTest.java +++ b/processing/src/test/java/org/apache/druid/data/input/impl/HttpEntityTest.java @@ -19,7 +19,9 @@ package org.apache.druid.data.input.impl; +import com.google.common.collect.ImmutableMap; import com.google.common.net.HttpHeaders; +import com.sun.net.httpserver.Headers; import com.sun.net.httpserver.HttpServer; import org.apache.commons.io.IOUtils; import org.apache.druid.java.util.common.StringUtils; @@ -42,6 +44,8 @@ import java.net.URISyntaxException; import java.net.URL; import java.net.URLConnection; import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.Map; public class HttpEntityTest { @@ -96,8 +100,61 @@ public class HttpEntityTest server.start(); URI url = new URI("http://" + server.getAddress().getHostName() + ":" + server.getAddress().getPort() + "/test"); - inputStream = HttpEntity.openInputStream(url, "", null, 0); - inputStreamPartial = HttpEntity.openInputStream(url, "", null, 5); + inputStream = HttpEntity.openInputStream(url, "", null, 0, Collections.emptyMap()); + inputStreamPartial = HttpEntity.openInputStream(url, "", null, 5, Collections.emptyMap()); + inputStream.skip(5); + Assert.assertTrue(IOUtils.contentEquals(inputStream, inputStreamPartial)); + } + finally { + IOUtils.closeQuietly(inputStream); + IOUtils.closeQuietly(inputStreamPartial); + if (server != null) { + server.stop(0); + } + if (serverSocket != null) { + serverSocket.close(); + } + } + } + + @Test + public void testRequestHeaders() throws IOException, URISyntaxException + { + HttpServer server = null; + InputStream inputStream = null; + InputStream inputStreamPartial = null; + ServerSocket serverSocket = null; + Map requestHeaders = ImmutableMap.of("r-Cookie", "test", "Content-Type", "application/json"); + try { + serverSocket = new ServerSocket(0); + int port = serverSocket.getLocalPort(); + // closing port so that the httpserver can use. Can cause race conditions. + serverSocket.close(); + server = HttpServer.create(new InetSocketAddress("localhost", port), 0); + server.createContext( + "/test", + (httpExchange) -> { + Headers headers = httpExchange.getRequestHeaders(); + for (Map.Entry entry : requestHeaders.entrySet()) { + Assert.assertTrue(headers.containsKey(entry.getKey())); + Assert.assertEquals(headers.get(entry.getKey()).get(0), entry.getValue()); + } + String payload = "12345678910"; + byte[] outputBytes = payload.getBytes(StandardCharsets.UTF_8); + httpExchange.sendResponseHeaders(200, outputBytes.length); + OutputStream os = httpExchange.getResponseBody(); + httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_TYPE, "application/octet-stream"); + httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_LENGTH, String.valueOf(outputBytes.length)); + httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_RANGE, "bytes 0"); + os.write(outputBytes); + os.close(); + } + ); + server.start(); + + URI url = new URI("http://" + server.getAddress().getHostName() + ":" + server.getAddress().getPort() + "/test"); + inputStream = HttpEntity.openInputStream(url, "", null, 0, requestHeaders); + inputStreamPartial = HttpEntity.openInputStream(url, "", null, 5, requestHeaders); inputStream.skip(5); Assert.assertTrue(IOUtils.contentEquals(inputStream, inputStreamPartial)); } @@ -119,7 +176,7 @@ public class HttpEntityTest long offset = 15; String contentRange = StringUtils.format("bytes %d-%d/%d", offset, 1000, 1000); Mockito.when(urlConnection.getHeaderField(HttpHeaders.CONTENT_RANGE)).thenReturn(contentRange); - HttpEntity.openInputStream(uri, "", null, offset); + HttpEntity.openInputStream(uri, "", null, offset, Collections.emptyMap()); Mockito.verify(inputStreamMock, Mockito.times(0)).skip(offset); } @@ -128,7 +185,7 @@ public class HttpEntityTest { long offset = 15; Mockito.when(urlConnection.getHeaderField(HttpHeaders.CONTENT_RANGE)).thenReturn(null); - HttpEntity.openInputStream(uri, "", null, offset); + HttpEntity.openInputStream(uri, "", null, offset, Collections.emptyMap()); Mockito.verify(inputStreamMock, Mockito.times(1)).skip(offset); } @@ -137,7 +194,7 @@ public class HttpEntityTest { long offset = 15; Mockito.when(urlConnection.getHeaderField(HttpHeaders.CONTENT_RANGE)).thenReturn("token 2-12/12"); - HttpEntity.openInputStream(uri, "", null, offset); + HttpEntity.openInputStream(uri, "", null, offset, Collections.emptyMap()); Mockito.verify(inputStreamMock, Mockito.times(1)).skip(offset); } } diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/HttpInputSourceConfigTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/HttpInputSourceConfigTest.java index 7f88fc7bf62..74cc62b4d81 100644 --- a/processing/src/test/java/org/apache/druid/data/input/impl/HttpInputSourceConfigTest.java +++ b/processing/src/test/java/org/apache/druid/data/input/impl/HttpInputSourceConfigTest.java @@ -24,6 +24,8 @@ import nl.jqno.equalsverifier.EqualsVerifier; import org.junit.Assert; import org.junit.Test; +import java.util.Collections; + public class HttpInputSourceConfigTest { @Test @@ -35,21 +37,33 @@ public class HttpInputSourceConfigTest @Test public void testNullAllowedProtocolsUseDefault() { - HttpInputSourceConfig config = new HttpInputSourceConfig(null); + HttpInputSourceConfig config = new HttpInputSourceConfig(null, null); Assert.assertEquals(HttpInputSourceConfig.DEFAULT_ALLOWED_PROTOCOLS, config.getAllowedProtocols()); + Assert.assertEquals(Collections.emptySet(), config.getAllowedHeaders()); } @Test public void testEmptyAllowedProtocolsUseDefault() { - HttpInputSourceConfig config = new HttpInputSourceConfig(ImmutableSet.of()); + HttpInputSourceConfig config = new HttpInputSourceConfig(ImmutableSet.of(), null); Assert.assertEquals(HttpInputSourceConfig.DEFAULT_ALLOWED_PROTOCOLS, config.getAllowedProtocols()); } @Test public void testCustomAllowedProtocols() { - HttpInputSourceConfig config = new HttpInputSourceConfig(ImmutableSet.of("druid")); + HttpInputSourceConfig config = new HttpInputSourceConfig(ImmutableSet.of("druid"), null); Assert.assertEquals(ImmutableSet.of("druid"), config.getAllowedProtocols()); } + + @Test + public void testAllowedHeaders() + { + HttpInputSourceConfig config = new HttpInputSourceConfig( + ImmutableSet.of("druid"), + ImmutableSet.of("Content-Type", "Referer") + ); + Assert.assertEquals(ImmutableSet.of("druid"), config.getAllowedProtocols()); + Assert.assertEquals(ImmutableSet.of("content-type", "referer"), config.getAllowedHeaders()); + } } diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/HttpInputSourceTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/HttpInputSourceTest.java index bcd6152f05d..c11cb96f34d 100644 --- a/processing/src/test/java/org/apache/druid/data/input/impl/HttpInputSourceTest.java +++ b/processing/src/test/java/org/apache/druid/data/input/impl/HttpInputSourceTest.java @@ -22,11 +22,14 @@ package org.apache.druid.data.input.impl; import com.fasterxml.jackson.databind.InjectableValues.Std; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.systemfield.SystemField; import org.apache.druid.data.input.impl.systemfield.SystemFields; +import org.apache.druid.error.DruidException; import org.apache.druid.metadata.DefaultPasswordProvider; import org.junit.Assert; import org.junit.Rule; @@ -35,7 +38,9 @@ import org.junit.rules.ExpectedException; import java.io.IOException; import java.net.URI; +import java.util.Collections; import java.util.EnumSet; +import java.util.HashSet; public class HttpInputSourceTest { @@ -45,7 +50,7 @@ public class HttpInputSourceTest @Test public void testSerde() throws IOException { - HttpInputSourceConfig httpInputSourceConfig = new HttpInputSourceConfig(null); + HttpInputSourceConfig httpInputSourceConfig = new HttpInputSourceConfig(null, null); final ObjectMapper mapper = new ObjectMapper(); mapper.setInjectableValues(new Std().addValue(HttpInputSourceConfig.class, httpInputSourceConfig)); final HttpInputSource source = new HttpInputSource( @@ -53,6 +58,7 @@ public class HttpInputSourceTest "myName", new DefaultPasswordProvider("myPassword"), new SystemFields(EnumSet.of(SystemField.URI)), + null, httpInputSourceConfig ); final byte[] json = mapper.writeValueAsBytes(source); @@ -68,7 +74,8 @@ public class HttpInputSourceTest "myName", new DefaultPasswordProvider("myPassword"), null, - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ); new HttpInputSource( @@ -76,7 +83,8 @@ public class HttpInputSourceTest "myName", new DefaultPasswordProvider("myPassword"), null, - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ); expectedException.expect(IllegalArgumentException.class); @@ -86,19 +94,21 @@ public class HttpInputSourceTest "myName", new DefaultPasswordProvider("myPassword"), null, - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ); } @Test public void testConstructorAllowsOnlyCustomProtocols() { - final HttpInputSourceConfig customConfig = new HttpInputSourceConfig(ImmutableSet.of("druid")); + final HttpInputSourceConfig customConfig = new HttpInputSourceConfig(ImmutableSet.of("druid"), null); new HttpInputSource( ImmutableList.of(URI.create("druid:///")), "myName", new DefaultPasswordProvider("myPassword"), null, + null, customConfig ); @@ -109,6 +119,7 @@ public class HttpInputSourceTest "myName", new DefaultPasswordProvider("myPassword"), null, + null, customConfig ); } @@ -116,12 +127,13 @@ public class HttpInputSourceTest @Test public void testSystemFields() { - HttpInputSourceConfig httpInputSourceConfig = new HttpInputSourceConfig(null); + HttpInputSourceConfig httpInputSourceConfig = new HttpInputSourceConfig(null, null); final HttpInputSource inputSource = new HttpInputSource( ImmutableList.of(URI.create("http://test.com/http-test")), "myName", new DefaultPasswordProvider("myPassword"), new SystemFields(EnumSet.of(SystemField.URI, SystemField.PATH)), + null, httpInputSourceConfig ); @@ -130,10 +142,53 @@ public class HttpInputSourceTest inputSource.getConfiguredSystemFields() ); - final HttpEntity entity = new HttpEntity(URI.create("https://example.com/foo"), null, null); + final HttpEntity entity = new HttpEntity(URI.create("https://example.com/foo"), null, null, null); Assert.assertEquals("https://example.com/foo", inputSource.getSystemFieldValue(entity, SystemField.URI)); Assert.assertEquals("/foo", inputSource.getSystemFieldValue(entity, SystemField.PATH)); + Assert.assertEquals(inputSource.getRequestHeaders(), Collections.emptyMap()); + } + + @Test + public void testEmptyAllowedHeaders() + { + HttpInputSourceConfig httpInputSourceConfig = new HttpInputSourceConfig( + null, + new HashSet<>() + ); + expectedException.expect(DruidException.class); + expectedException.expectMessage( + "Got forbidden header [r-Cookie], allowed headers are only [[]]. " + + "You can control the allowed headers by updating druid.ingestion.http.allowedHeaders"); + + final HttpInputSource inputSource = new HttpInputSource( + ImmutableList.of(URI.create("http://test.com/http-test")), + "myName", + new DefaultPasswordProvider("myPassword"), + new SystemFields(EnumSet.of(SystemField.URI, SystemField.PATH)), + ImmutableMap.of("r-Cookie", "test", "Content-Type", "application/json"), + httpInputSourceConfig + ); + } + + @Test + public void shouldFailOnForbiddenHeaders() + { + HttpInputSourceConfig httpInputSourceConfig = new HttpInputSourceConfig( + null, + Sets.newHashSet("R-cookie", "Content-type") + ); + expectedException.expect(DruidException.class); + expectedException.expectMessage( + "Got forbidden header [G-Cookie], allowed headers are only [[r-cookie, content-type]]"); + new HttpInputSource( + ImmutableList.of(URI.create("http://test.com/http-test")), + "myName", + new DefaultPasswordProvider("myPassword"), + new SystemFields(EnumSet.of(SystemField.URI, SystemField.PATH)), + ImmutableMap.of("G-Cookie", "test", "Content-Type", "application/json"), + httpInputSourceConfig + ); } @Test diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java index 80602d0508a..ad1422126dd 100644 --- a/processing/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java +++ b/processing/src/test/java/org/apache/druid/data/input/impl/InputEntityIteratingReaderTest.java @@ -84,7 +84,8 @@ public class InputEntityIteratingReaderTest extends InitializedNullHandlingTest null, null, false, - 0 + 0, + null ), CloseableIterators.withEmptyBaggage( files.stream().flatMap(file -> ImmutableList.of(new FileEntity(file)).stream()).iterator() @@ -152,7 +153,8 @@ public class InputEntityIteratingReaderTest extends InitializedNullHandlingTest null, null, false, - 0 + 0, + null ), CloseableIterators.withEmptyBaggage( files.stream().flatMap(file -> ImmutableList.of(new FileEntity(file)).stream()).iterator() @@ -200,11 +202,12 @@ public class InputEntityIteratingReaderTest extends InitializedNullHandlingTest null, null, false, - 0 + 0, + null ), CloseableIterators.withEmptyBaggage( ImmutableList.of( - new HttpEntity(new URI("testscheme://some/path"), null, null) + new HttpEntity(new URI("testscheme://some/path"), null, null, null) { @Override protected int getMaxRetries() diff --git a/processing/src/test/java/org/apache/druid/frame/FrameTest.java b/processing/src/test/java/org/apache/druid/frame/FrameTest.java index 029e34bc011..7a26bbacd04 100644 --- a/processing/src/test/java/org/apache/druid/frame/FrameTest.java +++ b/processing/src/test/java/org/apache/druid/frame/FrameTest.java @@ -31,10 +31,10 @@ import org.apache.druid.frame.key.KeyOrder; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.java.util.common.ByteBufferUtils; import org.apache.druid.java.util.common.io.Closer; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -74,7 +74,7 @@ public class FrameTest @Before public void setUp() { - final StorageAdapter adapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + final CursorFactory cursorFactory = new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); final List sortBy = ImmutableList.of( new KeyColumn("quality", KeyOrder.DESCENDING), @@ -83,7 +83,7 @@ public class FrameTest columnarFrame = Iterables.getOnlyElement( FrameSequenceBuilder - .fromAdapter(adapter) + .fromCursorFactory(cursorFactory) .frameType(FrameType.COLUMNAR) .frames() .toList() @@ -91,7 +91,7 @@ public class FrameTest rowBasedSortedFrame = Iterables.getOnlyElement( FrameSequenceBuilder - .fromAdapter(adapter) + .fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) .sortBy(sortBy) .frames() @@ -318,9 +318,9 @@ public class FrameTest @BeforeClass public static void setUpClass() throws Exception { - final StorageAdapter adapter = new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final CursorFactory cursorFactory = new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final Frame frame = - Iterables.getOnlyElement(FrameSequenceBuilder.fromAdapter(adapter) + Iterables.getOnlyElement(FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.COLUMNAR) .frames() .toList()); @@ -401,8 +401,8 @@ public class FrameTest private static Frame makeGoodFrame() { - final StorageAdapter adapter = new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); - return Iterables.getOnlyElement(FrameSequenceBuilder.fromAdapter(adapter) + final CursorFactory cursorFactory = new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); + return Iterables.getOnlyElement(FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.COLUMNAR) .frames() .toList()); diff --git a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java b/processing/src/test/java/org/apache/druid/frame/TestArrayCursorFactory.java similarity index 89% rename from processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java rename to processing/src/test/java/org/apache/druid/frame/TestArrayCursorFactory.java index 173750a74c7..4bde3d31fe0 100644 --- a/processing/src/test/java/org/apache/druid/frame/TestArrayStorageAdapter.java +++ b/processing/src/test/java/org/apache/druid/frame/TestArrayCursorFactory.java @@ -19,7 +19,6 @@ package org.apache.druid.frame; -import com.google.common.collect.Iterables; import org.apache.druid.query.OrderBy; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; @@ -32,7 +31,7 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.ObjectColumnSelector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; @@ -44,13 +43,17 @@ import javax.annotation.Nullable; import java.util.List; /** - * Storage adapter around {@link QueryableIndex} that transforms all multi-value strings columns into string arrays. + * {@link org.apache.druid.segment.CursorFactory} around {@link QueryableIndex} that transforms all multi-value strings + * columns into string arrays. */ -public class TestArrayStorageAdapter extends QueryableIndexStorageAdapter +public class TestArrayCursorFactory extends QueryableIndexCursorFactory { - public TestArrayStorageAdapter(QueryableIndex index) + private final RowSignature signature; + + public TestArrayCursorFactory(QueryableIndex index) { super(index); + this.signature = computeRowSignature(index); } @Override @@ -81,26 +84,10 @@ public class TestArrayStorageAdapter extends QueryableIndexStorageAdapter }; } - @Override public RowSignature getRowSignature() { - final RowSignature.Builder builder = RowSignature.builder(); - builder.addTimeColumn(); - - for (final String column : Iterables.concat(super.getAvailableDimensions(), super.getAvailableMetrics())) { - ColumnCapabilities columnCapabilities = super.getColumnCapabilities(column); - ColumnType columnType = columnCapabilities == null ? null : columnCapabilities.toColumnType(); - //change MV strings columns to Array - if (columnType != null - && columnType.equals(ColumnType.STRING) - && columnCapabilities.hasMultipleValues().isMaybeTrue()) { - columnType = ColumnType.STRING_ARRAY; - } - builder.add(column, columnType); - } - - return builder.build(); + return signature; } @Nullable @@ -115,6 +102,26 @@ public class TestArrayStorageAdapter extends QueryableIndexStorageAdapter } } + private static RowSignature computeRowSignature(final QueryableIndex index) + { + final RowSignature.Builder builder = RowSignature.builder(); + builder.addTimeColumn(); + + for (final String column : new QueryableIndexCursorFactory(index).getRowSignature().getColumnNames()) { + ColumnCapabilities columnCapabilities = index.getColumnCapabilities(column); + ColumnType columnType = columnCapabilities == null ? null : columnCapabilities.toColumnType(); + //change MV strings columns to Array + if (columnType != null + && columnType.equals(ColumnType.STRING) + && columnCapabilities.hasMultipleValues().isMaybeTrue()) { + columnType = ColumnType.STRING_ARRAY; + } + builder.add(column, columnType); + } + + return builder.build(); + } + private class DecoratedCursor implements Cursor { private final Cursor cursor; @@ -188,7 +195,7 @@ public class TestArrayStorageAdapter extends QueryableIndexStorageAdapter @Override public ColumnCapabilities getColumnCapabilities(String column) { - return TestArrayStorageAdapter.this.getColumnCapabilities(column); + return TestArrayCursorFactory.this.getColumnCapabilities(column); } }; } diff --git a/processing/src/main/java/org/apache/druid/segment/AbstractSegment.java b/processing/src/test/java/org/apache/druid/frame/allocation/ArenaMemoryAllocatorTest.java similarity index 76% rename from processing/src/main/java/org/apache/druid/segment/AbstractSegment.java rename to processing/src/test/java/org/apache/druid/frame/allocation/ArenaMemoryAllocatorTest.java index c41be41ec4d..e1a637ea047 100644 --- a/processing/src/main/java/org/apache/druid/segment/AbstractSegment.java +++ b/processing/src/test/java/org/apache/druid/frame/allocation/ArenaMemoryAllocatorTest.java @@ -17,13 +17,13 @@ * under the License. */ -package org.apache.druid.segment; +package org.apache.druid.frame.allocation; -/** - * @deprecated use {@link Segment} directly as this does nothing - */ -@Deprecated -public abstract class AbstractSegment implements Segment +public class ArenaMemoryAllocatorTest extends BaseMemoryAllocatorTest { - // i used to have a purpose + @Override + protected MemoryAllocator makeAllocator(int capacity) + { + return ArenaMemoryAllocator.createOnHeap(capacity); + } } diff --git a/processing/src/test/java/org/apache/druid/frame/allocation/BaseMemoryAllocatorTest.java b/processing/src/test/java/org/apache/druid/frame/allocation/BaseMemoryAllocatorTest.java new file mode 100644 index 00000000000..cd3952accdc --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/allocation/BaseMemoryAllocatorTest.java @@ -0,0 +1,166 @@ +/* + * 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.allocation; + +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.collections.ResourceHolder; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Optional; + +/** + * Tests for {@link MemoryAllocator}, subclassed for each concrete implementation. + */ +public abstract class BaseMemoryAllocatorTest +{ + private static final int ALLOCATOR_SIZE = 10; + + protected abstract MemoryAllocator makeAllocator(int capacity); + + @Test + public void testAllocationInSinglePass() + { + MemoryAllocator memoryAllocator = makeAllocator(ALLOCATOR_SIZE); + Optional> memoryResourceHolderOptional = memoryAllocator.allocate(ALLOCATOR_SIZE); + Assert.assertTrue(memoryResourceHolderOptional.isPresent()); + ResourceHolder memoryResourceHolder = memoryResourceHolderOptional.get(); + WritableMemory memory = memoryResourceHolder.get(); + for (int i = 0; i < ALLOCATOR_SIZE; ++i) { + memory.putByte(i, (byte) 0xFF); + } + } + + @Test + public void testAllocationInMultiplePasses() + { + MemoryAllocator memoryAllocator = makeAllocator(ALLOCATOR_SIZE); + + Optional> memoryResourceHolderOptional1 = memoryAllocator.allocate(ALLOCATOR_SIZE + - 4); + Assert.assertTrue(memoryResourceHolderOptional1.isPresent()); + ResourceHolder memoryResourceHolder1 = memoryResourceHolderOptional1.get(); + WritableMemory memory1 = memoryResourceHolder1.get(); + + Optional> memoryResourceHolderOptional2 = memoryAllocator.allocate(4); + Assert.assertTrue(memoryResourceHolderOptional2.isPresent()); + ResourceHolder memoryResourceHolder2 = memoryResourceHolderOptional2.get(); + WritableMemory memory2 = memoryResourceHolder2.get(); + + for (int i = 0; i < ALLOCATOR_SIZE - 4; ++i) { + memory1.putByte(i, (byte) 0xFF); + } + for (int i = 0; i < 4; ++i) { + memory2.putByte(i, (byte) 0xFE); + } + // Readback to ensure that value hasn't been overwritten + for (int i = 0; i < ALLOCATOR_SIZE - 4; ++i) { + Assert.assertEquals((byte) 0xFF, memory1.getByte(i)); + } + for (int i = 0; i < 4; ++i) { + Assert.assertEquals((byte) 0xFE, memory2.getByte(i)); + } + } + + @Test + public void testReleaseAllocationTwice() + { + final MemoryAllocator memoryAllocator = makeAllocator(ALLOCATOR_SIZE); + final int allocationSize = 4; + + final Optional> holder1 = memoryAllocator.allocate(allocationSize); + final Optional> holder2 = memoryAllocator.allocate(allocationSize); + Assert.assertTrue(holder1.isPresent()); + Assert.assertTrue(holder2.isPresent()); + Assert.assertEquals(ALLOCATOR_SIZE - allocationSize * 2, memoryAllocator.available()); + + // Release the second allocation. + holder2.get().close(); + Assert.assertEquals(ALLOCATOR_SIZE - allocationSize, memoryAllocator.available()); + + // Release again-- does nothing. + holder2.get().close(); + Assert.assertEquals(ALLOCATOR_SIZE - allocationSize, memoryAllocator.available()); + + // Release the first allocation. + holder1.get().close(); + Assert.assertEquals(ALLOCATOR_SIZE, memoryAllocator.available()); + } + + @Test + public void testReleaseLastAllocationFirst() + { + final MemoryAllocator memoryAllocator = makeAllocator(ALLOCATOR_SIZE); + final int allocationSize = 4; + + final Optional> holder1 = memoryAllocator.allocate(allocationSize); + final Optional> holder2 = memoryAllocator.allocate(allocationSize); + Assert.assertTrue(holder1.isPresent()); + Assert.assertTrue(holder2.isPresent()); + Assert.assertEquals(ALLOCATOR_SIZE - allocationSize * 2, memoryAllocator.available()); + + // Release the second allocation. + holder2.get().close(); + Assert.assertEquals(ALLOCATOR_SIZE - allocationSize, memoryAllocator.available()); + + // Release the first allocation. + holder1.get().close(); + Assert.assertEquals(ALLOCATOR_SIZE, memoryAllocator.available()); + } + + @Test + public void testReleaseLastAllocationLast() + { + final MemoryAllocator memoryAllocator = makeAllocator(ALLOCATOR_SIZE); + final int allocationSize = 4; + + final Optional> holder1 = memoryAllocator.allocate(allocationSize); + final Optional> holder2 = memoryAllocator.allocate(allocationSize); + Assert.assertTrue(holder1.isPresent()); + Assert.assertTrue(holder2.isPresent()); + Assert.assertEquals(ALLOCATOR_SIZE - allocationSize * 2, memoryAllocator.available()); + + // Don't check memoryAllocator.available() after holder1 is closed; behavior is not consistent between arena + // and heap. Arena won't reclaim this allocation because it wasn't the final one; heap will reclaim it. + // They converge to fully-reclaimed once all allocations are closed. + holder1.get().close(); + holder2.get().close(); + Assert.assertEquals(ALLOCATOR_SIZE, memoryAllocator.available()); + } + + @Test + public void testOverallocationInSinglePass() + { + MemoryAllocator memoryAllocator = makeAllocator(ALLOCATOR_SIZE); + Optional> memoryResourceHolderOptional = + memoryAllocator.allocate(ALLOCATOR_SIZE + 1); + Assert.assertFalse(memoryResourceHolderOptional.isPresent()); + } + + @Test + public void testOverallocationInMultiplePasses() + { + MemoryAllocator memoryAllocator = makeAllocator(ALLOCATOR_SIZE); + Optional> memoryResourceHolderOptional = + memoryAllocator.allocate(ALLOCATOR_SIZE - 4); + Assert.assertTrue(memoryResourceHolderOptional.isPresent()); + Assert.assertFalse(memoryAllocator.allocate(5).isPresent()); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/allocation/HeapMemoryAllocatorTest.java b/processing/src/test/java/org/apache/druid/frame/allocation/HeapMemoryAllocatorTest.java index ac69eaee8a5..67f63d312f5 100644 --- a/processing/src/test/java/org/apache/druid/frame/allocation/HeapMemoryAllocatorTest.java +++ b/processing/src/test/java/org/apache/druid/frame/allocation/HeapMemoryAllocatorTest.java @@ -19,78 +19,11 @@ package org.apache.druid.frame.allocation; -import org.apache.datasketches.memory.WritableMemory; -import org.apache.druid.collections.ResourceHolder; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Optional; - -public class HeapMemoryAllocatorTest +public class HeapMemoryAllocatorTest extends BaseMemoryAllocatorTest { - private static final int ALLOCATOR_SIZE = 10; - - @Test - public void testAllocationInSinglePass() + @Override + protected MemoryAllocator makeAllocator(int capacity) { - MemoryAllocator heapMemoryAllocator = new HeapMemoryAllocator(ALLOCATOR_SIZE); - Optional> memoryResourceHolderOptional = heapMemoryAllocator.allocate(ALLOCATOR_SIZE); - Assert.assertTrue(memoryResourceHolderOptional.isPresent()); - ResourceHolder memoryResourceHolder = memoryResourceHolderOptional.get(); - WritableMemory memory = memoryResourceHolder.get(); - for (int i = 0; i < ALLOCATOR_SIZE; ++i) { - memory.putByte(i, (byte) 0xFF); - } + return new HeapMemoryAllocator(capacity); } - - @Test - public void testAllocationInMultiplePasses() - { - MemoryAllocator heapMemoryAllocator = new HeapMemoryAllocator(ALLOCATOR_SIZE); - - Optional> memoryResourceHolderOptional1 = heapMemoryAllocator.allocate(ALLOCATOR_SIZE - - 4); - Assert.assertTrue(memoryResourceHolderOptional1.isPresent()); - ResourceHolder memoryResourceHolder1 = memoryResourceHolderOptional1.get(); - WritableMemory memory1 = memoryResourceHolder1.get(); - - Optional> memoryResourceHolderOptional2 = heapMemoryAllocator.allocate(4); - Assert.assertTrue(memoryResourceHolderOptional2.isPresent()); - ResourceHolder memoryResourceHolder2 = memoryResourceHolderOptional2.get(); - WritableMemory memory2 = memoryResourceHolder2.get(); - - for (int i = 0; i < ALLOCATOR_SIZE - 4; ++i) { - memory1.putByte(i, (byte) 0xFF); - } - for (int i = 0; i < 4; ++i) { - memory2.putByte(i, (byte) 0xFE); - } - // Readback to ensure that value hasn't been overwritten - for (int i = 0; i < ALLOCATOR_SIZE - 4; ++i) { - Assert.assertEquals((byte) 0xFF, memory1.getByte(i)); - } - for (int i = 0; i < 4; ++i) { - Assert.assertEquals((byte) 0xFE, memory2.getByte(i)); - } - } - - @Test - public void testOverallocationInSinglePass() - { - MemoryAllocator heapMemoryAllocator = new HeapMemoryAllocator(ALLOCATOR_SIZE); - Optional> memoryResourceHolderOptional = - heapMemoryAllocator.allocate(ALLOCATOR_SIZE + 1); - Assert.assertFalse(memoryResourceHolderOptional.isPresent()); - } - - @Test - public void testOverallocationInMultiplePasses() - { - MemoryAllocator heapMemoryAllocator = new HeapMemoryAllocator(ALLOCATOR_SIZE); - Optional> memoryResourceHolderOptional = - heapMemoryAllocator.allocate(ALLOCATOR_SIZE - 4); - Assert.assertTrue(memoryResourceHolderOptional.isPresent()); - Assert.assertFalse(heapMemoryAllocator.allocate(5).isPresent()); - } - } diff --git a/processing/src/test/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannelTest.java b/processing/src/test/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannelTest.java index 18168a1776e..a81d3914b23 100644 --- a/processing/src/test/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannelTest.java +++ b/processing/src/test/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannelTest.java @@ -30,7 +30,7 @@ import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; import org.junit.Assert; @@ -118,17 +118,30 @@ public class ReadableByteChunksFrameChannelTest channel.close(); } + @Test + public void testAddChunkAfterDoneWriting() + { + try (final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test", false)) { + channel.doneWriting(); + + Assert.assertThrows( + ChannelClosedForWritesException.class, + () -> channel.addChunk(new byte[]{}) + ); + } + } + @Test public void testTruncatedFrameFile() throws IOException { final int allocatorSize = 64000; final int truncatedSize = 30000; // Holds two full columnar frames + one partial frame, after compression. - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .allocator(ArenaMemoryAllocator.create(ByteBuffer.allocate(allocatorSize))) .frameType(FrameType.COLUMNAR) // No particular reason to test with both frame types .frames(), @@ -171,11 +184,11 @@ public class ReadableByteChunksFrameChannelTest final int allocatorSize = 64000; final int errorAtBytePosition = 30000; // Holds two full frames + one partial frame, after compression. - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .allocator(ArenaMemoryAllocator.create(ByteBuffer.allocate(allocatorSize))) .frameType(FrameType.COLUMNAR) // No particular reason to test with both frame types .frames(), @@ -239,11 +252,11 @@ public class ReadableByteChunksFrameChannelTest public void testWriteFullyThenRead() throws IOException { // Create a frame file. - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .maxRowsPerFrame(maxRowsPerFrame) .frameType(frameType) .frames(), @@ -285,8 +298,8 @@ public class ReadableByteChunksFrameChannelTest } FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(cursorFactory.getRowSignature())) ); } @@ -294,11 +307,11 @@ public class ReadableByteChunksFrameChannelTest public void testWriteReadInterleaved() throws IOException { // Create a frame file. - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .maxRowsPerFrame(maxRowsPerFrame) .frameType(frameType) .frames(), @@ -364,8 +377,8 @@ public class ReadableByteChunksFrameChannelTest } FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(outChannel.readable(), FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(outChannel.readable(), FrameReader.create(cursorFactory.getRowSignature())) ); } diff --git a/processing/src/test/java/org/apache/druid/frame/channel/ReadableConcatFrameChannelTest.java b/processing/src/test/java/org/apache/druid/frame/channel/ReadableConcatFrameChannelTest.java index e86a7d634d0..c2958c4c10c 100644 --- a/processing/src/test/java/org/apache/druid/frame/channel/ReadableConcatFrameChannelTest.java +++ b/processing/src/test/java/org/apache/druid/frame/channel/ReadableConcatFrameChannelTest.java @@ -24,7 +24,7 @@ import org.apache.druid.frame.FrameType; 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.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Test; @@ -37,9 +37,9 @@ public class ReadableConcatFrameChannelTest extends InitializedNullHandlingTest @Test public void testChannel() throws Exception { - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex()); + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex()); final List frames = - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) .maxRowsPerFrame(11) .frames() @@ -58,8 +58,8 @@ public class ReadableConcatFrameChannelTest extends InitializedNullHandlingTest final ReadableConcatFrameChannel concatChannel = ReadableConcatFrameChannel.open(channels.iterator()); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(concatChannel, FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(concatChannel, FrameReader.create(cursorFactory.getRowSignature())) ); } } diff --git a/processing/src/test/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannelTest.java b/processing/src/test/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannelTest.java index b3c0ec205aa..ff9952216c3 100644 --- a/processing/src/test/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannelTest.java +++ b/processing/src/test/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannelTest.java @@ -29,7 +29,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -59,8 +59,8 @@ public class ReadableInputStreamFrameChannelTest extends InitializedNullHandling @Rule public ExpectedException expectedException = ExpectedException.none(); - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); ExecutorService executorService = Execs.singleThreaded("input-stream-fetcher-test"); @@ -76,10 +76,10 @@ public class ReadableInputStreamFrameChannelTest extends InitializedNullHandling ); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), FrameTestUtil.readRowsFromFrameChannel( readableInputStreamFrameChannel, - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ) ); Assert.assertTrue(readableInputStreamFrameChannel.isFinished()); @@ -101,7 +101,7 @@ public class ReadableInputStreamFrameChannelTest extends InitializedNullHandling Assert.assertEquals(FrameTestUtil.readRowsFromFrameChannel( readableInputStreamFrameChannel, - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ).toList().size(), 0); Assert.assertTrue(readableInputStreamFrameChannel.isFinished()); readableInputStreamFrameChannel.close(); @@ -127,7 +127,7 @@ public class ReadableInputStreamFrameChannelTest extends InitializedNullHandling () -> FrameTestUtil.readRowsFromFrameChannel( readableInputStreamFrameChannel, - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ).toList() ); @@ -143,11 +143,11 @@ public class ReadableInputStreamFrameChannelTest extends InitializedNullHandling final int allocatorSize = 64000; final int truncatedSize = 30000; // Holds two full frames + one partial frame, after compression. - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .allocator(ArenaMemoryAllocator.create(ByteBuffer.allocate(allocatorSize))) .frameType(FrameType.ROW_BASED) .frames(), @@ -173,7 +173,7 @@ public class ReadableInputStreamFrameChannelTest extends InitializedNullHandling Assert.assertEquals(FrameTestUtil.readRowsFromFrameChannel( readableInputStreamFrameChannel, - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ).toList().size(), 0); Assert.assertTrue(readableInputStreamFrameChannel.isFinished()); readableInputStreamFrameChannel.close(); @@ -199,7 +199,7 @@ public class ReadableInputStreamFrameChannelTest extends InitializedNullHandling Assert.assertEquals(FrameTestUtil.readRowsFromFrameChannel( readableInputStreamFrameChannel, - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ).toList().size(), 0); Assert.assertTrue(readableInputStreamFrameChannel.isFinished()); readableInputStreamFrameChannel.close(); @@ -222,10 +222,10 @@ public class ReadableInputStreamFrameChannelTest extends InitializedNullHandling expectedException.expect(ISE.class); expectedException.expectMessage("Found error while reading input stream"); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), FrameTestUtil.readRowsFromFrameChannel( readableInputStreamFrameChannel, - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ) ); Assert.assertTrue(readableInputStreamFrameChannel.isFinished()); @@ -260,7 +260,7 @@ public class ReadableInputStreamFrameChannelTest extends InitializedNullHandling { try { final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter).maxRowsPerFrame(10).frameType(FrameType.ROW_BASED).frames(), + FrameSequenceBuilder.fromCursorFactory(cursorFactory).maxRowsPerFrame(10).frameType(FrameType.ROW_BASED).frames(), temporaryFolder.newFile() ); return Files.newInputStream(file.toPath()); diff --git a/processing/src/test/java/org/apache/druid/frame/field/FieldReaderRACTest.java b/processing/src/test/java/org/apache/druid/frame/field/FieldReaderRACTest.java index 70ff7037b8c..601739b7fd5 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/FieldReaderRACTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/FieldReaderRACTest.java @@ -26,8 +26,9 @@ import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; import org.apache.druid.query.rowsandcols.concrete.RowBasedFrameRowsAndColumns; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.BaseColumn; @@ -49,10 +50,10 @@ public class FieldReaderRACTest extends InitializedNullHandlingTest public void testDataSet() throws IOException { final QueryableIndex index = TestIndex.getMMappedTestIndex(); - final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(index); - final Frame frame = FrameTestUtil.adapterToFrame(storageAdapter, FrameType.ROW_BASED); + final CursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + final Frame frame = FrameTestUtil.cursorFactoryToFrame(cursorFactory, FrameType.ROW_BASED); - final RowSignature siggy = storageAdapter.getRowSignature(); + final RowSignature siggy = cursorFactory.getRowSignature(); final RowBasedFrameRowsAndColumns rowBasedRAC = new RowBasedFrameRowsAndColumns(frame, siggy); for (String columnName : siggy.getColumnNames()) { diff --git a/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java b/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java index 06c160e6840..7b0c5f5407c 100644 --- a/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java +++ b/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java @@ -31,9 +31,9 @@ import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.http.client.response.ClientResponse; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -70,7 +70,7 @@ public class FrameFileHttpResponseHandlerTest extends InitializedNullHandlingTes private final int maxRowsPerFrame; - private StorageAdapter adapter; + private CursorFactory cursorFactory; private File file; private ReadableByteChunksFrameChannel channel; private FrameFileHttpResponseHandler handler; @@ -95,9 +95,9 @@ public class FrameFileHttpResponseHandlerTest extends InitializedNullHandlingTes @Before public void setUp() throws IOException { - adapter = new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + cursorFactory = new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .maxRowsPerFrame(maxRowsPerFrame) .frameType(FrameType.ROW_BASED) // No particular reason to test with both frame types .frames(), @@ -134,8 +134,8 @@ public class FrameFileHttpResponseHandlerTest extends InitializedNullHandlingTes channel.doneWriting(); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(cursorFactory.getRowSignature())) ); // Backpressure future resolves once channel is read. @@ -230,8 +230,8 @@ public class FrameFileHttpResponseHandlerTest extends InitializedNullHandlingTes channel.doneWriting(); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(cursorFactory.getRowSignature())) ); // Backpressure future resolves after channel is read. @@ -341,8 +341,8 @@ public class FrameFileHttpResponseHandlerTest extends InitializedNullHandlingTes channel.doneWriting(); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(cursorFactory.getRowSignature())) ); } @@ -421,8 +421,8 @@ public class FrameFileHttpResponseHandlerTest extends InitializedNullHandlingTes Assert.assertEquals(allBytes.length, channel.getBytesAdded()); channel.doneWriting(); FrameTestUtil.assertRowsEqual( - FrameTestUtil.readRowsFromAdapter(adapter, null, false), - FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(adapter.getRowSignature())) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory), + FrameTestUtil.readRowsFromFrameChannel(channel, FrameReader.create(cursorFactory.getRowSignature())) ); } diff --git a/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java b/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java index 79cbe74ba28..c916a458564 100644 --- a/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java +++ b/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java @@ -23,25 +23,23 @@ import com.google.common.math.IntMath; import it.unimi.dsi.fastutil.ints.IntObjectPair; import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; -import org.apache.druid.frame.TestArrayStorageAdapter; +import org.apache.druid.frame.TestArrayCursorFactory; import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.frame.testutil.FrameTestUtil; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; 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.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.RowAdapters; -import org.apache.druid.segment.RowBasedSegment; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.RowBasedCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; -import org.apache.druid.timeline.SegmentId; import org.hamcrest.Matchers; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Assume; import org.junit.Before; @@ -52,17 +50,28 @@ import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; +import java.io.OutputStream; import java.math.RoundingMode; +import java.nio.file.Files; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.Objects; import java.util.function.Function; import java.util.stream.IntStream; @RunWith(Parameterized.class) public class FrameFileTest extends InitializedNullHandlingTest { + /** + * Static cache of generated frame files, to speed up tests. Cleared in {@link #afterClass()}. + */ + private static final Map FRAME_FILES = new HashMap<>(); + // Partition every 99 rows if "partitioned" is true. private static final int PARTITION_SIZE = 99; @@ -73,39 +82,60 @@ public class FrameFileTest extends InitializedNullHandlingTest { INCREMENTAL { @Override - StorageAdapter getAdapter() + CursorFactory getCursorFactory() { - return new IncrementalIndexStorageAdapter(TestIndex.getNoRollupIncrementalTestIndex()); + return new IncrementalIndexCursorFactory(TestIndex.getNoRollupIncrementalTestIndex()); + } + + @Override + int getRowCount() + { + return TestIndex.getNoRollupIncrementalTestIndex().size(); } }, MMAP { @Override - StorageAdapter getAdapter() + CursorFactory getCursorFactory() { - return new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + return new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); + } + + @Override + int getRowCount() + { + return TestIndex.getNoRollupMMappedTestIndex().getNumRows(); } }, MV_AS_STRING_ARRAYS { @Override - StorageAdapter getAdapter() + CursorFactory getCursorFactory() { - return new TestArrayStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + return new TestArrayCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); + } + + @Override + int getRowCount() + { + return TestIndex.getNoRollupMMappedTestIndex().getNumRows(); } }, EMPTY { @Override - StorageAdapter getAdapter() + CursorFactory getCursorFactory() { - return new RowBasedSegment<>( - SegmentId.dummy("EMPTY"), - Sequences.empty(), - RowAdapters.standardRow(), - RowSignature.empty() - ).asStorageAdapter(); + return new RowBasedCursorFactory<>(Sequences.empty(), RowAdapters.standardRow(), RowSignature.empty()); + } + + @Override + int getRowCount() + { + return 0; } }; - abstract StorageAdapter getAdapter(); + abstract CursorFactory getCursorFactory(); + + abstract int getRowCount(); } @Rule @@ -120,7 +150,8 @@ public class FrameFileTest extends InitializedNullHandlingTest private final AdapterType adapterType; private final int maxMmapSize; - private StorageAdapter adapter; + private CursorFactory cursorFactory; + private int rowCount; private File file; public FrameFileTest( @@ -175,39 +206,23 @@ public class FrameFileTest extends InitializedNullHandlingTest @Before public void setUp() throws IOException { - adapter = adapterType.getAdapter(); + cursorFactory = adapterType.getCursorFactory(); + rowCount = adapterType.getRowCount(); + file = temporaryFolder.newFile(); - if (partitioned) { - // Partition every PARTITION_SIZE rows. - file = FrameTestUtil.writeFrameFileWithPartitions( - FrameSequenceBuilder.fromAdapter(adapter).frameType(frameType).maxRowsPerFrame(maxRowsPerFrame).frames().map( - new Function>() - { - private int rows = 0; - - @Override - public IntObjectPair apply(final Frame frame) - { - final int partitionNum = rows / PARTITION_SIZE; - rows += frame.numRows(); - return IntObjectPair.of( - partitionNum >= SKIP_PARTITION ? partitionNum + 1 : partitionNum, - frame - ); - } - } - ), - temporaryFolder.newFile() - ); - - } else { - file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter).frameType(frameType).maxRowsPerFrame(maxRowsPerFrame).frames(), - temporaryFolder.newFile() - ); + try (final OutputStream out = Files.newOutputStream(file.toPath())) { + final FrameFileKey frameFileKey = new FrameFileKey(adapterType, frameType, maxRowsPerFrame, partitioned); + final byte[] frameFileBytes = FRAME_FILES.computeIfAbsent(frameFileKey, FrameFileTest::computeFrameFile); + out.write(frameFileBytes); } } + @AfterClass + public static void afterClass() + { + FRAME_FILES.clear(); + } + @Test public void test_numFrames() throws IOException { @@ -232,7 +247,7 @@ public class FrameFileTest extends InitializedNullHandlingTest Assume.assumeThat(frameFile.numFrames(), Matchers.greaterThan(0)); final Frame firstFrame = frameFile.frame(0); - Assert.assertEquals(Math.min(adapter.getNumRows(), maxRowsPerFrame), firstFrame.numRows()); + Assert.assertEquals(Math.min(rowCount, maxRowsPerFrame), firstFrame.numRows()); } } @@ -245,9 +260,9 @@ public class FrameFileTest extends InitializedNullHandlingTest final Frame lastFrame = frameFile.frame(frameFile.numFrames() - 1); Assert.assertEquals( - adapter.getNumRows() % maxRowsPerFrame != 0 - ? adapter.getNumRows() % maxRowsPerFrame - : Math.min(adapter.getNumRows(), maxRowsPerFrame), + rowCount % maxRowsPerFrame != 0 + ? rowCount % maxRowsPerFrame + : Math.min(rowCount, maxRowsPerFrame), lastFrame.numRows() ); } @@ -274,20 +289,20 @@ public class FrameFileTest extends InitializedNullHandlingTest } @Test - public void test_frame_readAllDataViaStorageAdapter() throws IOException + public void test_frame_readAllDataViaCursorFactory() throws IOException { - final FrameReader frameReader = FrameReader.create(adapter.getRowSignature()); + final FrameReader frameReader = FrameReader.create(cursorFactory.getRowSignature()); try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize, null)) { final Sequence> frameFileRows = Sequences.concat( () -> IntStream.range(0, frameFile.numFrames()) .mapToObj(frameFile::frame) - .map(frame -> new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY)) - .map(adapter -> FrameTestUtil.readRowsFromAdapter(adapter, null, true)) + .map(frameReader::makeCursorFactory) + .map(FrameTestUtil::readRowsFromCursorFactoryWithRowNumber) .iterator() ); - final Sequence> adapterRows = FrameTestUtil.readRowsFromAdapter(adapter, null, true); + final Sequence> adapterRows = FrameTestUtil.readRowsFromCursorFactoryWithRowNumber(cursorFactory); FrameTestUtil.assertRowsEqual(adapterRows, frameFileRows); } } @@ -373,7 +388,7 @@ public class FrameFileTest extends InitializedNullHandlingTest private int computeExpectedNumFrames() { - return IntMath.divide(countRows(adapter), maxRowsPerFrame, RoundingMode.CEILING); + return IntMath.divide(countRows(cursorFactory), maxRowsPerFrame, RoundingMode.CEILING); } private int computeExpectedNumPartitions() @@ -381,7 +396,7 @@ public class FrameFileTest extends InitializedNullHandlingTest if (partitioned) { return Math.min( computeExpectedNumFrames(), - IntMath.divide(countRows(adapter), PARTITION_SIZE, RoundingMode.CEILING) + IntMath.divide(countRows(cursorFactory), PARTITION_SIZE, RoundingMode.CEILING) ); } else { // 0 = not partitioned. @@ -389,10 +404,113 @@ public class FrameFileTest extends InitializedNullHandlingTest } } - private static int countRows(final StorageAdapter adapter) + private static int countRows(final CursorFactory cursorFactory) { - // Not using adapter.getNumRows(), because RowBasedStorageAdapter doesn't support it. - return FrameTestUtil.readRowsFromAdapter(adapter, RowSignature.empty(), false) + // Not using adapter.getNumRows(), because RowBasedCursorFactory doesn't support it. + return FrameTestUtil.readRowsFromCursorFactory(cursorFactory, RowSignature.empty(), false) .accumulate(0, (i, in) -> i + 1); } + + /** + * Returns bytes, in frame file format, corresponding to the given {@link FrameFileKey}. + */ + private static byte[] computeFrameFile(final FrameFileKey frameFileKey) + { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + try { + if (frameFileKey.partitioned) { + // Partition every PARTITION_SIZE rows. + FrameTestUtil.writeFrameFileWithPartitions( + FrameSequenceBuilder.fromCursorFactory(frameFileKey.adapterType.getCursorFactory()) + .frameType(frameFileKey.frameType) + .maxRowsPerFrame(frameFileKey.maxRowsPerFrame) + .frames() + .map( + new Function>() + { + private int rows = 0; + + @Override + public IntObjectPair apply(final Frame frame) + { + final int partitionNum = rows / PARTITION_SIZE; + rows += frame.numRows(); + return IntObjectPair.of( + partitionNum >= SKIP_PARTITION ? partitionNum + 1 : partitionNum, + frame + ); + } + } + ), + baos + ); + } else { + FrameTestUtil.writeFrameFile( + FrameSequenceBuilder.fromCursorFactory(frameFileKey.adapterType.getCursorFactory()) + .frameType(frameFileKey.frameType) + .maxRowsPerFrame(frameFileKey.maxRowsPerFrame) + .frames(), + baos + ); + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + + return baos.toByteArray(); + } + + /** + * Key for {@link #FRAME_FILES}, and input to {@link #computeFrameFile(FrameFileKey)}. + */ + private static class FrameFileKey + { + final AdapterType adapterType; + final FrameType frameType; + final int maxRowsPerFrame; + final boolean partitioned; + + public FrameFileKey(AdapterType adapterType, FrameType frameType, int maxRowsPerFrame, boolean partitioned) + { + this.adapterType = adapterType; + this.frameType = frameType; + this.maxRowsPerFrame = maxRowsPerFrame; + this.partitioned = partitioned; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FrameFileKey that = (FrameFileKey) o; + return maxRowsPerFrame == that.maxRowsPerFrame + && partitioned == that.partitioned + && adapterType == that.adapterType + && frameType == that.frameType; + } + + @Override + public int hashCode() + { + return Objects.hash(adapterType, frameType, maxRowsPerFrame, partitioned); + } + + @Override + public String toString() + { + return "FrameFileKey{" + + "adapterType=" + adapterType + + ", frameType=" + frameType + + ", maxRowsPerFrame=" + maxRowsPerFrame + + ", partitioned=" + partitioned + + '}'; + } + } } diff --git a/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java b/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java index 972371d0fde..58b3a4e0920 100644 --- a/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java @@ -26,7 +26,7 @@ import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -49,7 +49,7 @@ public class FrameFileWriterTest extends InitializedNullHandlingTest @Test public void test_abort_afterAllFrames() throws IOException { - final Sequence frames = FrameSequenceBuilder.fromAdapter(new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex())) + final Sequence frames = FrameSequenceBuilder.fromCursorFactory(new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex())) .allocator(ArenaMemoryAllocator.createOnHeap(1000000)) .frameType(FrameType.ROW_BASED) .frames(); diff --git a/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java b/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java index c53ca04a909..9e0547cae43 100644 --- a/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java +++ b/processing/src/test/java/org/apache/druid/frame/key/FrameComparisonWidgetImplTest.java @@ -26,8 +26,8 @@ import org.apache.druid.frame.FrameType; import org.apache.druid.frame.field.FieldReaders; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.RowBasedSegment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; @@ -50,7 +50,7 @@ public class FrameComparisonWidgetImplTest extends InitializedNullHandlingTest @Before public void setUp() { - final StorageAdapter rowBasedAdapterWithoutComplexColumn = new RowBasedSegment<>( + final CursorFactory rowBasedAdapterWithoutComplexColumn = new RowBasedSegment<>( SegmentId.dummy("test"), Sequences.simple(ByteRowKeyComparatorTest.KEY_OBJECTS_WITHOUT_COMPLEX_COLUMN), columnName -> { @@ -62,16 +62,16 @@ public class FrameComparisonWidgetImplTest extends InitializedNullHandlingTest } }, ByteRowKeyComparatorTest.NO_COMPLEX_SIGNATURE - ).asStorageAdapter(); + ).asCursorFactory(); frameWithoutComplexColumns = Iterables.getOnlyElement( - FrameSequenceBuilder.fromAdapter(rowBasedAdapterWithoutComplexColumn) + FrameSequenceBuilder.fromCursorFactory(rowBasedAdapterWithoutComplexColumn) .frameType(FrameType.ROW_BASED) .frames() .toList() ); - final StorageAdapter rowBasedAdapterWithComplexColumn = new RowBasedSegment<>( + final CursorFactory rowBasedAdapterWithComplexColumn = new RowBasedSegment<>( SegmentId.dummy("test"), Sequences.simple(ByteRowKeyComparatorTest.ALL_KEY_OBJECTS), columnName -> { @@ -83,10 +83,10 @@ public class FrameComparisonWidgetImplTest extends InitializedNullHandlingTest } }, ByteRowKeyComparatorTest.SIGNATURE - ).asStorageAdapter(); + ).asCursorFactory(); frameWithComplexColumns = Iterables.getOnlyElement( - FrameSequenceBuilder.fromAdapter(rowBasedAdapterWithComplexColumn) + FrameSequenceBuilder.fromCursorFactory(rowBasedAdapterWithComplexColumn) .frameType(FrameType.ROW_BASED) .frames() .toList() diff --git a/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java index 5600389f57b..f594b73368c 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java @@ -93,7 +93,7 @@ public class ComposingOutputChannelFactoryTest extends OutputChannelFactoryTest writableFrameChannel.write(new FrameWithPartition(frame, 1)); writableFrameChannel.close(); - verifySingleFrameReadableChannel(channel.getReadableChannel(), sourceAdapter); + verifySingleFrameReadableChannel(channel.getReadableChannel(), sourceCursorFactory); } diff --git a/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java b/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java index 55e5f7bb81c..4ed2c610525 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java @@ -45,10 +45,11 @@ import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.utils.CloseableUtils; import org.hamcrest.CoreMatchers; @@ -116,9 +117,9 @@ public class FrameProcessorExecutorTest { // 3 input files blasted to 2 outputs (2 copies of the data), then muxed to one file. - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); - final List inFiles = writeToNFiles(adapter, 3); + final IncrementalIndex index = TestIndex.getIncrementalTestIndex(); + final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + final List inFiles = writeToNFiles(cursorFactory, 3); final File outFile = temporaryFolder.newFile(); final BlockingQueueFrameChannel memoryChannel1 = BlockingQueueFrameChannel.minimal(); @@ -144,14 +145,14 @@ public class FrameProcessorExecutorTest final ListenableFuture blasterFuture = exec.runFully(blaster, null); final ListenableFuture muxerFuture = exec.runFully(muxer, null); - Assert.assertEquals(adapter.getNumRows(), (long) blasterFuture.get()); - Assert.assertEquals(adapter.getNumRows() * 2, (long) muxerFuture.get()); + Assert.assertEquals(index.size(), (long) blasterFuture.get()); + Assert.assertEquals(index.size() * 2, (long) muxerFuture.get()); Assert.assertEquals( - adapter.getNumRows() * 2, + index.size() * 2, FrameTestUtil.readRowsFromFrameChannel( new ReadableFileFrameChannel(FrameFile.open(outFile, null)), - FrameReader.create(adapter.getRowSignature()) + FrameReader.create(cursorFactory.getRowSignature()) ).toList().size() ); } @@ -180,9 +181,9 @@ public class FrameProcessorExecutorTest @Test public void test_runFully_errors() throws Exception { - final IncrementalIndexStorageAdapter adapter = - new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); - final File inFile = Iterables.getOnlyElement(writeToNFiles(adapter, 1)); + final IncrementalIndexCursorFactory cursorFactory = + new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); + final File inFile = Iterables.getOnlyElement(writeToNFiles(cursorFactory, 1)); final ReadableFrameChannel inChannel = openFileChannel(inFile); final BlockingQueueFrameChannel outChannel = BlockingQueueFrameChannel.minimal(); @@ -221,6 +222,7 @@ public class FrameProcessorExecutorTest final SettableFuture future = SettableFuture.create(); final String cancellationId = "xyzzy"; + exec.registerCancellationId(cancellationId); Assert.assertSame(future, exec.registerCancelableFuture(future, false, cancellationId)); exec.cancel(cancellationId); @@ -235,6 +237,8 @@ public class FrameProcessorExecutorTest { final SleepyFrameProcessor processor = new SleepyFrameProcessor(); final String cancellationId = "xyzzy"; + + exec.registerCancellationId(cancellationId); final ListenableFuture future = exec.runFully(processor, cancellationId); processor.awaitRun(); @@ -253,6 +257,8 @@ public class FrameProcessorExecutorTest { final SleepyFrameProcessor processor = new SleepyFrameProcessor(); final String cancellationId = "xyzzy"; + + exec.registerCancellationId(cancellationId); final ListenableFuture future = exec.runFully(processor, cancellationId); processor.awaitRun(); @@ -274,7 +280,7 @@ public class FrameProcessorExecutorTest // Doesn't matter what's in this frame. final Frame frame = Iterables.getOnlyElement( - FrameSequenceBuilder.fromAdapter(new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex())) + FrameSequenceBuilder.fromCursorFactory(new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex())) .frameType(FrameType.ROW_BASED) .frames() .toList() @@ -313,6 +319,8 @@ public class FrameProcessorExecutorTest // Start up all systems at once. for (final String systemId : systemGeneratorsMap.keySet()) { + exec.registerCancellationId(systemId); + for (InfiniteFrameProcessor generator : systemGeneratorsMap.get(systemId)) { processorFutureMap.put(generator, exec.runFully(generator, systemId)); } @@ -390,6 +398,22 @@ public class FrameProcessorExecutorTest // Just making sure no error is thrown when we refer to a nonexistent cancellationId. exec.cancel("nonexistent"); } + + @Test + public void test_runFully_nonexistentCancellationId() + { + final SleepyFrameProcessor processor = new SleepyFrameProcessor(); + final String cancellationId = "xyzzy"; + + // Don't registerCancellationId(cancellationId). + final ListenableFuture future = exec.runFully(processor, cancellationId); + + // Future should be immediately canceled, without running the processor. + Assert.assertTrue(future.isDone()); + Assert.assertTrue(future.isCancelled()); + Assert.assertFalse(processor.didGetInterrupt()); + Assert.assertFalse(processor.didCleanup()); + } } public abstract static class BaseFrameProcessorExecutorTestSuite extends InitializedNullHandlingTest @@ -427,7 +451,7 @@ public class FrameProcessorExecutorTest } } - List writeToNFiles(final StorageAdapter adapter, final int numFiles) throws IOException + List writeToNFiles(final CursorFactory cursorFactory, final int numFiles) throws IOException { final List files = new ArrayList<>(); final List writers = new ArrayList<>(); @@ -465,7 +489,7 @@ public class FrameProcessorExecutorTest }; FrameSequenceBuilder - .fromAdapter(adapter) + .fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) .allocator(ArenaMemoryAllocator.createOnHeap(1_000_000)) .maxRowsPerFrame(3) diff --git a/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java index 1e65ed7142a..22ad907bec6 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java @@ -27,15 +27,13 @@ import org.apache.druid.frame.channel.PartitionedReadableFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.WritableFrameChannel; import org.apache.druid.frame.read.FrameReader; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.frame.testutil.FrameTestUtil; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -49,15 +47,15 @@ public abstract class OutputChannelFactoryTest extends InitializedNullHandlingTe { private final OutputChannelFactory outputChannelFactory; private final long frameSize; - protected final StorageAdapter sourceAdapter; + protected final CursorFactory sourceCursorFactory; protected final Frame frame; public OutputChannelFactoryTest(OutputChannelFactory outputChannelFactory, long frameSize) { this.outputChannelFactory = outputChannelFactory; this.frameSize = frameSize; - this.sourceAdapter = new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); - this.frame = Iterables.getOnlyElement(FrameSequenceBuilder.fromAdapter(sourceAdapter) + this.sourceCursorFactory = new IncrementalIndexCursorFactory(TestIndex.getIncrementalTestIndex()); + this.frame = Iterables.getOnlyElement(FrameSequenceBuilder.fromCursorFactory(sourceCursorFactory) .frameType(FrameType.COLUMNAR) .frames() .toList()); @@ -79,7 +77,7 @@ public abstract class OutputChannelFactoryTest extends InitializedNullHandlingTe // read back data from the channel verifySingleFrameReadableChannel( channel.getReadableChannel(), - sourceAdapter + sourceCursorFactory ); Assert.assertEquals(frameSize, channel.getFrameMemoryAllocator().capacity()); } @@ -103,7 +101,7 @@ public abstract class OutputChannelFactoryTest extends InitializedNullHandlingTe for (int partition : partitions) { verifySingleFrameReadableChannel( partitionedReadableFrameChannelSupplier.get().getReadableFrameChannel(partition), - sourceAdapter + sourceCursorFactory ); Assert.assertEquals(frameSize, channel.getFrameMemoryAllocator().capacity()); } @@ -111,7 +109,7 @@ public abstract class OutputChannelFactoryTest extends InitializedNullHandlingTe protected void verifySingleFrameReadableChannel( ReadableFrameChannel readableFrameChannel, - StorageAdapter adapter + CursorFactory cursorFactory ) throws ExecutionException, InterruptedException { readableFrameChannel.readabilityFuture().get(); @@ -128,22 +126,18 @@ public abstract class OutputChannelFactoryTest extends InitializedNullHandlingTe Assert.assertTrue(readableFrameChannel.isFinished()); readableFrameChannel.close(); - FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter( - readbackFrame, - FrameReader.create(adapter.getRowSignature()), - Intervals.ETERNITY - ); + CursorFactory frameCursorFactory = FrameReader.create(cursorFactory.getRowSignature()).makeCursorFactory(readbackFrame); // build list of rows from written and read data to verify - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN); - final CursorHolder frameMaker = frameStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN) + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN); + final CursorHolder frameHolder = frameCursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN) ) { List> writtenData = FrameTestUtil.readRowsFromCursor( cursorHolder.asCursor(), - adapter.getRowSignature() + cursorFactory.getRowSignature() ).toList(); List> readData = FrameTestUtil.readRowsFromCursor( - frameMaker.asCursor(), - frameStorageAdapter.getRowSignature() + frameHolder.asCursor(), + frameCursorFactory.getRowSignature() ).toList(); Assert.assertEquals( diff --git a/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java b/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java index 9025d182086..88bb7cb07e8 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/ReadableFileFrameChannelTest.java @@ -26,8 +26,8 @@ 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.guava.Sequences; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; @@ -55,16 +55,16 @@ public class ReadableFileFrameChannelTest extends InitializedNullHandlingTest @Before public void setUp() throws IOException { - final StorageAdapter adapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + final CursorFactory cursorFactory = new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); final File file = FrameTestUtil.writeFrameFile( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) .maxRowsPerFrame(ROWS_PER_FRAME) .frames(), temporaryFolder.newFile() ); - allRows = FrameTestUtil.readRowsFromAdapter(adapter, adapter.getRowSignature(), false).toList(); - frameReader = FrameReader.create(adapter.getRowSignature()); + allRows = FrameTestUtil.readRowsFromCursorFactory(cursorFactory).toList(); + frameReader = FrameReader.create(cursorFactory.getRowSignature()); frameFile = FrameFile.open(file, null, FrameFile.Flag.DELETE_ON_CLOSE); } 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 b82aed94ef8..d0ae5a986a0 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 @@ -41,7 +41,7 @@ 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.Sequences; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -160,7 +160,7 @@ public class RunAllFullyWidgetTest extends FrameProcessorExecutorTest.BaseFrameP // Doesn't matter what's in this frame. final Frame frame = Iterables.getOnlyElement( - FrameSequenceBuilder.fromAdapter(new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex())) + FrameSequenceBuilder.fromCursorFactory(new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex())) .frameType(FrameType.ROW_BASED) .frames() .toList() @@ -409,6 +409,8 @@ public class RunAllFullyWidgetTest extends FrameProcessorExecutorTest.BaseFrameP .mapToObj(i -> new SleepyFrameProcessor()) .collect(Collectors.toList()); + final String cancellationId = "xyzzy"; + exec.registerCancellationId(cancellationId); final ListenableFuture future = exec.runAllFully( possiblyDelay( ensureClose( @@ -418,7 +420,7 @@ public class RunAllFullyWidgetTest extends FrameProcessorExecutorTest.BaseFrameP ), maxOutstandingProcessors, bouncer, - "xyzzy" + cancellationId ); for (int i = 0; i < expectedRunningProcessors; i++) { diff --git a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java index b40fde04a9a..80e7f6352d0 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java @@ -29,13 +29,7 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; import org.apache.druid.frame.channel.BlockingQueueFrameChannel; -import org.apache.druid.frame.channel.ByteTracker; -import org.apache.druid.frame.channel.ReadableFileFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; -import org.apache.druid.frame.channel.WritableFrameChannel; -import org.apache.druid.frame.channel.WritableFrameFileChannel; -import org.apache.druid.frame.file.FrameFile; -import org.apache.druid.frame.file.FrameFileWriter; import org.apache.druid.frame.key.ClusterBy; import org.apache.druid.frame.key.ClusterByPartition; import org.apache.druid.frame.key.ClusterByPartitions; @@ -44,17 +38,20 @@ import org.apache.druid.frame.key.KeyOrder; import org.apache.druid.frame.key.KeyTestUtils; import org.apache.druid.frame.key.RowKey; import org.apache.druid.frame.key.RowKeyReader; +import org.apache.druid.frame.processor.test.AlwaysAsyncPartitionedReadableFrameChannel; +import org.apache.druid.frame.processor.test.AlwaysAsyncReadableFrameChannel; 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.frame.write.FrameWriters; import org.apache.druid.java.util.common.ISE; 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.guava.Sequences; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.RowSignature; @@ -62,8 +59,10 @@ import org.apache.druid.testing.InitializedNullHandlingTest; import org.hamcrest.MatcherAssert; import org.hamcrest.Matchers; import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -73,12 +72,12 @@ import org.junit.runners.Parameterized; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.channels.Channels; -import java.nio.file.Files; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -228,6 +227,15 @@ public class SuperSorterTest @RunWith(Parameterized.class) public static class ParameterizedCasesTest extends InitializedNullHandlingTest { + private static CursorFactory CURSOR_FACTORY; + private static RowSignature CURSOR_FACTORY_SIGNATURE_WITH_ROW_NUMBER; + + /** + * Static cache of sorted versions of the {@link #CURSOR_FACTORY} dataset, to speed up tests. + * Cleared in {@link #tearDownClass()}. + */ + private static final Map>> SORTED_TEST_ROWS = new HashMap<>(); + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -241,7 +249,6 @@ public class SuperSorterTest private final boolean partitionsDeferred; private final long limitHint; - private StorageAdapter adapter; private RowSignature signature; private FrameProcessorExecutor exec; private List inputChannels; @@ -285,11 +292,12 @@ public class SuperSorterTest { final List constructors = new ArrayList<>(); - for (int maxRowsPerFrame : new int[]{Integer.MAX_VALUE, 50, 1}) { + // Add some constructors for testing maxRowsPerFrame > 1. Later on, we'll add some for maxRowsPerFrame = 1. + for (int maxRowsPerFrame : new int[]{Integer.MAX_VALUE, 50}) { for (int maxBytesPerFrame : new int[]{20_000, 2_000_000}) { for (int numChannels : new int[]{1, 3}) { - for (int maxActiveProcessors : new int[]{1, 2, 4}) { - for (int maxChannelsPerProcessor : new int[]{2, 3, 8}) { + for (int maxActiveProcessors : new int[]{1, 3}) { + for (int maxChannelsPerProcessor : new int[]{2, 7}) { for (int numThreads : new int[]{1, 3}) { for (boolean isComposedStorage : new boolean[]{true, false}) { for (boolean partitionsDeferred : new boolean[]{true, false}) { @@ -317,16 +325,51 @@ public class SuperSorterTest } } + // Add some constructors for testing maxRowsPerFrame = 1. This isn't part of the full matrix since it's quite + // slow, but we still want to exercise it a bit. + for (boolean isComposedStorage : new boolean[]{true, false}) { + for (long limitHint : new long[]{SuperSorter.UNLIMITED, 3, 1_000}) { + constructors.add( + new Object[]{ + 1 /* maxRowsPerFrame */, + 20_000 /* maxBytesPerFrame */, + 3 /* numChannels */, + 2 /* maxActiveProcessors */, + 3 /* maxChannelsPerProcessor */, + 1 /* numThreads */, + isComposedStorage, + false /* partitionsDeferred */, + limitHint + } + ); + } + } + return constructors; } + @BeforeClass + public static void setUpClass() + { + CURSOR_FACTORY = new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); + CURSOR_FACTORY_SIGNATURE_WITH_ROW_NUMBER = + FrameSequenceBuilder.signatureWithRowNumber(CURSOR_FACTORY.getRowSignature()); + } + + @AfterClass + public static void tearDownClass() + { + CURSOR_FACTORY = null; + CURSOR_FACTORY_SIGNATURE_WITH_ROW_NUMBER = null; + SORTED_TEST_ROWS.clear(); + } + @Before public void setUp() { exec = new FrameProcessorExecutor( MoreExecutors.listeningDecorator(Execs.multiThreaded(numThreads, getClass().getSimpleName() + "[%d]")) ); - adapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); } @After @@ -352,15 +395,15 @@ public class SuperSorterTest } final FrameSequenceBuilder frameSequenceBuilder = - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(CURSOR_FACTORY) .maxRowsPerFrame(maxRowsPerFrame) .sortBy(clusterBy.getColumns()) .allocator(ArenaMemoryAllocator.create(ByteBuffer.allocate(maxBytesPerFrame))) .frameType(FrameType.ROW_BASED) .populateRowNumber(); - inputChannels = makeFileChannels(frameSequenceBuilder.frames(), temporaryFolder.newFolder(), numChannels); - signature = frameSequenceBuilder.signature(); + inputChannels = makeRoundRobinChannels(frameSequenceBuilder.frames(), numChannels); + signature = FrameWriters.sortableSignature(CURSOR_FACTORY_SIGNATURE_WITH_ROW_NUMBER, clusterBy.getColumns()); frameReader = FrameReader.create(signature); } @@ -393,8 +436,8 @@ public class SuperSorterTest clusterByPartitionsFuture, exec, FrameProcessorDecorator.NONE, - new FileOutputChannelFactory(tempFolder, maxBytesPerFrame, null), - outputChannelFactory, + makeOutputChannelFactory(new FileOutputChannelFactory(tempFolder, maxBytesPerFrame, null)), + makeOutputChannelFactory(outputChannelFactory), maxActiveProcessors, maxChannelsPerProcessor, limitHint, @@ -411,7 +454,7 @@ public class SuperSorterTest Assert.assertEquals(clusterByPartitions.size(), outputChannels.getAllChannels().size()); Assert.assertEquals(Double.valueOf(1.0), superSorterProgressTracker.snapshot().getProgressDigest()); - final int[] clusterByPartColumns = clusterBy.getColumns().stream().mapToInt( + final int[] clusterByColumns = clusterBy.getColumns().stream().mapToInt( part -> signature.indexOf(part.columnName()) ).toArray(); @@ -427,33 +470,36 @@ public class SuperSorterTest frameReader ).forEach( row -> { - final Object[] array = new Object[clusterByPartColumns.length]; + final Object[] array = new Object[clusterByColumns.length]; for (int i = 0; i < array.length; i++) { - array[i] = row.get(clusterByPartColumns[i]); + array[i] = row.get(clusterByColumns[i]); } final RowKey key = createKey(clusterBy, array); - Assert.assertTrue( - StringUtils.format( - "Key %s >= partition %,d start %s", - keyReader.read(key), - partitionNumber, - partition.getStart() == null ? null : keyReader.read(partition.getStart()) - ), - partition.getStart() == null || keyComparator.compare(key, partition.getStart()) >= 0 - ); + if (!(partition.getStart() == null || keyComparator.compare(key, partition.getStart()) >= 0)) { + // Defer formatting of error message until it's actually needed + Assert.fail( + StringUtils.format( + "Key %s >= partition %,d start %s", + keyReader.read(key), + partitionNumber, + partition.getStart() == null ? null : keyReader.read(partition.getStart()) + ) + ); + } - Assert.assertTrue( - StringUtils.format( - "Key %s < partition %,d end %s", - keyReader.read(key), - partitionNumber, - partition.getEnd() == null ? null : keyReader.read(partition.getEnd()) - ), - partition.getEnd() == null || keyComparator.compare(key, partition.getEnd()) < 0 - ); + if (!(partition.getEnd() == null || keyComparator.compare(key, partition.getEnd()) < 0)) { + Assert.fail( + StringUtils.format( + "Key %s < partition %,d end %s", + keyReader.read(key), + partitionNumber, + partition.getEnd() == null ? null : keyReader.read(partition.getEnd()) + ) + ); + } readRows.add(row); } @@ -464,21 +510,9 @@ public class SuperSorterTest MatcherAssert.assertThat(readRows.size(), Matchers.greaterThanOrEqualTo(Ints.checkedCast(limitHint))); } - final Sequence> expectedRows = Sequences.sort( - FrameTestUtil.readRowsFromAdapter(adapter, signature, true), - Comparator.comparing( - row -> { - final Object[] array = new Object[clusterByPartColumns.length]; - - for (int i = 0; i < array.length; i++) { - array[i] = row.get(clusterByPartColumns[i]); - } - - return createKey(clusterBy, array); - }, - keyComparator - ) - ).limit(limitHint == SuperSorter.UNLIMITED ? Long.MAX_VALUE : readRows.size()); + final Sequence> expectedRows = + Sequences.simple(getOrComputeSortedTestRows(clusterBy)) + .limit(limitHint == SuperSorter.UNLIMITED ? Long.MAX_VALUE : readRows.size()); FrameTestUtil.assertRowsEqual(expectedRows, Sequences.simple(readRows)); } @@ -724,29 +758,63 @@ public class SuperSorterTest final RowSignature keySignature = KeyTestUtils.createKeySignature(clusterBy.getColumns(), signature); return KeyTestUtils.createKey(keySignature, objects); } + + /** + * Retrieve sorted test rows from {@link #SORTED_TEST_ROWS}, or else compute using + * {@link #computeSortedTestRows(ClusterBy)}. + */ + private static List> getOrComputeSortedTestRows(final ClusterBy clusterBy) + { + return SORTED_TEST_ROWS.computeIfAbsent(clusterBy, SuperSorterTest.ParameterizedCasesTest::computeSortedTestRows); + } + + /** + * Sort test rows from {@link TestIndex#getNoRollupMMappedTestIndex()} by the given {@link ClusterBy}. + */ + private static List> computeSortedTestRows(final ClusterBy clusterBy) + { + final QueryableIndexCursorFactory cursorFactory = + new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); + final RowSignature signature = + FrameWriters.sortableSignature( + FrameSequenceBuilder.signatureWithRowNumber(cursorFactory.getRowSignature()), + clusterBy.getColumns() + ); + final RowSignature keySignature = KeyTestUtils.createKeySignature(clusterBy.getColumns(), signature); + final int[] clusterByColumns = + clusterBy.getColumns().stream().mapToInt(part -> signature.indexOf(part.columnName())).toArray(); + final Comparator keyComparator = clusterBy.keyComparator(keySignature); + + return Sequences.sort( + FrameTestUtil.readRowsFromCursorFactory(cursorFactory, signature, true), + Comparator.comparing( + row -> { + final Object[] array = new Object[clusterByColumns.length]; + + for (int i = 0; i < array.length; i++) { + array[i] = row.get(clusterByColumns[i]); + } + + return KeyTestUtils.createKey(keySignature, array); + }, + keyComparator + ) + ).toList(); + } } - private static List makeFileChannels( + /** + * Distribute frames round-robin to some number of channels. + */ + private static List makeRoundRobinChannels( final Sequence frames, - final File tmpDir, final int numChannels ) throws IOException { - final List files = new ArrayList<>(); - final List writableChannels = new ArrayList<>(); + final List channels = new ArrayList<>(numChannels); for (int i = 0; i < numChannels; i++) { - final File file = new File(tmpDir, StringUtils.format("channel-%d", i)); - files.add(file); - writableChannels.add( - new WritableFrameFileChannel( - FrameFileWriter.open( - Channels.newChannel(Files.newOutputStream(file.toPath())), - null, - ByteTracker.unboundedTracker() - ) - ) - ); + channels.add(new BlockingQueueFrameChannel(2000) /* enough even for 1 row per frame; dataset has < 2000 rows */); } frames.forEach( @@ -758,7 +826,7 @@ public class SuperSorterTest public void accept(final Frame frame) { try { - writableChannels.get(i % writableChannels.size()).write(frame); + channels.get(i % channels.size()).writable().write(frame); } catch (IOException e) { throw new RuntimeException(e); @@ -771,20 +839,50 @@ public class SuperSorterTest final List retVal = new ArrayList<>(); - for (int i = 0; i < writableChannels.size(); i++) { - WritableFrameChannel writableChannel = writableChannels.get(i); - writableChannel.close(); - retVal.add(new ReadableFileFrameChannel(FrameFile.open(files.get(i), null))); + for (final BlockingQueueFrameChannel channel : channels) { + channel.writable().close(); + retVal.add(new AlwaysAsyncReadableFrameChannel(channel.readable())); } return retVal; } - private static long countSequence(final Sequence sequence) + /** + * Wraps an underlying {@link OutputChannelFactory} in one that uses {@link AlwaysAsyncReadableFrameChannel} + * for all of its readable channels. This helps catch bugs due to improper usage of {@link ReadableFrameChannel} + * methods that enable async reads. + */ + private static OutputChannelFactory makeOutputChannelFactory(final OutputChannelFactory baseFactory) { - return sequence.accumulate( - 0L, - (accumulated, in) -> accumulated + 1 - ); + return new OutputChannelFactory() { + @Override + public OutputChannel openChannel(int partitionNumber) throws IOException + { + final OutputChannel channel = baseFactory.openChannel(partitionNumber); + return OutputChannel.pair( + channel.getWritableChannel(), + channel.getFrameMemoryAllocator(), + () -> new AlwaysAsyncReadableFrameChannel(channel.getReadableChannelSupplier().get()), + channel.getPartitionNumber() + ); + } + + @Override + public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) throws IOException + { + final PartitionedOutputChannel channel = baseFactory.openPartitionedChannel(name, deleteAfterRead); + return PartitionedOutputChannel.pair( + channel.getWritableChannel(), + channel.getFrameMemoryAllocator(), + () -> new AlwaysAsyncPartitionedReadableFrameChannel(channel.getReadableChannelSupplier().get()) + ); + } + + @Override + public OutputChannel openNilChannel(int partitionNumber) + { + return baseFactory.openNilChannel(partitionNumber); + } + }; } } diff --git a/processing/src/test/java/org/apache/druid/frame/processor/manager/ConcurrencyLimitedProcessorManagerTest.java b/processing/src/test/java/org/apache/druid/frame/processor/manager/ConcurrencyLimitedProcessorManagerTest.java new file mode 100644 index 00000000000..85739efda8a --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/processor/manager/ConcurrencyLimitedProcessorManagerTest.java @@ -0,0 +1,103 @@ +/* + * 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 org.apache.druid.java.util.common.Unit; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Optional; + +public class ConcurrencyLimitedProcessorManagerTest +{ + @Test + public void test_empty() throws Exception + { + try (final ConcurrencyLimitedProcessorManager manager = + new ConcurrencyLimitedProcessorManager<>(ProcessorManagers.none(), 1)) { + final ListenableFuture>> future = manager.next(); + Assert.assertTrue(future.isDone()); + Assert.assertFalse(future.get().isPresent()); + Assert.assertEquals(0, (long) manager.result()); + } + } + + @Test + public void test_one_limitOne() throws Exception + { + final NilFrameProcessor processor = new NilFrameProcessor<>(); + + try (final ConcurrencyLimitedProcessorManager manager = + new ConcurrencyLimitedProcessorManager<>(ProcessorManagers.of(ImmutableList.of(processor)), 1)) { + // First element. + ListenableFuture>> future = manager.next(); + Assert.assertTrue(future.isDone()); + Assert.assertTrue(future.get().isPresent()); + Assert.assertSame(processor, future.get().get().processor()); + + // Simulate processor finishing. + future.get().get().onComplete(Unit.instance()); + + // End of sequence. + future = manager.next(); + Assert.assertTrue(future.isDone()); + Assert.assertFalse(future.get().isPresent()); + } + } + + @Test + public void test_two_limitOne() throws Exception + { + final NilFrameProcessor processor0 = new NilFrameProcessor<>(); + final NilFrameProcessor processor1 = new NilFrameProcessor<>(); + final ImmutableList> processors = ImmutableList.of(processor0, processor1); + + try (final ConcurrencyLimitedProcessorManager manager = + new ConcurrencyLimitedProcessorManager<>(ProcessorManagers.of(processors), 1)) { + // First element. + ListenableFuture>> future0 = manager.next(); + Assert.assertTrue(future0.isDone()); + Assert.assertTrue(future0.get().isPresent()); + Assert.assertSame(processors.get(0), future0.get().get().processor()); + + // Second element. Not yet ready to run due to the limit. + ListenableFuture>> future1 = manager.next(); + Assert.assertFalse(future1.isDone()); + + // Simulate processor0 finishing. + future0.get().get().onComplete(Unit.instance()); + + // processor1 is now ready to run. + Assert.assertTrue(future1.isDone()); + Assert.assertTrue(future1.get().isPresent()); + Assert.assertSame(processors.get(1), future1.get().get().processor()); + + // Simulate processor1 finishing. + future1.get().get().onComplete(Unit.instance()); + + // End of sequence. + future1 = manager.next(); + Assert.assertTrue(future1.isDone()); + Assert.assertFalse(future1.get().isPresent()); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/processor/manager/NilFrameProcessor.java b/processing/src/test/java/org/apache/druid/frame/processor/manager/NilFrameProcessor.java new file mode 100644 index 00000000000..3cd749bbcc2 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/processor/manager/NilFrameProcessor.java @@ -0,0 +1,60 @@ +/* + * 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 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 java.util.Collections; +import java.util.List; + +/** + * Frame processor that throws an exception from {@link #runIncrementally(IntSet)}. Used as a dummy processor + * by tests of {@link ProcessorManager}. + */ +public 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/processor/manager/SequenceProcessorManagerTest.java b/processing/src/test/java/org/apache/druid/frame/processor/manager/SequenceProcessorManagerTest.java index a1ce465540a..7bbbfca3617 100644 --- 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 @@ -21,18 +21,13 @@ 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; @@ -59,7 +54,7 @@ public class SequenceProcessorManagerTest @Test public void test_one() throws Exception { - final NilFrameProcessor processor = new NilFrameProcessor(); + final NilFrameProcessor processor = new NilFrameProcessor<>(); final AtomicLong closed = new AtomicLong(); try (final SequenceProcessorManager> manager = @@ -84,8 +79,8 @@ public class SequenceProcessorManagerTest @Test public void test_two() throws Exception { - final NilFrameProcessor processor0 = new NilFrameProcessor(); - final NilFrameProcessor processor1 = new NilFrameProcessor(); + final NilFrameProcessor processor0 = new NilFrameProcessor<>(); + final NilFrameProcessor processor1 = new NilFrameProcessor<>(); final AtomicLong closed = new AtomicLong(); try (final SequenceProcessorManager> manager = @@ -139,31 +134,4 @@ public class SequenceProcessorManagerTest // Sequence is not closed because it never started iterating. Assert.assertEquals(0, closed.get()); } - - public 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/processor/test/AlwaysAsyncPartitionedReadableFrameChannel.java b/processing/src/test/java/org/apache/druid/frame/processor/test/AlwaysAsyncPartitionedReadableFrameChannel.java new file mode 100644 index 00000000000..4013889df6e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/processor/test/AlwaysAsyncPartitionedReadableFrameChannel.java @@ -0,0 +1,51 @@ +/* + * 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.test; + +import org.apache.druid.frame.channel.PartitionedReadableFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; + +import java.io.IOException; + +/** + * Implementation of {@link PartitionedReadableFrameChannel} that wraps all underlying channels in + * {@link AlwaysAsyncReadableFrameChannel}. + */ +public class AlwaysAsyncPartitionedReadableFrameChannel implements PartitionedReadableFrameChannel +{ + private final PartitionedReadableFrameChannel delegate; + + public AlwaysAsyncPartitionedReadableFrameChannel(PartitionedReadableFrameChannel delegate) + { + this.delegate = delegate; + } + + @Override + public ReadableFrameChannel getReadableFrameChannel(int partitionNumber) + { + return new AlwaysAsyncReadableFrameChannel(delegate.getReadableFrameChannel(partitionNumber)); + } + + @Override + public void close() throws IOException + { + delegate.close(); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/processor/test/AlwaysAsyncReadableFrameChannel.java b/processing/src/test/java/org/apache/druid/frame/processor/test/AlwaysAsyncReadableFrameChannel.java new file mode 100644 index 00000000000..8ff10aeb7b0 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/processor/test/AlwaysAsyncReadableFrameChannel.java @@ -0,0 +1,85 @@ +/* + * 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.test; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.java.util.common.ISE; + +/** + * Wraps an underlying channel and forces an async style of reading. After each call to {@link #read()}, the + * {@link #canRead()} and {@link #isFinished()} methods return false until {@link #readabilityFuture()} is called. + */ +public class AlwaysAsyncReadableFrameChannel implements ReadableFrameChannel +{ + private final ReadableFrameChannel delegate; + private boolean defer; + + public AlwaysAsyncReadableFrameChannel(ReadableFrameChannel delegate) + { + this.delegate = delegate; + } + + @Override + public boolean isFinished() + { + if (defer) { + return false; + } + + return delegate.isFinished(); + } + + @Override + public boolean canRead() + { + if (defer) { + return false; + } + + return delegate.canRead(); + } + + @Override + public Frame read() + { + if (defer) { + throw new ISE("Cannot call read() while deferred"); + } + + defer = true; + return delegate.read(); + } + + @Override + public ListenableFuture readabilityFuture() + { + defer = false; + return delegate.readabilityFuture(); + } + + @Override + public void close() + { + defer = false; + delegate.close(); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/processor/test/TestFrameProcessorUtils.java b/processing/src/test/java/org/apache/druid/frame/processor/test/TestFrameProcessorUtils.java index 0a25993a49a..baac335f0c4 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/test/TestFrameProcessorUtils.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/test/TestFrameProcessorUtils.java @@ -26,13 +26,10 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.testutil.FrameSequenceBuilder; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; @@ -44,19 +41,15 @@ public final class TestFrameProcessorUtils { } - public static StorageAdapter toStorageAdapter(List inputRows) + public static CursorFactory toCursorFactory(List inputRows) { final IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( - new IncrementalIndexSchema( - 0, - new TimestampSpec("__time", "millis", null), - Granularities.NONE, - VirtualColumns.EMPTY, - DimensionsSpec.builder().useSchemaDiscovery(true).build(), - new AggregatorFactory[0], - false - ) + IncrementalIndexSchema.builder() + .withTimestampSpec(new TimestampSpec("__time", "millis", null)) + .withDimensionsSpec(DimensionsSpec.builder().useSchemaDiscovery(true).build()) + .withRollup(false) + .build() ) .setMaxRowCount(1000) .build(); @@ -70,13 +63,13 @@ public final class TestFrameProcessorUtils throw new RuntimeException(e); } - return new IncrementalIndexStorageAdapter(index); + return new IncrementalIndexCursorFactory(index); } public static Frame toFrame(List inputRows) { - final StorageAdapter storageAdapter = toStorageAdapter(inputRows); - return Iterables.getOnlyElement(FrameSequenceBuilder.fromAdapter(storageAdapter) + final CursorFactory cursorFactory = toCursorFactory(inputRows); + return Iterables.getOnlyElement(FrameSequenceBuilder.fromCursorFactory(cursorFactory) .frameType(FrameType.ROW_BASED) .frames() .toList()); diff --git a/processing/src/test/java/org/apache/druid/frame/read/FrameReaderTest.java b/processing/src/test/java/org/apache/druid/frame/read/FrameReaderTest.java index bf24b6cb5f0..a5c15a7121c 100644 --- a/processing/src/test/java/org/apache/druid/frame/read/FrameReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/read/FrameReaderTest.java @@ -24,8 +24,8 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.allocation.HeapMemoryAllocator; import org.apache.druid.frame.testutil.FrameSequenceBuilder; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; @@ -42,7 +42,7 @@ public class FrameReaderTest extends InitializedNullHandlingTest { private final FrameType frameType; - private StorageAdapter inputAdapter; + private CursorFactory inputCursorFactory; private Frame frame; private FrameReader frameReader; @@ -66,10 +66,10 @@ public class FrameReaderTest extends InitializedNullHandlingTest @Before public void setUp() { - inputAdapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex()); + inputCursorFactory = new QueryableIndexCursorFactory(TestIndex.getNoRollupMMappedTestIndex()); final FrameSequenceBuilder frameSequenceBuilder = - FrameSequenceBuilder.fromAdapter(inputAdapter) + FrameSequenceBuilder.fromCursorFactory(inputCursorFactory) .frameType(frameType) .allocator(HeapMemoryAllocator.unlimited()); @@ -80,16 +80,16 @@ public class FrameReaderTest extends InitializedNullHandlingTest @Test public void testSignature() { - Assert.assertEquals(inputAdapter.getRowSignature(), frameReader.signature()); + Assert.assertEquals(inputCursorFactory.getRowSignature(), frameReader.signature()); } @Test public void testColumnCapabilitiesToColumnType() { - for (final String columnName : inputAdapter.getRowSignature().getColumnNames()) { + for (final String columnName : inputCursorFactory.getRowSignature().getColumnNames()) { Assert.assertEquals( columnName, - inputAdapter.getRowSignature().getColumnCapabilities(columnName).toColumnType(), + inputCursorFactory.getRowSignature().getColumnCapabilities(columnName).toColumnType(), frameReader.columnCapabilities(frame, columnName).toColumnType() ); } diff --git a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/frame/segment/FrameCursorFactoryTest.java similarity index 77% rename from processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/frame/segment/FrameCursorFactoryTest.java index 2a8f525bf78..08978e20b0c 100644 --- a/processing/src/test/java/org/apache/druid/frame/segment/FrameStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/segment/FrameCursorFactoryTest.java @@ -37,11 +37,10 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; -import org.apache.druid.segment.DimensionDictionarySelector; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -70,7 +69,7 @@ import java.util.Collections; import java.util.List; import java.util.function.Function; -public class FrameStorageAdapterTest +public class FrameCursorFactoryTest { /** * Basic tests: everything except makeCursor, makeVectorCursor. @@ -80,9 +79,9 @@ public class FrameStorageAdapterTest { private final FrameType frameType; - private StorageAdapter queryableAdapter; + private CursorFactory queryableCursorFactory; private FrameSegment frameSegment; - private StorageAdapter frameAdapter; + private CursorFactory frameCursorFactory; public BasicTests(final FrameType frameType) { @@ -105,9 +104,9 @@ public class FrameStorageAdapterTest public void setUp() { - queryableAdapter = new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex()); - frameSegment = FrameTestUtil.adapterToFrameSegment(queryableAdapter, frameType); - frameAdapter = frameSegment.asStorageAdapter(); + queryableCursorFactory = new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex()); + frameSegment = FrameTestUtil.cursorFactoryToFrameSegment(queryableCursorFactory, frameType); + frameCursorFactory = frameSegment.asCursorFactory(); } @After @@ -118,62 +117,18 @@ public class FrameStorageAdapterTest } } - @Test - public void test_getInterval() - { - Assert.assertEquals(queryableAdapter.getInterval(), frameAdapter.getInterval()); - } - @Test public void test_getRowSignature() { - Assert.assertEquals(queryableAdapter.getRowSignature(), frameAdapter.getRowSignature()); - } - - @Test - public void test_getAvailableDimensions() - { - // All columns are dimensions to the frameAdapter. - Assert.assertEquals( - queryableAdapter.getRowSignature().getColumnNames(), - ImmutableList.copyOf(frameAdapter.getAvailableDimensions()) - ); - } - - @Test - public void test_getAvailableMetrics() - { - // All columns are dimensions to the frameAdapter. - Assert.assertEquals(Collections.emptyList(), frameAdapter.getAvailableMetrics()); - } - - @Test - public void test_getDimensionCardinality_knownColumns() - { - for (final String columnName : frameAdapter.getRowSignature().getColumnNames()) { - Assert.assertEquals( - columnName, - DimensionDictionarySelector.CARDINALITY_UNKNOWN, - frameAdapter.getDimensionCardinality(columnName) - ); - } - } - - @Test - public void test_getDimensionCardinality_unknownColumn() - { - Assert.assertEquals( - DimensionDictionarySelector.CARDINALITY_UNKNOWN, - frameAdapter.getDimensionCardinality("nonexistent") - ); + Assert.assertEquals(queryableCursorFactory.getRowSignature(), frameCursorFactory.getRowSignature()); } @Test public void test_getColumnCapabilities_typeOfKnownColumns() { - for (final String columnName : frameAdapter.getRowSignature().getColumnNames()) { - final ColumnCapabilities expectedCapabilities = queryableAdapter.getColumnCapabilities(columnName); - final ColumnCapabilities actualCapabilities = frameAdapter.getColumnCapabilities(columnName); + for (final String columnName : frameCursorFactory.getRowSignature().getColumnNames()) { + final ColumnCapabilities expectedCapabilities = queryableCursorFactory.getColumnCapabilities(columnName); + final ColumnCapabilities actualCapabilities = frameCursorFactory.getColumnCapabilities(columnName); Assert.assertEquals( StringUtils.format("column [%s] type", columnName), @@ -204,19 +159,7 @@ public class FrameStorageAdapterTest @Test public void test_getColumnCapabilities_unknownColumn() { - Assert.assertNull(frameAdapter.getColumnCapabilities("nonexistent")); - } - - @Test - public void test_getNumRows() - { - Assert.assertEquals(queryableAdapter.getNumRows(), frameAdapter.getNumRows()); - } - - @Test - public void test_getMetadata() - { - Assert.assertNull(frameAdapter.getMetadata()); + Assert.assertNull(frameCursorFactory.getColumnCapabilities("nonexistent")); } } @@ -235,9 +178,9 @@ public class FrameStorageAdapterTest private final VirtualColumns virtualColumns; private final boolean descending; - private StorageAdapter queryableAdapter; + private CursorFactory queryableCursorFactory; private FrameSegment frameSegment; - private StorageAdapter frameAdapter; + private CursorFactory frameCursorFactory; private final QueryContext queryContext = QueryContext.of( ImmutableMap.of(QueryContexts.VECTOR_SIZE_KEY, VECTOR_SIZE) ); @@ -338,9 +281,9 @@ public class FrameStorageAdapterTest @Before public void setUp() { - queryableAdapter = new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex()); - frameSegment = FrameTestUtil.adapterToFrameSegment(queryableAdapter, frameType); - frameAdapter = frameSegment.asStorageAdapter(); + queryableCursorFactory = new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex()); + frameSegment = FrameTestUtil.cursorFactoryToFrameSegment(queryableCursorFactory, frameType); + frameCursorFactory = frameSegment.asCursorFactory(); } @After @@ -354,7 +297,7 @@ public class FrameStorageAdapterTest @Test public void test_makeCursor() { - final RowSignature signature = frameAdapter.getRowSignature(); + final RowSignature signature = frameCursorFactory.getRowSignature(); // Frame adapters don't know the order of the underlying frames, so they should ignore the "preferred ordering" // of the cursor build spec. We test this by passing the frameAdapter a build spec with a preferred ordering, @@ -362,8 +305,8 @@ public class FrameStorageAdapterTest final CursorBuildSpec queryableBuildSpec = CursorBuildSpec.builder(buildSpec).setPreferredOrdering(Collections.emptyList()).build(); - try (final CursorHolder queryableCursorHolder = queryableAdapter.makeCursorHolder(queryableBuildSpec); - final CursorHolder frameCursorHolder = frameAdapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder queryableCursorHolder = queryableCursorFactory.makeCursorHolder(queryableBuildSpec); + final CursorHolder frameCursorHolder = frameCursorFactory.makeCursorHolder(buildSpec)) { final Sequence> queryableRows = FrameTestUtil.readRowsFromCursor(advanceAndReset(queryableCursorHolder.asCursor()), signature); final Sequence> frameRows = @@ -378,18 +321,18 @@ public class FrameStorageAdapterTest // Conditions for frames to be vectorizable. Assume.assumeThat(frameType, CoreMatchers.equalTo(FrameType.COLUMNAR)); Assume.assumeFalse(descending); - assertVectorCursorsMatch(adapter -> adapter.makeCursorHolder(buildSpec)); + assertVectorCursorsMatch(cursorFactory -> cursorFactory.makeCursorHolder(buildSpec)); } - private void assertVectorCursorsMatch(final Function call) + private void assertVectorCursorsMatch(final Function call) { - final CursorHolder cursorHolder = call.apply(queryableAdapter); - final CursorHolder frameCursorHolder = call.apply(frameAdapter); + final CursorHolder cursorHolder = call.apply(queryableCursorFactory); + final CursorHolder frameCursorHolder = call.apply(frameCursorFactory); Assert.assertTrue("queryable cursor is vectorizable", cursorHolder.canVectorize()); Assert.assertTrue("frame cursor is vectorizable", frameCursorHolder.canVectorize()); - final RowSignature signature = frameAdapter.getRowSignature(); + final RowSignature signature = frameCursorFactory.getRowSignature(); final Sequence> queryableRows = FrameTestUtil.readRowsFromVectorCursor(advanceAndReset(cursorHolder.asVectorCursor()), signature) .withBaggage(cursorHolder); diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java b/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java index eb816225196..1cb6298b8b1 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/FrameSequenceBuilder.java @@ -34,8 +34,8 @@ import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -49,7 +49,7 @@ import java.util.NoSuchElementException; */ public class FrameSequenceBuilder { - private final StorageAdapter adapter; + private final CursorFactory cursorFactory; private FrameType frameType = null; private MemoryAllocator allocator = HeapMemoryAllocator.unlimited(); @@ -57,14 +57,25 @@ public class FrameSequenceBuilder private int maxRowsPerFrame = Integer.MAX_VALUE; private boolean populateRowNumber = false; - private FrameSequenceBuilder(StorageAdapter adapter) + private FrameSequenceBuilder(CursorFactory cursorFactory) { - this.adapter = adapter; + this.cursorFactory = cursorFactory; } - public static FrameSequenceBuilder fromAdapter(final StorageAdapter adapter) + public static FrameSequenceBuilder fromCursorFactory(final CursorFactory cursorFactory) { - return new FrameSequenceBuilder(adapter); + return new FrameSequenceBuilder(cursorFactory); + } + + /** + * Returns what {@link #signature()} would return if {@link #populateRowNumber()} is set. + */ + public static RowSignature signatureWithRowNumber(final RowSignature signature) + { + return RowSignature.builder() + .addAll(signature) + .add(FrameTestUtil.ROW_NUMBER_COLUMN, ColumnType.LONG) + .build(); } public FrameSequenceBuilder frameType(final FrameType frameType) @@ -108,12 +119,9 @@ public class FrameSequenceBuilder final RowSignature baseSignature; if (populateRowNumber) { - baseSignature = RowSignature.builder() - .addAll(adapter.getRowSignature()) - .add(FrameTestUtil.ROW_NUMBER_COLUMN, ColumnType.LONG) - .build(); + baseSignature = signatureWithRowNumber(cursorFactory.getRowSignature()); } else { - baseSignature = adapter.getRowSignature(); + baseSignature = cursorFactory.getRowSignature(); } return FrameWriters.sortableSignature(baseSignature, keyColumns); @@ -139,7 +147,7 @@ public class FrameSequenceBuilder throw DruidException.defensive("Unrecognized frame type"); } - final CursorHolder cursorHolder = FrameTestUtil.makeCursorForAdapter(adapter, populateRowNumber); + final CursorHolder cursorHolder = FrameTestUtil.makeCursorForCursorFactory(cursorFactory, populateRowNumber); final Cursor cursor = cursorHolder.asCursor(); return new BaseSequence<>( new BaseSequence.IteratorMaker>() diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java b/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java index e1981f3618b..2bb8789740c 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java @@ -30,7 +30,6 @@ import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.file.FrameFileWriter; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.segment.FrameSegment; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.util.SettableLongVirtualColumn; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.guava.Sequence; @@ -42,9 +41,9 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -57,9 +56,10 @@ import org.junit.Assert; import javax.annotation.Nullable; import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; +import java.io.OutputStream; import java.nio.channels.Channels; +import java.nio.file.Files; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -80,12 +80,14 @@ public class FrameTestUtil public static File writeFrameFile(final Sequence frames, final File file) throws IOException { - try ( - final FileOutputStream fos = new FileOutputStream(file); - final FrameFileWriter writer = FrameFileWriter.open( - Channels.newChannel(fos), null, ByteTracker.unboundedTracker() - ) - ) { + writeFrameFile(frames, Files.newOutputStream(file.toPath())); + return file; + } + + public static void writeFrameFile(final Sequence frames, final OutputStream out) throws IOException + { + try (final FrameFileWriter writer = + FrameFileWriter.open(Channels.newChannel(out), null, ByteTracker.unboundedTracker())) { frames.forEach( frame -> { try { @@ -97,17 +99,15 @@ public class FrameTestUtil } ); } - - return file; } - public static File writeFrameFileWithPartitions( + public static void writeFrameFileWithPartitions( final Sequence> framesWithPartitions, - final File file + final OutputStream out ) throws IOException { try (final FrameFileWriter writer = FrameFileWriter.open( - Channels.newChannel(new FileOutputStream(file)), + Channels.newChannel(out), null, ByteTracker.unboundedTracker() )) { @@ -122,8 +122,6 @@ public class FrameTestUtil } ); } - - return file; } public static void assertRowsEqual(final Sequence> expected, final Sequence> actual) @@ -182,10 +180,10 @@ public class FrameTestUtil } } - public static Frame adapterToFrame(final StorageAdapter adapter, final FrameType frameType) + public static Frame cursorFactoryToFrame(final CursorFactory adapter, final FrameType frameType) { return Iterables.getOnlyElement( - FrameSequenceBuilder.fromAdapter(adapter) + FrameSequenceBuilder.fromCursorFactory(adapter) .allocator(HeapMemoryAllocator.unlimited()) .frameType(frameType) .frames() @@ -193,34 +191,34 @@ public class FrameTestUtil ); } - public static FrameSegment adapterToFrameSegment( - final StorageAdapter adapter, + public static FrameSegment cursorFactoryToFrameSegment( + final CursorFactory adapter, final FrameType frameType ) { return new FrameSegment( - adapterToFrame(adapter, frameType), + cursorFactoryToFrame(adapter, frameType), FrameReader.create(adapter.getRowSignature()), - SegmentId.of("TestFrame", adapter.getInterval(), "0", 0) + SegmentId.of("TestFrame", Intervals.ETERNITY, "0", 0) ); } - public static FrameSegment adapterToFrameSegment( - final StorageAdapter adapter, + public static FrameSegment cursorFactoryToFrameSegment( + final CursorFactory cursorFactory, final FrameType frameType, final SegmentId segmentId ) { return new FrameSegment( - adapterToFrame(adapter, frameType), - FrameReader.create(adapter.getRowSignature()), + cursorFactoryToFrame(cursorFactory, frameType), + FrameReader.create(cursorFactory.getRowSignature()), segmentId ); } /** * Reads a sequence of rows from a frame channel using a non-vectorized cursor from - * {@link FrameStorageAdapter#makeCursorHolder(CursorBuildSpec)}. + * {@link FrameReader#makeCursorFactory(Frame)}. * * @param channel the channel * @param frameReader reader for this channel @@ -233,8 +231,8 @@ public class FrameTestUtil return new FrameChannelSequence(channel) .flatMap( frame -> { - final CursorHolder cursorHolder = new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) - .makeCursorHolder(CursorBuildSpec.FULL_SCAN); + final CursorHolder cursorHolder = frameReader.makeCursorFactory(frame) + .makeCursorHolder(CursorBuildSpec.FULL_SCAN); final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return Sequences.withBaggage(Sequences.empty(), cursorHolder); @@ -245,23 +243,24 @@ public class FrameTestUtil } /** - * Reads a sequence of rows from a storage adapter. + * Reads a sequence of rows from a {@link CursorFactory}. * * If {@param populateRowNumberIfPresent} is set, and the provided signature contains {@link #ROW_NUMBER_COLUMN}, - * then that column will be populated with a row number from the adapter. + * then that column will be populated with a row number from the cursor. * - * @param adapter the adapter - * @param signature optional signature for returned rows; will use {@code adapter.rowSignature()} if null + * @param cursorFactory the cursor factory + * @param signature optional signature for returned rows; will use {@code cursorFactory.rowSignature()} if + * null * @param populateRowNumber whether to populate {@link #ROW_NUMBER_COLUMN} */ - public static Sequence> readRowsFromAdapter( - final StorageAdapter adapter, + public static Sequence> readRowsFromCursorFactory( + final CursorFactory cursorFactory, @Nullable final RowSignature signature, final boolean populateRowNumber ) { - final RowSignature signatureToUse = signature == null ? adapter.getRowSignature() : signature; - final CursorHolder cursorHolder = makeCursorForAdapter(adapter, populateRowNumber); + final RowSignature signatureToUse = signature == null ? cursorFactory.getRowSignature() : signature; + final CursorHolder cursorHolder = makeCursorForCursorFactory(cursorFactory, populateRowNumber); final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return Sequences.withBaggage(Sequences.empty(), cursorHolder); @@ -269,16 +268,26 @@ public class FrameTestUtil return readRowsFromCursor(cursor, signatureToUse).withBaggage(cursorHolder); } + public static Sequence> readRowsFromCursorFactory(CursorFactory cursorFactory) + { + return readRowsFromCursorFactory(cursorFactory, null, false); + } + + public static Sequence> readRowsFromCursorFactoryWithRowNumber(CursorFactory cursorFactory) + { + return readRowsFromCursorFactory(cursorFactory, null, true); + } + /** - * Creates a Cursor and from a storage adapter. + * Creates a {@link CursorHolder} from a {@link CursorFactory}. * * If {@param populateRowNumber} is set, the row number will be populated into {@link #ROW_NUMBER_COLUMN}. * - * @param adapter the adapter + * @param cursorFactory the cursor factory * @param populateRowNumber whether to populate {@link #ROW_NUMBER_COLUMN} */ - public static CursorHolder makeCursorForAdapter( - final StorageAdapter adapter, + public static CursorHolder makeCursorForCursorFactory( + final CursorFactory cursorFactory, final boolean populateRowNumber ) { @@ -297,7 +306,7 @@ public class FrameTestUtil .setVirtualColumns(virtualColumns) .build(); - final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec); + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec); if (populateRowNumber) { return new CursorHolder() { diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java index 34cdab25a48..4c6f81a2a16 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/RowNumberUpdatingCursor.java @@ -24,7 +24,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; /** - * Used by {@link FrameTestUtil#readRowsFromAdapter} and {@link FrameTestUtil#readRowsFromCursor}. + * Used by {@link FrameTestUtil#readRowsFromCursorFactory} and {@link FrameTestUtil#readRowsFromCursor}. */ public class RowNumberUpdatingCursor implements Cursor { 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 9d7a9b72553..3103348284b 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 @@ -35,10 +35,8 @@ import org.apache.druid.frame.key.RowKey; import org.apache.druid.frame.key.RowKeyComparator; import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.segment.FrameSegment; -import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.guice.BuiltInTypesModule; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; @@ -50,6 +48,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.RowBasedSegment; @@ -438,15 +437,11 @@ public class FrameWriterTest extends InitializedNullHandlingTest final RowSignature signature ) { - final FrameStorageAdapter frameAdapter = new FrameStorageAdapter( - frame, - FrameReader.create(signature), - Intervals.ETERNITY - ); + final CursorFactory cursorFactory = FrameReader.create(signature).makeCursorFactory(frame); FrameTestUtil.assertRowsEqual( expectedRows, - FrameTestUtil.readRowsFromAdapter(frameAdapter, signature, false) + FrameTestUtil.readRowsFromCursorFactory(cursorFactory, signature, false) ); } @@ -577,8 +572,7 @@ public class FrameWriterTest extends InitializedNullHandlingTest inputSegment = new FrameSegment(inputFrame, FrameReader.create(signature), SegmentId.dummy("xxx")); } - try (final CursorHolder cursorHolder = inputSegment.asStorageAdapter() - .makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = inputSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); int numRows = 0; diff --git a/processing/src/test/java/org/apache/druid/frame/write/RowBasedFrameWriterTest.java b/processing/src/test/java/org/apache/druid/frame/write/RowBasedFrameWriterTest.java index bb7f9e55878..593004abb6d 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/RowBasedFrameWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/write/RowBasedFrameWriterTest.java @@ -20,21 +20,76 @@ package org.apache.druid.frame.write; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.frame.Frame; import org.apache.druid.frame.allocation.AppendableMemory; +import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; import org.apache.druid.frame.allocation.HeapMemoryAllocator; import org.apache.druid.frame.field.LongFieldWriter; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.RowAdapters; +import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; +import java.util.Arrays; import java.util.Collections; -public class RowBasedFrameWriterTest +public class RowBasedFrameWriterTest extends InitializedNullHandlingTest { @Test - public void testAddSelectionWithException() + public void test_addSelection_singleLargeRow() + { + final RowSignature signature = + RowSignature.builder() + .add("n", ColumnType.LONG) + .add("s", ColumnType.STRING) + .build(); + + final byte[] largeUtf8 = new byte[990000]; + Arrays.fill(largeUtf8, (byte) 'F'); + final String largeString = StringUtils.fromUtf8(largeUtf8); + final Row largeRow = new MapBasedRow(0L, ImmutableMap.of("n", 3L, "s", largeString)); + + final FrameWriterFactory frameWriterFactory = FrameWriters.makeRowBasedFrameWriterFactory( + new ArenaMemoryAllocatorFactory(1_000_000), + signature, + ImmutableList.of(), + false + ); + + final ColumnSelectorFactory columnSelectorFactory = RowBasedColumnSelectorFactory.create( + RowAdapters.standardRow(), + () -> largeRow, + signature, + false, + false + ); + + final Frame frame; + try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(columnSelectorFactory)) { + Assert.assertTrue(frameWriter.addSelection()); + frame = Frame.wrap(frameWriter.toByteArray()); + } + + FrameTestUtil.assertRowsEqual( + Sequences.simple(Collections.singletonList(ImmutableList.of(3L, largeString))), + FrameTestUtil.readRowsFromCursorFactory(FrameReader.create(signature).makeCursorFactory(frame)) + ); + } + + @Test + public void test_addSelection_withException() { String colName = "colName"; String errorMsg = "Frame writer exception"; diff --git a/processing/src/test/java/org/apache/druid/jackson/DefaultObjectMapperTest.java b/processing/src/test/java/org/apache/druid/jackson/DefaultObjectMapperTest.java index 989d137770e..92c8a2cb298 100644 --- a/processing/src/test/java/org/apache/druid/jackson/DefaultObjectMapperTest.java +++ b/processing/src/test/java/org/apache/druid/jackson/DefaultObjectMapperTest.java @@ -22,12 +22,18 @@ package org.apache.druid.jackson; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.exc.InvalidTypeIdException; +import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; 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.guava.Yielders; import org.apache.druid.query.Query; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumns; +import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumnsTest; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.junit.Assert; @@ -35,6 +41,8 @@ import org.junit.Test; import java.util.Arrays; +import static org.junit.Assert.assertEquals; + /** * */ @@ -102,4 +110,22 @@ public class DefaultObjectMapperTest } Assert.fail("We expect InvalidTypeIdException to be thrown"); } + + @Test + public void testColumnBasedFrameRowsAndColumns() throws Exception + { + DefaultObjectMapper om = new DefaultObjectMapper("test"); + + MapOfColumnsRowsAndColumns input = (MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "colA", new IntArrayColumn(new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + "colB", new IntArrayColumn(new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0}) + ))); + + ColumnBasedFrameRowsAndColumns frc = ColumnBasedFrameRowsAndColumnsTest.buildFrame(input); + byte[] bytes = om.writeValueAsBytes(frc); + + ColumnBasedFrameRowsAndColumns frc2 = (ColumnBasedFrameRowsAndColumns) om.readValue(bytes, RowsAndColumns.class); + assertEquals(frc, frc2); + } } diff --git a/processing/src/test/java/org/apache/druid/java/util/common/parsers/FlatTextFormatParserTest.java b/processing/src/test/java/org/apache/druid/java/util/common/parsers/FlatTextFormatParserTest.java index d9a498a026d..1b69d0bdec4 100644 --- a/processing/src/test/java/org/apache/druid/java/util/common/parsers/FlatTextFormatParserTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/common/parsers/FlatTextFormatParserTest.java @@ -112,7 +112,7 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest public void testWithSkipHeaderRows() { final int skipHeaderRows = 2; - final Parser parser = PARSER_FACTORY.get(format, false, skipHeaderRows); + final Parser parser = PARSER_FACTORY.get(format, false, skipHeaderRows, false); parser.startFileFromBeginning(); final String[] body = new String[]{ concat(format, "header", "line", "1"), @@ -134,7 +134,7 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest @Test public void testWithHeaderRow() { - final Parser parser = PARSER_FACTORY.get(format, true, 0); + final Parser parser = PARSER_FACTORY.get(format, true, 0, false); parser.startFileFromBeginning(); final String[] body = new String[]{ concat(format, "time", "value1", "value2"), @@ -152,7 +152,7 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest @Test public void testWithHeaderRowOfEmptyColumns() { - final Parser parser = PARSER_FACTORY.get(format, true, 0); + final Parser parser = PARSER_FACTORY.get(format, true, 0, false); parser.startFileFromBeginning(); final String[] body = new String[]{ concat(format, "time", "", "value2", ""), @@ -170,7 +170,7 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest @Test public void testWithDifferentHeaderRows() { - final Parser parser = PARSER_FACTORY.get(format, true, 0); + final Parser parser = PARSER_FACTORY.get(format, true, 0, false); parser.startFileFromBeginning(); final String[] body = new String[]{ concat(format, "time", "value1", "value2"), @@ -207,7 +207,7 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest ); final int skipHeaderRows = 2; - final Parser parser = PARSER_FACTORY.get(format, false, skipHeaderRows); + final Parser parser = PARSER_FACTORY.get(format, false, skipHeaderRows, false); final String[] body = new String[]{ concat(format, "header", "line", "1"), concat(format, "header", "line", "2"), @@ -219,7 +219,7 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest @Test public void testWithNullValues() { - final Parser parser = PARSER_FACTORY.get(format, true, 0); + final Parser parser = PARSER_FACTORY.get(format, true, 0, false); parser.startFileFromBeginning(); final String[] body = new String[]{ concat(format, "time", "value1", "value2"), @@ -234,16 +234,16 @@ public class FlatTextFormatParserTest extends InitializedNullHandlingTest { public Parser get(FlatTextFormat format) { - return get(format, false, 0); + return get(format, false, 0, false); } - public Parser get(FlatTextFormat format, boolean hasHeaderRow, int maxSkipHeaderRows) + public Parser get(FlatTextFormat format, boolean hasHeaderRow, int maxSkipHeaderRows, boolean tryParseNumbers) { switch (format) { case CSV: - return new CSVParser(null, hasHeaderRow, maxSkipHeaderRows); + return new CSVParser(null, hasHeaderRow, maxSkipHeaderRows, tryParseNumbers); case DELIMITED: - return new DelimitedParser("\t", null, hasHeaderRow, maxSkipHeaderRows); + return new DelimitedParser("\t", null, hasHeaderRow, maxSkipHeaderRows, tryParseNumbers); default: throw new IAE("Unknown format[%s]", format); } diff --git a/processing/src/test/java/org/apache/druid/java/util/common/parsers/ParserUtilsTest.java b/processing/src/test/java/org/apache/druid/java/util/common/parsers/ParserUtilsTest.java index 56457330254..caf4fc030da 100644 --- a/processing/src/test/java/org/apache/druid/java/util/common/parsers/ParserUtilsTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/common/parsers/ParserUtilsTest.java @@ -19,27 +19,269 @@ package org.apache.druid.java.util.common.parsers; +import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import org.junit.Assert; +import org.apache.druid.common.config.NullHandling; import org.junit.Test; import java.util.Collections; import java.util.List; +import static org.apache.druid.java.util.common.parsers.ParserUtils.findDuplicates; +import static org.apache.druid.java.util.common.parsers.ParserUtils.getTransformationFunction; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + public class ParserUtilsTest { + static { + NullHandling.initializeForTests(); + } + @Test public void testFindDuplicatesMixedCases() { final List fields = ImmutableList.of("f1", "f2", "F1", "F2", "f3"); - Assert.assertEquals(Collections.emptySet(), ParserUtils.findDuplicates(fields)); + assertEquals(Collections.emptySet(), findDuplicates(fields)); } @Test public void testFindDuplicates() { final List fields = ImmutableList.of("f1", "f2", "F1", "F2", "f1", "F2"); - Assert.assertEquals(ImmutableSet.of("f1", "F2"), ParserUtils.findDuplicates(fields)); + assertEquals(ImmutableSet.of("f1", "F2"), findDuplicates(fields)); + } + + @Test + public void testInputWithDelimiterAndParserDisabled() + { + assertNull( + getTransformationFunction("|", Splitter.on("|"), true).apply(null) + ); + assertEquals( + NullHandling.emptyToNullIfNeeded(""), + getTransformationFunction("|", Splitter.on("|"), true).apply("") + ); + assertEquals( + ImmutableList.of("foo", "boo"), + getTransformationFunction("|", Splitter.on("|"), false).apply("foo|boo") + ); + assertEquals( + ImmutableList.of("1", "2", "3"), + getTransformationFunction("|", Splitter.on("|"), false).apply("1|2|3") + ); + assertEquals( + ImmutableList.of("1", "-2", "3", "0", "-2"), + getTransformationFunction("|", Splitter.on("|"), false).apply("1|-2|3|0|-2") + ); + assertEquals( + "100", + getTransformationFunction("|", Splitter.on("|"), false).apply("100") + ); + assertEquals( + "1.23", + getTransformationFunction("|", Splitter.on("|"), false).apply("1.23") + ); + assertEquals( + "-2.0", + getTransformationFunction("|", Splitter.on("|"), false).apply("-2.0") + ); + assertEquals( + "1e2", + getTransformationFunction("|", Splitter.on("|"), false).apply("1e2") + ); + assertEquals( + ImmutableList.of("1", "2", "3"), + getTransformationFunction("|", Splitter.on("|"), false).apply("1|2|3") + ); + assertEquals( + ImmutableList.of("1", "-2", "3", "0", "-2"), + getTransformationFunction("|", Splitter.on("|"), false).apply("1|-2|3|0|-2") + ); + assertEquals( + ImmutableList.of("-1.0", "-2.2", "3.1", "0.2", "-2.1"), + getTransformationFunction("|", Splitter.on("|"), false).apply("-1.0|-2.2|3.1|0.2|-2.1") + ); + + // Some mixed types + assertEquals( + ImmutableList.of("-1.23", "3.13", "23"), + getTransformationFunction("|", Splitter.on("|"), false).apply("-1.23|3.13|23") + ); + assertEquals( + ImmutableList.of("-1.23", "3.13", "23", "foo", "-9"), + getTransformationFunction("|", Splitter.on("|"), false).apply("-1.23|3.13|23|foo|-9") + ); + } + + @Test + public void testInputWithDelimiterAndParserEnabled() + { + assertNull( + getTransformationFunction("|", Splitter.on("|"), true).apply(null) + ); + assertEquals( + NullHandling.emptyToNullIfNeeded(""), + getTransformationFunction("|", Splitter.on("|"), true).apply("") + ); + assertEquals( + ImmutableList.of("foo", "boo"), + getTransformationFunction("|", Splitter.on("|"), true).apply("foo|boo") + ); + assertEquals( + ImmutableList.of(1L, 2L, 3L), + getTransformationFunction("|", Splitter.on("|"), true).apply("1|2|3") + ); + assertEquals( + ImmutableList.of(1L, -2L, 3L, 0L, -2L), + getTransformationFunction("|", Splitter.on("|"), true).apply("1|-2|3|0|-2") + ); + assertEquals( + 100L, + getTransformationFunction("|", Splitter.on("|"), true).apply("100") + ); + assertEquals( + 1.23, + getTransformationFunction("|", Splitter.on("|"), true).apply("1.23") + ); + assertEquals( + -2.0, + getTransformationFunction("|", Splitter.on("|"), true).apply("-2.0") + ); + assertEquals( + 100.0, + getTransformationFunction("$", Splitter.on("|"), true).apply("1e2") + ); + assertEquals( + ImmutableList.of(1L, 2L, 3L), + getTransformationFunction("|", Splitter.on("|"), true).apply("1|2|3") + ); + assertEquals( + ImmutableList.of(1L, -2L, 3L, 0L, -2L), + getTransformationFunction("|", Splitter.on("|"), true).apply("1|-2|3|0|-2") + ); + assertEquals( + ImmutableList.of(-1.0, -2.2, 3.1, 0.2, -2.1), + getTransformationFunction("|", Splitter.on("|"), true).apply("-1.0|-2.2|3.1|0.2|-2.1") + ); + + // Some mixed types + assertEquals( + ImmutableList.of(-1.23, 3.13, 23L), + getTransformationFunction("|", Splitter.on("|"), true).apply("-1.23|3.13|23") + ); + assertEquals( + ImmutableList.of(-1.23, 3.13, 23L, "foo", -9L), + getTransformationFunction("|", Splitter.on("|"), true).apply("-1.23|3.13|23|foo|-9") + ); + } + + @Test + public void testInputWithoutDelimiterAndNumberParsingDisabled() + { + assertNull( + getTransformationFunction("|", Splitter.on("$"), false).apply(null) + ); + assertEquals( + NullHandling.emptyToNullIfNeeded(""), + getTransformationFunction("|", Splitter.on("$"), false).apply("") + ); + assertEquals( + "foo|boo", + getTransformationFunction("$", Splitter.on("$"), false).apply("foo|boo") + ); + assertEquals( + "100", + getTransformationFunction("$", Splitter.on("$"), false).apply("100") + ); + assertEquals( + "1.23", + getTransformationFunction("$", Splitter.on("$"), false).apply("1.23") + ); + assertEquals( + "-2.0", + getTransformationFunction("$", Splitter.on("$"), false).apply("-2.0") + ); + assertEquals( + "1e2", + getTransformationFunction("$", Splitter.on("$"), false).apply("1e2") + ); + assertEquals( + "1|2|3", + getTransformationFunction("$", Splitter.on("$"), false).apply("1|2|3") + ); + assertEquals( + "1|-2|3|0|-2", + getTransformationFunction("$", Splitter.on("$"), false).apply("1|-2|3|0|-2") + ); + assertEquals( + "-1.0|-2.2|3.1|0.2|-2.1", + getTransformationFunction("$", Splitter.on("$"), false).apply("-1.0|-2.2|3.1|0.2|-2.1") + ); + + // Some mixed types + assertEquals( + "-1.23|3.13|23", + getTransformationFunction("$", Splitter.on("$"), false).apply("-1.23|3.13|23") + ); + assertEquals( + "-1.23|3.13|23|foo|-9", + getTransformationFunction("$", Splitter.on("$"), false).apply("-1.23|3.13|23|foo|-9") + ); + } + + @Test + public void testInputWithoutDelimiterAndNumberParsingEnabled() + { + assertNull( + getTransformationFunction("$", Splitter.on("$"), true).apply(null) + ); + assertEquals( + NullHandling.emptyToNullIfNeeded(""), + getTransformationFunction("$", Splitter.on("$"), true).apply("") + ); + assertEquals( + "foo|boo", + getTransformationFunction("$", Splitter.on("$"), true).apply("foo|boo") + ); + assertEquals( + 100L, + getTransformationFunction("$", Splitter.on("$"), true).apply("100") + ); + assertEquals( + 1.23, + getTransformationFunction("$", Splitter.on("$"), true).apply("1.23") + ); + assertEquals( + -2.0, + getTransformationFunction("$", Splitter.on("$"), true).apply("-2.0") + ); + assertEquals( + 100.0, + getTransformationFunction("$", Splitter.on("$"), true).apply("1e2") + ); + assertEquals( + "1|2|3", + getTransformationFunction("$", Splitter.on("$"), true).apply("1|2|3") + ); + assertEquals( + "1|-2|3|0|-2", + getTransformationFunction("$", Splitter.on("$"), true).apply("1|-2|3|0|-2") + ); + assertEquals( + "-1.0|-2.2|3.1|0.2|-2.1", + getTransformationFunction("$", Splitter.on("$"), true).apply("-1.0|-2.2|3.1|0.2|-2.1") + ); + + // Some mixed types + assertEquals( + "-1.23|3.13|23", + getTransformationFunction("$", Splitter.on("$"), true).apply("-1.23|3.13|23") + ); + assertEquals( + "-1.23|3.13|23|foo|-9", + getTransformationFunction("$", Splitter.on("$"), true).apply("-1.23|3.13|23|foo|-9") + ); } } diff --git a/processing/src/test/java/org/apache/druid/java/util/metrics/OshiSysMonitorTest.java b/processing/src/test/java/org/apache/druid/java/util/metrics/OshiSysMonitorTest.java index 3b86b1efd76..d1c0f0ed1ba 100644 --- a/processing/src/test/java/org/apache/druid/java/util/metrics/OshiSysMonitorTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/metrics/OshiSysMonitorTest.java @@ -72,7 +72,7 @@ public class OshiSysMonitorTest { ServiceEmitter serviceEmitter = Mockito.mock(ServiceEmitter.class); - OshiSysMonitor sysMonitorOshi = new OshiSysMonitor(); + OshiSysMonitor sysMonitorOshi = createMonitor(new SystemInfo()); serviceEmitter.start(); sysMonitorOshi.monitor(serviceEmitter); @@ -84,7 +84,7 @@ public class OshiSysMonitorTest public void testDefaultFeedSysMonitorOshi() { StubServiceEmitter emitter = new StubServiceEmitter("dev/monitor-test", "localhost:0000"); - OshiSysMonitor m = new OshiSysMonitor(); + OshiSysMonitor m = createMonitor(new SystemInfo()); m.start(); m.monitor(emitter); // Sleep for 2 sec to get all metrics which are difference of prev and now metrics @@ -103,7 +103,7 @@ public class OshiSysMonitorTest Mockito.when(mem.getAvailable()).thenReturn(16L); Mockito.when(hal.getMemory()).thenReturn(mem); - OshiSysMonitor m = new OshiSysMonitor(si); + OshiSysMonitor m = createMonitor(si); m.start(); m.monitorMemStats(emitter); m.stop(); @@ -116,6 +116,27 @@ public class OshiSysMonitorTest emitter.verifyValue("sys/mem/free", 16L); } + @Test + public void testMemStatsSkipOthers() + { + StubServiceEmitter emitter = new StubServiceEmitter("dev/monitor-test", "localhost:0000"); + GlobalMemory mem = Mockito.mock(GlobalMemory.class); + Mockito.when(mem.getTotal()).thenReturn(64L); + Mockito.when(mem.getAvailable()).thenReturn(16L); + Mockito.when(hal.getMemory()).thenReturn(mem); + + OshiSysMonitor m = createMonitor(si, ImmutableList.of("mem")); + m.start(); + m.doMonitor(emitter); + m.stop(); + Assert.assertEquals(3, emitter.getEvents().size()); + emitter.verifyEmitted("sys/mem/max", 1); + emitter.verifyEmitted("sys/mem/used", 1); + emitter.verifyEmitted("sys/mem/free", 1); + emitter.verifyEmitted("sys/swap/pageIn", 0); + emitter.verifyEmitted("sys/fs/max", 0); + } + @Test public void testSwapStats() { @@ -129,7 +150,7 @@ public class OshiSysMonitorTest Mockito.when(mem.getVirtualMemory()).thenReturn(swap); Mockito.when(hal.getMemory()).thenReturn(mem); - OshiSysMonitor m = new OshiSysMonitor(si); + OshiSysMonitor m = createMonitor(si); m.start(); m.monitorSwapStats(emitter); Assert.assertEquals(4, emitter.getEvents().size()); @@ -177,7 +198,7 @@ public class OshiSysMonitorTest Mockito.when(fileSystem.getFileStores(true)).thenReturn(osFileStores); Mockito.when(os.getFileSystem()).thenReturn(fileSystem); - OshiSysMonitor m = new OshiSysMonitor(si); + OshiSysMonitor m = createMonitor(si); m.start(); m.monitorFsStats(emitter); Assert.assertEquals(8, emitter.getEvents().size()); @@ -248,7 +269,7 @@ public class OshiSysMonitorTest List hwDiskStores = ImmutableList.of(disk1, disk2); Mockito.when(hal.getDiskStores()).thenReturn(hwDiskStores); - OshiSysMonitor m = new OshiSysMonitor(si); + OshiSysMonitor m = createMonitor(si); m.start(); m.monitorDiskStats(emitter); Assert.assertEquals(0, emitter.getEvents().size()); @@ -338,7 +359,7 @@ public class OshiSysMonitorTest List networkIFS = ImmutableList.of(net1); Mockito.when(hal.getNetworkIFs()).thenReturn(networkIFS); - OshiSysMonitor m = new OshiSysMonitor(si); + OshiSysMonitor m = createMonitor(si); m.start(); m.monitorNetStats(emitter); Assert.assertEquals(0, emitter.getEvents().size()); @@ -436,7 +457,7 @@ public class OshiSysMonitorTest Mockito.when(processor.getProcessorCpuLoadTicks()).thenReturn(procTicks); Mockito.when(hal.getProcessor()).thenReturn(processor); - OshiSysMonitor m = new OshiSysMonitor(si); + OshiSysMonitor m = createMonitor(si); m.start(); m.monitorCpuStats(emitter); Assert.assertEquals(0, emitter.getEvents().size()); @@ -533,7 +554,7 @@ public class OshiSysMonitorTest Mockito.when(processor.getSystemLoadAverage(3)).thenReturn(la); Mockito.when(hal.getProcessor()).thenReturn(processor); - OshiSysMonitor m = new OshiSysMonitor(si); + OshiSysMonitor m = createMonitor(si); m.start(); m.monitorSysStats(emitter); Assert.assertEquals(4, emitter.getEvents().size()); @@ -567,7 +588,7 @@ public class OshiSysMonitorTest Mockito.when(ipstats.getTCPv4Stats()).thenReturn(tcpv4); Mockito.when(os.getInternetProtocolStats()).thenReturn(ipstats); - OshiSysMonitor m = new OshiSysMonitor(si); + OshiSysMonitor m = createMonitor(si); m.start(); m.monitorTcpStats(emitter); @@ -607,5 +628,13 @@ public class OshiSysMonitorTest } } + private OshiSysMonitor createMonitor(SystemInfo si) + { + return createMonitor(si, ImmutableList.of()); + } + private OshiSysMonitor createMonitor(SystemInfo si, List categories) + { + return new OshiSysMonitor(ImmutableMap.of(), new OshiSysMonitorConfig(categories), si); + } } diff --git a/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java b/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java index d313749fef8..4ab7c7be0ed 100644 --- a/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java +++ b/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java @@ -20,13 +20,10 @@ package org.apache.druid.math.expr; import com.google.common.collect.ImmutableMap; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import java.util.Map; @@ -48,9 +45,6 @@ public class OutputTypeTest extends InitializedNullHandlingTest .build() ); - @Rule - public ExpectedException expectedException = ExpectedException.none(); - @Test public void testConstantsAndIdentifiers() { @@ -541,7 +535,6 @@ public class OutputTypeTest extends InitializedNullHandlingTest ExpressionType.DOUBLE, ExpressionTypeConversion.operator(ExpressionType.LONG, ExpressionType.STRING) ); - // unless it is an array, and those have to be the same Assert.assertEquals( ExpressionType.LONG_ARRAY, ExpressionTypeConversion.operator(ExpressionType.LONG_ARRAY, ExpressionType.LONG_ARRAY) @@ -554,7 +547,30 @@ public class OutputTypeTest extends InitializedNullHandlingTest ExpressionType.STRING_ARRAY, ExpressionTypeConversion.operator(ExpressionType.STRING_ARRAY, ExpressionType.STRING_ARRAY) ); - + Assert.assertEquals( + ExpressionType.LONG_ARRAY, + ExpressionTypeConversion.operator(ExpressionType.LONG_ARRAY, ExpressionType.LONG) + ); + Assert.assertEquals( + ExpressionType.STRING_ARRAY, + ExpressionTypeConversion.operator(ExpressionType.STRING, ExpressionType.LONG_ARRAY) + ); + Assert.assertEquals( + ExpressionType.DOUBLE_ARRAY, + ExpressionTypeConversion.operator(ExpressionType.LONG_ARRAY, ExpressionType.DOUBLE_ARRAY) + ); + Assert.assertEquals( + ExpressionType.DOUBLE_ARRAY, + ExpressionTypeConversion.operator(ExpressionType.LONG, ExpressionType.DOUBLE_ARRAY) + ); + Assert.assertEquals( + ExpressionType.STRING_ARRAY, + ExpressionTypeConversion.operator(ExpressionType.LONG_ARRAY, ExpressionType.STRING_ARRAY) + ); + Assert.assertEquals( + ExpressionType.STRING_ARRAY, + ExpressionTypeConversion.operator(ExpressionType.STRING_ARRAY, ExpressionType.DOUBLE_ARRAY) + ); ExpressionType nested = ExpressionType.fromColumnType(ColumnType.NESTED_DATA); Assert.assertEquals( nested, @@ -619,7 +635,6 @@ public class OutputTypeTest extends InitializedNullHandlingTest ExpressionType.STRING, ExpressionTypeConversion.function(ExpressionType.STRING, ExpressionType.STRING) ); - // unless it is an array, and those have to be the same Assert.assertEquals( ExpressionType.LONG_ARRAY, ExpressionTypeConversion.function(ExpressionType.LONG_ARRAY, ExpressionType.LONG_ARRAY) @@ -632,6 +647,30 @@ public class OutputTypeTest extends InitializedNullHandlingTest ExpressionType.STRING_ARRAY, ExpressionTypeConversion.function(ExpressionType.STRING_ARRAY, ExpressionType.STRING_ARRAY) ); + Assert.assertEquals( + ExpressionType.DOUBLE_ARRAY, + ExpressionTypeConversion.function(ExpressionType.DOUBLE_ARRAY, ExpressionType.LONG_ARRAY) + ); + Assert.assertEquals( + ExpressionType.STRING_ARRAY, + ExpressionTypeConversion.function(ExpressionType.DOUBLE_ARRAY, ExpressionType.STRING_ARRAY) + ); + Assert.assertEquals( + ExpressionType.STRING_ARRAY, + ExpressionTypeConversion.function(ExpressionType.STRING_ARRAY, ExpressionType.LONG_ARRAY) + ); + Assert.assertEquals( + ExpressionType.STRING_ARRAY, + ExpressionTypeConversion.function(ExpressionType.STRING, ExpressionType.LONG_ARRAY) + ); + Assert.assertEquals( + ExpressionType.DOUBLE_ARRAY, + ExpressionTypeConversion.function(ExpressionType.LONG_ARRAY, ExpressionType.DOUBLE_ARRAY) + ); + Assert.assertEquals( + ExpressionType.DOUBLE_ARRAY, + ExpressionTypeConversion.function(ExpressionType.LONG, ExpressionType.DOUBLE_ARRAY) + ); ExpressionType nested = ExpressionType.fromColumnType(ColumnType.NESTED_DATA); Assert.assertEquals( nested, @@ -719,27 +758,6 @@ public class OutputTypeTest extends InitializedNullHandlingTest ); } - @Test - public void testAutoConversionArrayMismatchArrays() - { - expectedException.expect(IAE.class); - ExpressionTypeConversion.function(ExpressionType.DOUBLE_ARRAY, ExpressionType.LONG_ARRAY); - } - - @Test - public void testAutoConversionArrayMismatchArrayScalar() - { - expectedException.expect(IAE.class); - ExpressionTypeConversion.function(ExpressionType.DOUBLE_ARRAY, ExpressionType.LONG); - } - - @Test - public void testAutoConversionArrayMismatchScalarArray() - { - expectedException.expect(IAE.class); - ExpressionTypeConversion.function(ExpressionType.DOUBLE, ExpressionType.LONG_ARRAY); - } - private void assertOutputType(String expression, Expr.InputBindingInspector inspector, ExpressionType outputType) { final Expr expr = Parser.parse(expression, ExprMacroTable.nil(), false); diff --git a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java index 75119bfb897..863c01bb390 100644 --- a/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java +++ b/processing/src/test/java/org/apache/druid/query/CursorGranularizerTest.java @@ -30,18 +30,19 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.testing.InitializedNullHandlingTest; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -57,8 +58,9 @@ public class CursorGranularizerTest extends InitializedNullHandlingTest @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - private StorageAdapter adapter; + private CursorFactory cursorFactory; private TimeBoundaryInspector timeBoundaryInspector; + private Interval interval; @Before public void setup() throws IOException @@ -152,21 +154,22 @@ public class CursorGranularizerTest extends InitializedNullHandlingTest .tmpDir(temporaryFolder.newFolder()); final QueryableIndex index = bob.buildMMappedIndex(); - adapter = new QueryableIndexStorageAdapter(index); + interval = index.getDataInterval(); + cursorFactory = new QueryableIndexCursorFactory(index); timeBoundaryInspector = QueryableIndexTimeBoundaryInspector.create(index); } @Test public void testGranularizeFullScan() { - try (CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); CursorGranularizer granularizer = CursorGranularizer.create( cursor, timeBoundaryInspector, Order.ASCENDING, Granularities.HOUR, - adapter.getInterval() + interval ); final ColumnSelectorFactory selectorFactory = cursor.getColumnSelectorFactory(); @@ -212,14 +215,14 @@ public class CursorGranularizerTest extends InitializedNullHandlingTest final CursorBuildSpec descending = CursorBuildSpec.builder() .setPreferredOrdering(Cursors.descendingTimeOrder()) .build(); - try (CursorHolder cursorHolder = adapter.makeCursorHolder(descending)) { + try (CursorHolder cursorHolder = cursorFactory.makeCursorHolder(descending)) { final Cursor cursor = cursorHolder.asCursor(); CursorGranularizer granularizer = CursorGranularizer.create( cursor, timeBoundaryInspector, Order.DESCENDING, Granularities.HOUR, - adapter.getInterval() + interval ); final ColumnSelectorFactory selectorFactory = cursor.getColumnSelectorFactory(); diff --git a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java index 23215b95d1e..9efeeb642e5 100644 --- a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java +++ b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java @@ -147,7 +147,8 @@ public class NestedDataTestUtils null, false, false, - 0 + 0, + null ); public static final TransformSpec SIMPLE_DATA_TSV_TRANSFORM = new TransformSpec( diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/first/StringFirstTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/first/StringFirstTimeseriesQueryTest.java index 2c44d080deb..ee0f5c052c6 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/first/StringFirstTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/first/StringFirstTimeseriesQueryTest.java @@ -37,13 +37,13 @@ import org.apache.druid.query.timeseries.TimeseriesQueryEngine; import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; @@ -147,7 +147,7 @@ public class StringFirstTimeseriesQueryTest extends InitializedNullHandlingTest final Iterable> iiResults = engine.process( query, - new IncrementalIndexStorageAdapter(incrementalIndex), + new IncrementalIndexCursorFactory(incrementalIndex), new IncrementalIndexTimeBoundaryInspector(incrementalIndex), defaultTimeseriesQueryMetrics ).toList(); @@ -155,7 +155,7 @@ public class StringFirstTimeseriesQueryTest extends InitializedNullHandlingTest final Iterable> qiResults = engine.process( query, - new QueryableIndexStorageAdapter(queryableIndex), + new QueryableIndexCursorFactory(queryableIndex), QueryableIndexTimeBoundaryInspector.create(queryableIndex), defaultTimeseriesQueryMetrics ).toList(); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/last/StringLastTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/last/StringLastTimeseriesQueryTest.java index 1c9b4033cd9..fb189685544 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/last/StringLastTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/firstlast/last/StringLastTimeseriesQueryTest.java @@ -37,13 +37,13 @@ import org.apache.druid.query.timeseries.TimeseriesQueryEngine; import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; @@ -151,7 +151,7 @@ public class StringLastTimeseriesQueryTest extends InitializedNullHandlingTest final Iterable> iiResults = engine.process( query, - new IncrementalIndexStorageAdapter(incrementalIndex), + new IncrementalIndexCursorFactory(incrementalIndex), new IncrementalIndexTimeBoundaryInspector(incrementalIndex), defaultTimeseriesQueryMetrics ).toList(); @@ -159,7 +159,7 @@ public class StringLastTimeseriesQueryTest extends InitializedNullHandlingTest final Iterable> qiResults = engine.process( query, - new QueryableIndexStorageAdapter(queryableIndex), + new QueryableIndexCursorFactory(queryableIndex), QueryableIndexTimeBoundaryInspector.create(queryableIndex), defaultTimeseriesQueryMetrics ).toList(); diff --git a/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java b/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java index b14edb2d17b..c9fe553469a 100644 --- a/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java +++ b/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java @@ -49,6 +49,7 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest new NestedDataExpressions.JsonPathsExprMacro(), new NestedDataExpressions.JsonKeysExprMacro(), new NestedDataExpressions.JsonObjectExprMacro(), + new NestedDataExpressions.JsonMergeExprMacro(JSON_MAPPER), new NestedDataExpressions.JsonValueExprMacro(), new NestedDataExpressions.JsonQueryExprMacro(), new NestedDataExpressions.JsonQueryArrayExprMacro(), @@ -112,6 +113,63 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest Assert.assertEquals(ImmutableMap.of("a", "hello", "b", "world"), ((Map) eval.value()).get("y")); } + @Test + public void testJsonMergeExpression() throws JsonProcessingException + { + Expr expr = Parser.parse("json_merge('{\"a\":\"x\"}','{\"b\":\"y\"}')", MACRO_TABLE); + ExprEval eval = expr.eval(inputBindings); + Assert.assertEquals("{\"a\":\"x\",\"b\":\"y\"}", JSON_MAPPER.writeValueAsString(eval.value())); + Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type()); + + expr = Parser.parse("json_merge('{\"a\":\"x\"}', null)", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertEquals("{\"a\":\"x\"}", JSON_MAPPER.writeValueAsString(eval.value())); + Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type()); + + expr = Parser.parse("json_merge('{\"a\":\"x\"}','{\"b\":\"y\"}','{\"c\":[1,2,3]}')", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertEquals("{\"a\":\"x\",\"b\":\"y\",\"c\":[1,2,3]}", JSON_MAPPER.writeValueAsString(eval.value())); + Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type()); + + expr = Parser.parse("json_merge(json_object('a', 'x'),json_object('b', 'y'))", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertEquals("{\"a\":\"x\",\"b\":\"y\"}", JSON_MAPPER.writeValueAsString(eval.value())); + Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type()); + + expr = Parser.parse("json_merge('{\"a\":\"x\"}',json_merge('{\"a\":\"z\"}','{\"a\":\"y\"}'))", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertEquals("{\"a\":\"y\"}", JSON_MAPPER.writeValueAsString(eval.value())); + Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type()); + + expr = Parser.parse("json_merge('[\"a\", \"b\"]', '[\"c\", \"d\"]')", MACRO_TABLE); + eval = expr.eval(inputBindings); + Assert.assertEquals("[\"a\",\"b\",\"c\",\"d\"]", JSON_MAPPER.writeValueAsString(eval.value())); + Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type()); + } + + @Test + public void testJsonMergeOverflow() throws JsonProcessingException + { + Expr.ObjectBinding input1 = InputBindings.forInputSuppliers( + new ImmutableMap.Builder>() + .put("attr", InputBindings.inputSupplier(ExpressionType.NESTED_DATA, () -> ImmutableMap.of("key", "blah", "value", "blahblah"))) + .build() + ); + Expr.ObjectBinding input2 = InputBindings.forInputSuppliers( + new ImmutableMap.Builder>() + .put("attr", InputBindings.inputSupplier(ExpressionType.NESTED_DATA, () -> ImmutableMap.of("key", "blah2", "value", "blahblah2"))) + .build() + ); + + Expr expr = Parser.parse("json_merge(json_object(), json_object(json_value(attr, '$.key'), json_value(attr, '$.value')))", MACRO_TABLE); + ExprEval eval = expr.eval(input1); + Assert.assertEquals("{\"blah\":\"blahblah\"}", JSON_MAPPER.writeValueAsString(eval.value())); + Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type()); + eval = expr.eval(input2); + Assert.assertEquals("{\"blah2\":\"blahblah2\"}", JSON_MAPPER.writeValueAsString(eval.value())); + Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type()); + } + @Test public void testJsonKeysExpression() { diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java index dcbcb79724f..1eb8774c207 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java @@ -336,7 +336,6 @@ public class GroupByLimitPushDownInsufficientBufferTest extends InitializedNullH final GroupingEngine groupingEngine = new GroupingEngine( druidProcessingConfig, configSupplier, - bufferPool, groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), @@ -346,7 +345,6 @@ public class GroupByLimitPushDownInsufficientBufferTest extends InitializedNullH final GroupingEngine tooSmallEngine = new GroupingEngine( tooSmallDruidProcessingConfig, configSupplier, - bufferPool2, tooSmallGroupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), @@ -355,12 +353,14 @@ public class GroupByLimitPushDownInsufficientBufferTest extends InitializedNullH groupByFactory = new GroupByQueryRunnerFactory( groupingEngine, - new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool) + new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool), + bufferPool ); tooSmallGroupByFactory = new GroupByQueryRunnerFactory( tooSmallEngine, - new GroupByQueryQueryToolChest(tooSmallEngine, tooSmallGroupByResourcesReservationPool) + new GroupByQueryQueryToolChest(tooSmallEngine, tooSmallGroupByResourcesReservationPool), + bufferPool2 ); } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java index 9b9fea03231..987d9a03f2d 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java @@ -588,7 +588,6 @@ public class GroupByLimitPushDownMultiNodeMergeTest extends InitializedNullHandl final GroupingEngine groupingEngineBroker = new GroupingEngine( druidProcessingConfig, configSupplier, - bufferPool, groupByResourcesReservationPoolBroker, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), @@ -597,7 +596,6 @@ public class GroupByLimitPushDownMultiNodeMergeTest extends InitializedNullHandl final GroupingEngine groupingEngineHistorical = new GroupingEngine( druidProcessingConfig, configSupplier, - bufferPool, groupByResourcesReservationPoolHistorical, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), @@ -606,7 +604,6 @@ public class GroupByLimitPushDownMultiNodeMergeTest extends InitializedNullHandl final GroupingEngine groupingEngineHistorical2 = new GroupingEngine( druidProcessingConfig, configSupplier, - bufferPool, groupByResourcesReservationPoolHistorical2, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), @@ -615,17 +612,20 @@ public class GroupByLimitPushDownMultiNodeMergeTest extends InitializedNullHandl groupByFactoryBroker = new GroupByQueryRunnerFactory( groupingEngineBroker, - new GroupByQueryQueryToolChest(groupingEngineBroker, groupByResourcesReservationPoolBroker) + new GroupByQueryQueryToolChest(groupingEngineBroker, groupByResourcesReservationPoolBroker), + bufferPool ); groupByFactoryHistorical = new GroupByQueryRunnerFactory( groupingEngineHistorical, - new GroupByQueryQueryToolChest(groupingEngineHistorical, groupByResourcesReservationPoolHistorical) + new GroupByQueryQueryToolChest(groupingEngineHistorical, groupByResourcesReservationPoolHistorical), + bufferPool ); groupByFactoryHistorical2 = new GroupByQueryRunnerFactory( groupingEngineHistorical2, - new GroupByQueryQueryToolChest(groupingEngineHistorical2, groupByResourcesReservationPoolHistorical2) + new GroupByQueryQueryToolChest(groupingEngineHistorical2, groupByResourcesReservationPoolHistorical2), + bufferPool ); } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java index d461876c9bd..9632ceba0b1 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java @@ -245,7 +245,6 @@ public class GroupByMultiSegmentTest extends InitializedNullHandlingTest final GroupingEngine groupingEngine = new GroupingEngine( druidProcessingConfig, configSupplier, - bufferPool, groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), @@ -254,7 +253,8 @@ public class GroupByMultiSegmentTest extends InitializedNullHandlingTest groupByFactory = new GroupByQueryRunnerFactory( groupingEngine, - new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool) + new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool), + bufferPool ); } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java index 5de602b8772..77f116aa6e4 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryMergeBufferTest.java @@ -130,7 +130,6 @@ public class GroupByQueryMergeBufferTest extends InitializedNullHandlingTest final GroupingEngine groupingEngine = new GroupingEngine( PROCESSING_CONFIG, configSupplier, - BUFFER_POOL, groupByResourcesReservationPool, TestHelper.makeJsonMapper(), mapper, @@ -140,7 +139,7 @@ public class GroupByQueryMergeBufferTest extends InitializedNullHandlingTest groupingEngine, groupByResourcesReservationPool ); - return new GroupByQueryRunnerFactory(groupingEngine, toolChest); + return new GroupByQueryRunnerFactory(groupingEngine, toolChest, BUFFER_POOL); } private static final CloseableStupidPool BUFFER_POOL = new CloseableStupidPool<>( diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java index d9aefd5f55e..15c127c68cb 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java @@ -28,9 +28,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import org.apache.druid.collections.BlockingPool; import org.apache.druid.collections.DefaultBlockingPool; -import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.SerializablePair; -import org.apache.druid.collections.StupidPool; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.Row; import org.apache.druid.jackson.AggregatorsModule; @@ -1293,10 +1291,6 @@ public class GroupByQueryQueryToolChestTest extends InitializedNullHandlingTest final Supplier bufferSupplier = () -> ByteBuffer.allocateDirect(processingConfig.intermediateComputeSizeBytes()); - final NonBlockingPool bufferPool = new StupidPool<>( - "GroupByQueryEngine-bufferPool", - bufferSupplier - ); final BlockingPool mergeBufferPool = new DefaultBlockingPool<>( bufferSupplier, processingConfig.getNumMergeBuffers() @@ -1305,7 +1299,6 @@ public class GroupByQueryQueryToolChestTest extends InitializedNullHandlingTest final GroupingEngine groupingEngine = new GroupingEngine( processingConfig, queryConfigSupplier, - bufferPool, groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java index 69debeb902c..a3eb5ef724d 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFailureTest.java @@ -97,18 +97,19 @@ public class GroupByQueryRunnerFailureTest ) { final Supplier configSupplier = Suppliers.ofInstance(config); - GroupByResourcesReservationPool groupByResourcesReservationPool = new GroupByResourcesReservationPool(MERGE_BUFFER_POOL, config); + GroupByResourcesReservationPool groupByResourcesReservationPool = + new GroupByResourcesReservationPool(MERGE_BUFFER_POOL, config); final GroupingEngine groupingEngine = new GroupingEngine( DEFAULT_PROCESSING_CONFIG, configSupplier, - BUFFER_POOL, groupByResourcesReservationPool, TestHelper.makeJsonMapper(), mapper, QueryRunnerTestHelper.NOOP_QUERYWATCHER ); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool); - return new GroupByQueryRunnerFactory(groupingEngine, toolChest); + final GroupByQueryQueryToolChest toolChest = + new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool); + return new GroupByQueryRunnerFactory(groupingEngine, toolChest, BUFFER_POOL); } private static final CloseableStupidPool BUFFER_POOL = new CloseableStupidPool<>( diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java index 53050b20ad5..cc36d000957 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java @@ -361,7 +361,6 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest final GroupingEngine groupingEngine = new GroupingEngine( processingConfig, configSupplier, - bufferPools.getProcessingPool(), groupByResourcesReservationPool, mapper, mapper, @@ -373,7 +372,7 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest DefaultGroupByQueryMetricsFactory.instance(), groupByResourcesReservationPool ); - return new GroupByQueryRunnerFactory(groupingEngine, toolChest); + return new GroupByQueryRunnerFactory(groupingEngine, toolChest, bufferPools.getProcessingPool()); } @Parameterized.Parameters(name = "{0}") @@ -13662,7 +13661,7 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest private void assumeTimeOrdered() { try (final CursorHolder cursorHolder = - originalRunner.getSegment().asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + originalRunner.getSegment().asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Assume.assumeTrue(Cursors.getTimeOrdering(cursorHolder.getOrdering()) == Order.ASCENDING); } } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java index adbcbc187bb..a0aef93381f 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java @@ -659,7 +659,7 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest boolean allCanVectorize = segments.stream() .allMatch( s -> { - final CursorHolder cursorHolder = s.asStorageAdapter() + final CursorHolder cursorHolder = s.asCursorFactory() .makeCursorHolder(spec); final boolean canVectorize = cursorHolder.canVectorize(); cursorHolder.close(); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java index b75616c4593..a28e782bf65 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java @@ -292,7 +292,6 @@ public class NestedQueryPushDownTest extends InitializedNullHandlingTest final GroupingEngine engine1 = new GroupingEngine( druidProcessingConfig, configSupplier, - bufferPool, groupByResourcesReservationPool, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), @@ -301,7 +300,6 @@ public class NestedQueryPushDownTest extends InitializedNullHandlingTest final GroupingEngine engine2 = new GroupingEngine( druidProcessingConfig, configSupplier, - bufferPool, groupByResourcesReservationPool2, TestHelper.makeJsonMapper(), new ObjectMapper(new SmileFactory()), @@ -310,12 +308,14 @@ public class NestedQueryPushDownTest extends InitializedNullHandlingTest groupByFactory = new GroupByQueryRunnerFactory( engine1, - new GroupByQueryQueryToolChest(engine1, groupByResourcesReservationPool) + new GroupByQueryQueryToolChest(engine1, groupByResourcesReservationPool), + bufferPool ); groupByFactory2 = new GroupByQueryRunnerFactory( engine2, - new GroupByQueryQueryToolChest(engine2, groupByResourcesReservationPool2) + new GroupByQueryQueryToolChest(engine2, groupByResourcesReservationPool2), + bufferPool ); } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java index 3976a20bd2d..02bae02eb6f 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/UnnestGroupByQueryRunnerTest.java @@ -178,7 +178,6 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest final GroupingEngine groupingEngine = new GroupingEngine( processingConfig, configSupplier, - bufferPools.getProcessingPool(), groupByResourcesReservationPool, TestHelper.makeJsonMapper(), mapper, @@ -186,7 +185,7 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest ); final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool); - return new GroupByQueryRunnerFactory(groupingEngine, toolChest); + return new GroupByQueryRunnerFactory(groupingEngine, toolChest, bufferPools.getProcessingPool()); } @Parameterized.Parameters(name = "{0}") diff --git a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java index 15982c2caab..d570621eca7 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngineIteratorTest.java @@ -31,10 +31,10 @@ import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.epinephelinae.VectorGrouper; import org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine.VectorGroupByEngineIterator; import org.apache.druid.segment.ColumnProcessors; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.vector.VectorCursor; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -62,10 +62,10 @@ public class VectorGroupByEngineIteratorTest extends InitializedNullHandlingTest .setDimensions(new DefaultDimensionSpec("market", null, null)) .setAggregatorSpecs(factory) .build(); - final StorageAdapter storageAdapter = new QueryableIndexStorageAdapter(TestIndex.getMMappedTestIndex()); + final CursorFactory cursorFactory = new QueryableIndexCursorFactory(TestIndex.getMMappedTestIndex()); final QueryableIndexTimeBoundaryInspector timeBoundaryInspector = QueryableIndexTimeBoundaryInspector.create(TestIndex.getMMappedTestIndex()); - final CursorHolder cursorHolder = storageAdapter.makeCursorHolder( + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder( GroupingEngine.makeCursorBuildSpec(query, null) ); final ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[4096]); @@ -83,7 +83,6 @@ public class VectorGroupByEngineIteratorTest extends InitializedNullHandlingTest query, new GroupByQueryConfig(), GroupByQueryRunnerTest.DEFAULT_PROCESSING_CONFIG, - storageAdapter, timeBoundaryInspector, cursor, cursorHolder.getTimeOrder(), diff --git a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java index 142b23b8391..8b93bc06011 100644 --- a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java @@ -21,7 +21,6 @@ package org.apache.druid.query.lookup; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSortedMap; -import com.google.common.collect.Lists; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.extraction.MapLookupExtractor; @@ -29,14 +28,15 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; -import org.apache.druid.segment.DimensionDictionarySelector; -import org.apache.druid.segment.RowBasedStorageAdapter; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.RowBasedCursorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.timeline.SegmentId; import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; import org.junit.Assert; import org.junit.Test; @@ -118,6 +118,7 @@ public class LookupSegmentTest @Test public void test_asQueryableIndex() { + Assert.assertNull(LOOKUP_SEGMENT.as(QueryableIndex.class)); Assert.assertNull(LOOKUP_SEGMENT.asQueryableIndex()); } @@ -129,32 +130,14 @@ public class LookupSegmentTest .add("k", ColumnType.STRING) .add("v", ColumnType.STRING) .build(), - LOOKUP_SEGMENT.asStorageAdapter().getRowSignature() - ); - } - - @Test - public void test_asStorageAdapter_getAvailableDimensions() - { - Assert.assertEquals( - ImmutableList.of("k", "v"), - Lists.newArrayList(LOOKUP_SEGMENT.asStorageAdapter().getAvailableDimensions().iterator()) - ); - } - - @Test - public void test_asStorageAdapter_getAvailableMetrics() - { - Assert.assertEquals( - ImmutableList.of(), - Lists.newArrayList(LOOKUP_SEGMENT.asStorageAdapter().getAvailableMetrics()) + LOOKUP_SEGMENT.asCursorFactory().getRowSignature() ); } @Test public void test_asStorageAdapter_getColumnCapabilitiesK() { - final ColumnCapabilities capabilities = LOOKUP_SEGMENT.asStorageAdapter().getColumnCapabilities("k"); + final ColumnCapabilities capabilities = LOOKUP_SEGMENT.asCursorFactory().getColumnCapabilities("k"); Assert.assertEquals(ValueType.STRING, capabilities.getType()); @@ -168,7 +151,7 @@ public class LookupSegmentTest @Test public void test_asStorageAdapter_getColumnCapabilitiesV() { - final ColumnCapabilities capabilities = LOOKUP_SEGMENT.asStorageAdapter().getColumnCapabilities("v"); + final ColumnCapabilities capabilities = LOOKUP_SEGMENT.asCursorFactory().getColumnCapabilities("v"); // Note: the "v" column does not actually have multiple values, but the RowBasedStorageAdapter doesn't allow // reporting complete single-valued capabilities. It would be good to change this in the future, so query engines @@ -178,37 +161,13 @@ public class LookupSegmentTest Assert.assertFalse(capabilities.isDictionaryEncoded().isTrue()); } - @Test - public void test_asStorageAdapter_getInterval() - { - Assert.assertEquals(Intervals.ETERNITY, LOOKUP_SEGMENT.asStorageAdapter().getInterval()); - } - - @Test - public void test_asStorageAdapter_getDimensionCardinalityK() - { - Assert.assertEquals( - DimensionDictionarySelector.CARDINALITY_UNKNOWN, - LOOKUP_SEGMENT.asStorageAdapter().getDimensionCardinality("k") - ); - } - - @Test - public void test_asStorageAdapter_getDimensionCardinalityV() - { - Assert.assertEquals( - DimensionDictionarySelector.CARDINALITY_UNKNOWN, - LOOKUP_SEGMENT.asStorageAdapter().getDimensionCardinality("v") - ); - } - @Test public void test_asStorageAdapter_makeCursor() { final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("1970/PT1H")) .build(); - try (final CursorHolder cursorHolder = LOOKUP_SEGMENT.asStorageAdapter().makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = LOOKUP_SEGMENT.asCursorFactory().makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); final List> kvs = new ArrayList<>(); @@ -234,8 +193,8 @@ public class LookupSegmentTest @Test public void test_asStorageAdapter_isRowBasedAdapter() { - // This allows us to assume that RowBasedStorageAdapterTest is further exercising makeCursor and verifying misc. + // This allows us to assume that LookupSegmentTest is further exercising makeCursor and verifying misc. // methods like getMinTime, getMaxTime, getMetadata, etc, without checking them explicitly in _this_ test class. - Assert.assertThat(LOOKUP_SEGMENT.asStorageAdapter(), CoreMatchers.instanceOf(RowBasedStorageAdapter.class)); + MatcherAssert.assertThat(LOOKUP_SEGMENT.asCursorFactory(), CoreMatchers.instanceOf(RowBasedCursorFactory.class)); } } diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java index 8a5605bc673..bd787a9d92a 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java @@ -59,7 +59,6 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.data.ListIndexed; import org.apache.druid.segment.data.ObjectStrategy; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -430,9 +429,6 @@ public class SegmentAnalyzerTest extends InitializedNullHandlingTest QueryableIndex mockIndex = EasyMock.createMock(QueryableIndex.class); EasyMock.expect(mockIndex.getNumRows()).andReturn(100).atLeastOnce(); EasyMock.expect(mockIndex.getColumnNames()).andReturn(Collections.singletonList("x")).atLeastOnce(); - EasyMock.expect(mockIndex.getAvailableDimensions()) - .andReturn(new ListIndexed<>(Collections.singletonList("x"))) - .atLeastOnce(); EasyMock.expect(mockIndex.getColumnCapabilities(ColumnHolder.TIME_COLUMN_NAME)) .andReturn(ColumnCapabilitiesImpl.createDefault().setType(ColumnType.LONG)) .atLeastOnce(); diff --git a/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryQueryToolChestTest.java new file mode 100644 index 00000000000..2100b36e57d --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryQueryToolChestTest.java @@ -0,0 +1,162 @@ +/* + * 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.query.operator; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.Druids; +import org.apache.druid.query.FrameSignaturePair; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.ResultSerializationMode; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.operator.window.WindowOperatorFactory; +import org.apache.druid.query.operator.window.ranking.WindowRowNumberProcessor; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.spec.LegacySegmentSpec; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; + +public class WindowOperatorQueryQueryToolChestTest extends InitializedNullHandlingTest +{ + + private final WindowOperatorQueryQueryToolChest toolchest = new WindowOperatorQueryQueryToolChest(); + + @Test + public void mergeResultsWithRowResultSerializationMode() + { + RowSignature inputSignature = RowSignature.builder() + .add("length", ColumnType.LONG) + .build(); + RowSignature outputSignature = RowSignature.builder() + .addAll(inputSignature) + .add("w0", ColumnType.LONG) + .build(); + + final WindowOperatorQuery query = new WindowOperatorQuery( + new QueryDataSource( + Druids.newScanQueryBuilder() + .dataSource(new TableDataSource("test")) + .intervals(new LegacySegmentSpec(Intervals.ETERNITY)) + .columns("length") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(new HashMap<>()) + .build() + ), + new LegacySegmentSpec(Intervals.ETERNITY), + new HashMap<>(), + outputSignature, + ImmutableList.of( + new WindowOperatorFactory(new WindowRowNumberProcessor("w0")) + ), + ImmutableList.of() + ); + List results = toolchest.mergeResults( + (queryPlus, responseContext) -> Sequences.simple( + Collections.singletonList( + MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of("length", new IntArrayColumn(new int[]{1, 5, 10})) + ) + ) + ) + ).run(QueryPlus.wrap(query)).toList(); + + Assert.assertTrue(results.get(0) instanceof Object[]); + Assert.assertEquals(3, results.size()); + List expectedResults = ImmutableList.of( + new Object[]{1, 1}, + new Object[]{5, 2}, + new Object[]{10, 3} + ); + + for (int i = 0; i < 3; ++i) { + Assert.assertArrayEquals(expectedResults.get(i), (Object[]) results.get(i)); + } + } + + @Test + public void mergeResultsWithFrameResultSerializationMode() + { + RowSignature inputSignature = RowSignature.builder() + .add("length", ColumnType.LONG) + .build(); + RowSignature outputSignature = RowSignature.builder() + .addAll(inputSignature) + .add("w0", ColumnType.LONG) + .build(); + + final WindowOperatorQuery query = new WindowOperatorQuery( + new QueryDataSource( + Druids.newScanQueryBuilder() + .dataSource(new TableDataSource("test")) + .intervals(new LegacySegmentSpec(Intervals.ETERNITY)) + .columns("length") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(new HashMap<>()) + .build() + ), + new LegacySegmentSpec(Intervals.ETERNITY), + Collections.singletonMap(ResultSerializationMode.CTX_SERIALIZATION_PARAMETER, ResultSerializationMode.FRAMES.toString()), + outputSignature, + ImmutableList.of( + new WindowOperatorFactory(new WindowRowNumberProcessor("w0")) + ), + ImmutableList.of() + ); + List results = toolchest.mergeResults( + (queryPlus, responseContext) -> Sequences.simple( + Collections.singletonList( + MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of("length", new IntArrayColumn(new int[]{1, 5, 10})) + ) + ) + ) + ).run(QueryPlus.wrap(query)).toList(); + + Assert.assertTrue(results.get(0) instanceof FrameSignaturePair); + Assert.assertEquals(1, results.size()); + + FrameReader reader = FrameReader.create(((FrameSignaturePair) results.get(0)).getRowSignature()); + List> resultRows = FrameTestUtil.readRowsFromCursorFactory( + reader.makeCursorFactory(((FrameSignaturePair) results.get(0)).getFrame()) + ).toList(); + + List> expectedResults = ImmutableList.of( + ImmutableList.of(1L, 1L), + ImmutableList.of(5L, 2L), + ImmutableList.of(10L, 3L) + ); + Assertions.assertEquals(expectedResults, resultRows); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/RowsAndColumnsHelper.java b/processing/src/test/java/org/apache/druid/query/operator/window/RowsAndColumnsHelper.java index 2636156b53c..cdc84620ab0 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/RowsAndColumnsHelper.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/RowsAndColumnsHelper.java @@ -29,6 +29,7 @@ import org.apache.druid.query.rowsandcols.column.ColumnAccessor; import org.apache.druid.segment.column.ColumnType; import org.junit.Assert; +import java.util.ArrayList; import java.util.Collection; import java.util.LinkedHashMap; import java.util.Map; @@ -280,6 +281,17 @@ public class RowsAndColumnsHelper } else { Assert.assertEquals(msg, ((Long) expectedVal).longValue(), accessor.getLong(i)); } + } else if (expectedVal instanceof Object[]) { + Object actualVal = accessor.getObject(i); + if (expectedNulls[i]) { + Assert.assertNull(msg, accessor.getObject(i)); + } else { + if (actualVal instanceof ArrayList) { + Assert.assertArrayEquals(msg, (Object[]) expectedVals[i], ((ArrayList) actualVal).toArray()); + } else { + Assert.assertArrayEquals(msg, (Object[]) expectedVals[i], (Object[]) actualVal); + } + } } else { if (expectedNulls[i]) { Assert.assertNull(msg, accessor.getObject(i)); diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/CursorFactoryRowsAndColumnsTest.java similarity index 69% rename from processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java rename to processing/src/test/java/org/apache/druid/query/rowsandcols/CursorFactoryRowsAndColumnsTest.java index 4395ce9dd5b..6aeac726e6d 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/StorageAdapterRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/CursorFactoryRowsAndColumnsTest.java @@ -21,24 +21,24 @@ package org.apache.druid.query.rowsandcols; import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumns; import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumnsTest; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; import java.util.function.Function; -public class StorageAdapterRowsAndColumnsTest extends RowsAndColumnsTestBase +public class CursorFactoryRowsAndColumnsTest extends RowsAndColumnsTestBase { - public StorageAdapterRowsAndColumnsTest() + public CursorFactoryRowsAndColumnsTest() { - super(StorageAdapterRowsAndColumns.class); + super(CursorFactoryRowsAndColumns.class); } - public static Function MAKER = - StorageAdapterRowsAndColumnsTest::buildFrame; + public static Function MAKER = + CursorFactoryRowsAndColumnsTest::buildFrame; - private static StorageAdapterRowsAndColumns buildFrame(MapOfColumnsRowsAndColumns input) + private static CursorFactoryRowsAndColumns buildFrame(MapOfColumnsRowsAndColumns input) { try (ColumnBasedFrameRowsAndColumns fRAC = ColumnBasedFrameRowsAndColumnsTest.buildFrame(input)) { - return new StorageAdapterRowsAndColumns(fRAC.as(StorageAdapter.class)); + return new CursorFactoryRowsAndColumns(fRAC.as(CursorFactory.class)); } } } diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/NoAsRowsAndColumns.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/NoAsRowsAndColumns.java index 422c87c8b7c..16cd44e870b 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/NoAsRowsAndColumns.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/NoAsRowsAndColumns.java @@ -21,7 +21,6 @@ package org.apache.druid.query.rowsandcols; import org.apache.druid.query.rowsandcols.column.Column; -import javax.annotation.Nullable; import java.util.Collection; public class NoAsRowsAndColumns implements RowsAndColumns @@ -50,12 +49,4 @@ public class NoAsRowsAndColumns implements RowsAndColumns { return rac.findColumn(name); } - - @Nullable - @Override - public T as(Class clazz) - { - // Pretend like this doesn't implement any semantic interfaces - return null; - } } diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java index 7b639b3d48d..2973df31ab5 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java @@ -69,7 +69,7 @@ public abstract class RowsAndColumnsTestBase new Object[]{ConcatRowsAndColumns.class, ConcatRowsAndColumnsTest.MAKER}, new Object[]{RearrangedRowsAndColumns.class, RearrangedRowsAndColumnsTest.MAKER}, new Object[]{ColumnBasedFrameRowsAndColumns.class, ColumnBasedFrameRowsAndColumnsTest.MAKER}, - new Object[]{StorageAdapterRowsAndColumns.class, StorageAdapterRowsAndColumnsTest.MAKER}, + new Object[]{CursorFactoryRowsAndColumns.class, CursorFactoryRowsAndColumnsTest.MAKER}, new Object[]{RowBasedFrameRowsAndColumns.class, RowBasedFrameRowsAndColumnsTest.MAKER} ); } diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumnsTest.java index acfcbe6f83e..f6a10e01146 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumnsTest.java @@ -37,7 +37,15 @@ public class ColumnBasedFrameRowsAndColumnsTest extends RowsAndColumnsTestBase public static ColumnBasedFrameRowsAndColumns buildFrame(MapOfColumnsRowsAndColumns input) { - LazilyDecoratedRowsAndColumns rac = new LazilyDecoratedRowsAndColumns(input, null, null, null, OffsetLimit.limit(Integer.MAX_VALUE), null, null); + LazilyDecoratedRowsAndColumns rac = new LazilyDecoratedRowsAndColumns( + input, + null, + null, + null, + OffsetLimit.limit(Integer.MAX_VALUE), + null, + null + ); rac.numRows(); // materialize return (ColumnBasedFrameRowsAndColumns) rac.getBase(); diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/EvaluateRowsAndColumnsTest.java similarity index 70% rename from processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java rename to processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/EvaluateRowsAndColumnsTest.java index c26508694d4..e2cee35a8e9 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/EvaluateRowsAndColumnsTest.java @@ -26,7 +26,7 @@ import org.apache.druid.query.operator.window.RowsAndColumnsHelper; import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns; import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -38,33 +38,45 @@ import java.util.function.Function; import static org.junit.Assert.assertEquals; import static org.junit.Assume.assumeNotNull; -public class TestVirtualColumnEvaluationRowsAndColumnsTest extends SemanticTestBase +public class EvaluateRowsAndColumnsTest extends SemanticTestBase { - public TestVirtualColumnEvaluationRowsAndColumnsTest(String name, Function fn) + public EvaluateRowsAndColumnsTest(String name, Function fn) { super(name, fn); } @Test - public void testMaterializeVirtualColumns() + public void testMaterializeColumns() { Object[][] vals = new Object[][] { - {1L, "a", 123L, 0L}, - {2L, "a", 456L, 1L}, - {3L, "b", 789L, 2L}, - {4L, "b", 123L, 3L}, + {1L, "a", 123L, new Object[]{"xyz", "x"}, 0L}, + {2L, "a", 456L, new Object[]{"abc"}, 1L}, + {3L, "b", 789L, new Object[]{null}, 2L}, + {4L, null, 123L, null, 3L}, }; RowSignature siggy = RowSignature.builder() .add("__time", ColumnType.LONG) .add("dim", ColumnType.STRING) .add("val", ColumnType.LONG) + .add("array", ColumnType.STRING_ARRAY) .add("arrayIndex", ColumnType.LONG) .build(); final RowsAndColumns base = make(MapOfColumnsRowsAndColumns.fromRowObjects(vals, siggy)); - assumeNotNull("skipping: StorageAdapter not supported", base.as(StorageAdapter.class)); + Object[] expectedArr = new Object[][] { + {"xyz", "x"}, + {"abc"}, + {null}, + null + }; + + new RowsAndColumnsHelper() + .expectColumn("array", expectedArr, ColumnType.STRING_ARRAY) + .validate(base); + + assumeNotNull("skipping: CursorFactory not supported", base.as(CursorFactory.class)); LazilyDecoratedRowsAndColumns ras = new LazilyDecoratedRowsAndColumns( base, @@ -82,12 +94,18 @@ public class TestVirtualColumnEvaluationRowsAndColumnsTest extends SemanticTestB // do the materialziation ras.numRows(); - assertEquals(Lists.newArrayList("__time", "dim", "val", "arrayIndex", "expr"), ras.getColumnNames()); + assertEquals(Lists.newArrayList("__time", "dim", "val", "array", "arrayIndex", "expr"), ras.getColumnNames()); new RowsAndColumnsHelper() .expectColumn("expr", new long[] {123 * 2, 456L * 2, 789 * 2, 123 * 2}) .validate(ras); - } + new RowsAndColumnsHelper() + .expectColumn("dim", new String[] {"a", "a", "b", null}, ColumnType.STRING) + .validate(ras); + new RowsAndColumnsHelper() + .expectColumn("array", expectedArr, ColumnType.STRING_ARRAY) + .validate(ras); + } } diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java index 023a6294ab9..41295f48017 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java @@ -19,6 +19,7 @@ package org.apache.druid.query.rowsandcols.semantic; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -32,6 +33,9 @@ import org.apache.druid.query.operator.OffsetLimit; import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; +import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumns; +import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumnsTest; import org.apache.druid.segment.ArrayListSegment; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; @@ -214,6 +218,39 @@ public class RowsAndColumnsDecoratorTest extends SemanticTestBase } } + @Test + public void testDecoratorWithColumnBasedFrameRAC() + { + RowSignature siggy = RowSignature.builder() + .add("colA", ColumnType.LONG) + .add("colB", ColumnType.LONG) + .build(); + + Object[][] vals = new Object[][]{ + {1L, 4L}, + {2L, -4L}, + {3L, 3L}, + {4L, -3L}, + {5L, 4L}, + {6L, 82L}, + {7L, -90L}, + {8L, 4L}, + {9L, 0L}, + {10L, 0L} + }; + + MapOfColumnsRowsAndColumns input = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "colA", new IntArrayColumn(new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + "colB", new IntArrayColumn(new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0}) + ) + ); + + ColumnBasedFrameRowsAndColumns frc = ColumnBasedFrameRowsAndColumnsTest.buildFrame(input); + + validateDecorated(frc, siggy, vals, null, null, OffsetLimit.NONE, null); + } + private void validateDecorated( RowsAndColumns base, RowSignature siggy, @@ -254,7 +291,7 @@ public class RowsAndColumnsDecoratorTest extends SemanticTestBase if (interval != null) { builder.setInterval(interval); } - try (final CursorHolder cursorHolder = seggy.asStorageAdapter().makeCursorHolder(builder.build())) { + try (final CursorHolder cursorHolder = seggy.asCursorFactory().makeCursorHolder(builder.build())) { final Cursor cursor = cursorHolder.asCursor(); vals = new ArrayList<>(); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java index 2bfd47fb699..776f89cc8b5 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java @@ -158,7 +158,7 @@ public class ScanQueryResultOrderingTest extends InitializedNullHandlingTest // Try every limit up to one past the total number of rows. final Set limits = new TreeSet<>(); - final int totalNumRows = SEGMENTS.stream().mapToInt(s -> s.asStorageAdapter().getNumRows()).sum(); + int totalNumRows = 19; for (int i = 0; i <= totalNumRows + 1; i++) { limits.add(i); } diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java index 727086b7be3..c99b1f0fb08 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/NestedDataTimeseriesQueryTest.java @@ -666,7 +666,7 @@ public class NestedDataTimeseriesQueryTest extends InitializedNullHandlingTest boolean allCanVectorize = segments.stream() .allMatch( s -> { - final CursorHolder cursorHolder = s.asStorageAdapter() + final CursorHolder cursorHolder = s.asCursorFactory() .makeCursorHolder(spec); final boolean canVectorize = cursorHolder.canVectorize(); cursorHolder.close(); diff --git a/processing/src/test/java/org/apache/druid/query/topn/PooledTopNAlgorithmTest.java b/processing/src/test/java/org/apache/druid/query/topn/PooledTopNAlgorithmTest.java index 341a3c18078..5bc121daa7b 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/PooledTopNAlgorithmTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/PooledTopNAlgorithmTest.java @@ -20,7 +20,6 @@ package org.apache.druid.query.topn; import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.segment.StorageAdapter; import org.easymock.EasyMock; import org.junit.Test; @@ -31,14 +30,14 @@ public class PooledTopNAlgorithmTest @Test public void testCleanupWithNullParams() { - PooledTopNAlgorithm pooledTopNAlgorithm = new PooledTopNAlgorithm(EasyMock.mock(StorageAdapter.class), null, null); + PooledTopNAlgorithm pooledTopNAlgorithm = new PooledTopNAlgorithm(null, EasyMock.mock(TopNCursorInspector.class), null); pooledTopNAlgorithm.cleanup(null); } @Test public void cleanup() { - PooledTopNAlgorithm pooledTopNAlgorithm = new PooledTopNAlgorithm(EasyMock.mock(StorageAdapter.class), null, null); + PooledTopNAlgorithm pooledTopNAlgorithm = new PooledTopNAlgorithm(null, EasyMock.mock(TopNCursorInspector.class), null); PooledTopNAlgorithm.PooledTopNParams params = EasyMock.createMock(PooledTopNAlgorithm.PooledTopNParams.class); ResourceHolder resourceHolder = EasyMock.createMock(ResourceHolder.class); EasyMock.expect(params.getResultsBufHolder()).andReturn(resourceHolder).times(1); diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java index ae320ac4261..c374602ecf4 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java @@ -30,24 +30,20 @@ import org.apache.druid.query.aggregation.DoubleMinAggregatorFactory; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorHolder; +import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IdLookup; -import org.apache.druid.segment.Metadata; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.SimpleTopNOptimizationInspector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.IndexedInts; -import org.joda.time.Interval; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; import javax.annotation.Nullable; import java.util.List; -public class TopNMetricSpecOptimizationsTest +public class TopNMetricSpecOptimizationsTest extends InitializedNullHandlingTest { private static final List AGGS = Lists.newArrayList( Iterables.concat( @@ -76,15 +72,13 @@ public class TopNMetricSpecOptimizationsTest .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) .build(); - StorageAdapter adapter = - makeFakeStorageAdapter("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality); DimensionSelector dimSelector = makeFakeDimSelector(cardinality); BaseTopNAlgorithm.AggregatorArrayProvider arrayProviderToTest = new BaseTopNAlgorithm.AggregatorArrayProvider( dimSelector, query, - cardinality, - adapter + makeCursorInspector("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality), + cardinality ); arrayProviderToTest.ignoreAfterThreshold(); @@ -110,16 +104,14 @@ public class TopNMetricSpecOptimizationsTest .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) .build(); - StorageAdapter adapter = - makeFakeStorageAdapter("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality); DimensionSelector dimSelector = makeFakeDimSelector(cardinality); BaseTopNAlgorithm.AggregatorArrayProvider arrayProviderToTest = new BaseTopNAlgorithm.AggregatorArrayProvider( dimSelector, query, - cardinality, - adapter + makeCursorInspector("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality), + cardinality ); arrayProviderToTest.ignoreAfterThreshold(); @@ -145,16 +137,14 @@ public class TopNMetricSpecOptimizationsTest .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) .build(); - StorageAdapter adapter = - makeFakeStorageAdapter("2018-05-30T00:00:00Z", "2018-05-31T00:00:00Z", cardinality); DimensionSelector dimSelector = makeFakeDimSelector(cardinality); BaseTopNAlgorithm.AggregatorArrayProvider arrayProviderToTest = new BaseTopNAlgorithm.AggregatorArrayProvider( dimSelector, query, - cardinality, - adapter + makeCursorInspector("2018-05-30T00:00:00Z", "2018-05-31T00:00:00Z", cardinality), + cardinality ); arrayProviderToTest.ignoreAfterThreshold(); @@ -181,15 +171,13 @@ public class TopNMetricSpecOptimizationsTest .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) .build(); - StorageAdapter adapter = - makeFakeStorageAdapter("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality); DimensionSelector dimSelector = makeFakeDimSelector(cardinality); BaseTopNAlgorithm.AggregatorArrayProvider arrayProviderToTest = new BaseTopNAlgorithm.AggregatorArrayProvider( dimSelector, query, - cardinality, - adapter + makeCursorInspector("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality), + cardinality ); arrayProviderToTest.ignoreAfterThreshold(); @@ -215,17 +203,13 @@ public class TopNMetricSpecOptimizationsTest .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) .build(); - - StorageAdapter adapter = - makeFakeStorageAdapter("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality); - DimensionSelector dimSelector = makeFakeDimSelector(cardinality); BaseTopNAlgorithm.AggregatorArrayProvider arrayProviderToTest = new BaseTopNAlgorithm.AggregatorArrayProvider( dimSelector, query, - cardinality, - adapter + makeCursorInspector("2018-05-30T00:00:00Z", "2018-05-30T01:00:00Z", cardinality), + cardinality ); Pair thePair = arrayProviderToTest.computeStartEnd(cardinality); @@ -233,85 +217,22 @@ public class TopNMetricSpecOptimizationsTest Assert.assertEquals(new Integer(cardinality), thePair.rhs); } - private StorageAdapter makeFakeStorageAdapter(String start, String end, int cardinality) + private TopNCursorInspector makeCursorInspector(String start, String end, int cardinality) { - StorageAdapter adapter = new StorageAdapter() - { - @Override - public Interval getInterval() - { - return Intervals.of(start + "/" + end); - } - - @Override - public int getDimensionCardinality(String column) - { - return cardinality; - } - - // stubs below this line not important for tests - - @Override - public Indexed getAvailableDimensions() - { - return null; - } - - @Override - public Iterable getAvailableMetrics() - { - return null; - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - return null; - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - return null; - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - return null; - } - - @Override - public int getNumRows() - { - return 0; - } - - @Override - public Metadata getMetadata() - { - return null; - } - - @Override - public CursorHolder makeCursorHolder(CursorBuildSpec spec) - { - return new CursorHolder() + return new TopNCursorInspector( + new ColumnInspector() { @Nullable @Override - public Cursor asCursor() + public ColumnCapabilities getColumnCapabilities(String column) { return null; } - }; - } - }; - - return adapter; + }, + new SimpleTopNOptimizationInspector(true), + Intervals.of(start + "/" + end), + cardinality + ); } private DimensionSelector makeFakeDimSelector(int cardinality) diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java index c674e575d70..285ccf31a60 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java @@ -6381,7 +6381,7 @@ public class TopNQueryRunnerTest extends InitializedNullHandlingTest private void assumeTimeOrdered() { try (final CursorHolder cursorHolder = - runner.getSegment().asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + runner.getSegment().asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Assume.assumeTrue(Cursors.getTimeOrdering(cursorHolder.getOrdering()) == Order.ASCENDING); } } diff --git a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java index 788181859ff..2c570981f65 100644 --- a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java @@ -26,15 +26,13 @@ import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.BuiltInTypesModule; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.nested.StructuredData; @@ -147,8 +145,8 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest index.add(makeInputRow(minTimestamp + 4, true, STRING_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, STRING_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -184,7 +182,7 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0))); Assert.assertNull(dimensionSelector.getObject()); - Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(STRING_COL).toColumnType()); + Assert.assertEquals(ColumnType.STRING, cursorFactory.getColumnCapabilities(STRING_COL).toColumnType()); } } @@ -200,8 +198,8 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest index.add(makeInputRow(minTimestamp + 4, true, LONG_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -267,7 +265,7 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest ); Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject()); } - Assert.assertEquals(ColumnType.LONG, storageAdapter.getColumnCapabilities(LONG_COL).toColumnType()); + Assert.assertEquals(ColumnType.LONG, cursorFactory.getColumnCapabilities(LONG_COL).toColumnType()); } } @@ -283,8 +281,8 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest index.add(makeInputRow(minTimestamp + 4, true, DOUBLE_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -350,7 +348,7 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest ); Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject()); } - Assert.assertEquals(ColumnType.DOUBLE, storageAdapter.getColumnCapabilities(DOUBLE_COL).toColumnType()); + Assert.assertEquals(ColumnType.DOUBLE, cursorFactory.getColumnCapabilities(DOUBLE_COL).toColumnType()); } } @@ -366,8 +364,8 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest index.add(makeInputRow(minTimestamp + 4, true, STRING_ARRAY_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec( STRING_ARRAY_COL, @@ -397,7 +395,7 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest Assert.assertNull(valueSelector.getObject()); Assert.assertEquals( ColumnType.STRING_ARRAY, - storageAdapter.getColumnCapabilities(STRING_ARRAY_COL).toColumnType() + cursorFactory.getColumnCapabilities(STRING_ARRAY_COL).toColumnType() ); } } @@ -414,8 +412,8 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest index.add(makeInputRow(minTimestamp + 4, true, VARIANT_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -442,7 +440,7 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest cursor.advance(); Assert.assertNull(valueSelector.getObject()); Assert.assertNull(dimensionSelector.getObject()); - Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(VARIANT_COL).toColumnType()); + Assert.assertEquals(ColumnType.STRING, cursorFactory.getColumnCapabilities(VARIANT_COL).toColumnType()); } } @@ -458,8 +456,8 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest index.add(makeInputRow(minTimestamp + 4, true, NESTED_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -482,7 +480,7 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest cursor.advance(); Assert.assertNull(valueSelector.getObject()); - Assert.assertEquals(ColumnType.NESTED_DATA, storageAdapter.getColumnCapabilities(NESTED_COL).toColumnType()); + Assert.assertEquals(ColumnType.NESTED_DATA, cursorFactory.getColumnCapabilities(NESTED_COL).toColumnType()); } } @@ -494,18 +492,17 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest long minTimestamp = System.currentTimeMillis(); IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( - new IncrementalIndexSchema( - minTimestamp, - new TimestampSpec(TIME_COL, "millis", null), - Granularities.NONE, - VirtualColumns.EMPTY, - DimensionsSpec.builder() - .setDimensions(ImmutableList.of(new AutoTypeColumnSchema(NESTED_COL, ColumnType.STRING))) - .useSchemaDiscovery(true) - .build(), - new AggregatorFactory[0], - false - ) + IncrementalIndexSchema.builder() + .withMinTimestamp(minTimestamp) + .withTimestampSpec(new TimestampSpec(TIME_COL, "millis", null)) + .withDimensionsSpec( + DimensionsSpec.builder() + .setDimensions(ImmutableList.of(new AutoTypeColumnSchema(NESTED_COL, ColumnType.STRING))) + .useSchemaDiscovery(true) + .build() + ) + .withRollup(false) + .build() ) .setMaxRowCount(1000) .build(); @@ -517,8 +514,8 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest index.add(makeInputRow(minTimestamp + 4, true, NESTED_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -545,7 +542,7 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest Assert.assertNull(valueSelector.getObject()); Assert.assertNull(dimensionSelector.getObject()); - Assert.assertEquals(ColumnType.STRING, storageAdapter.getColumnCapabilities(NESTED_COL).toColumnType()); + Assert.assertEquals(ColumnType.STRING, cursorFactory.getColumnCapabilities(NESTED_COL).toColumnType()); } } @@ -699,15 +696,16 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest { IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( - new IncrementalIndexSchema( - minTimestamp, - new TimestampSpec(TIME_COL, "millis", null), - Granularities.NONE, - VirtualColumns.EMPTY, - DimensionsSpec.builder().useSchemaDiscovery(true).build(), - new AggregatorFactory[0], - false - ) + IncrementalIndexSchema.builder() + .withMinTimestamp(minTimestamp) + .withTimestampSpec(new TimestampSpec(TIME_COL, "millis", null)) + .withDimensionsSpec( + DimensionsSpec.builder() + .useSchemaDiscovery(true) + .build() + ) + .withRollup(false) + .build() ) .setMaxRowCount(1000) .build(); diff --git a/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java b/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java new file mode 100644 index 00000000000..2c256c1d2ad --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/CompleteSegmentTest.java @@ -0,0 +1,49 @@ +/* + * 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.segment; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +import java.io.IOException; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class CompleteSegmentTest +{ + @Test + public void testCloseSegment() throws IOException + { + Segment segment = mock(Segment.class); + CompleteSegment completeSegment = new CompleteSegment(null, segment); + completeSegment.close(); + verify(segment).close(); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(CompleteSegment.class) + .withNonnullFields("segment", "dataSegment") + .usingGetClass() + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java b/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java new file mode 100644 index 00000000000..52c129379fc --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java @@ -0,0 +1,268 @@ +/* + * 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.segment; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.collections.CloseableDefaultBlockingPool; +import org.apache.druid.collections.CloseableStupidPool; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.Druids; +import org.apache.druid.query.IterableRowsCursorHelper; +import org.apache.druid.query.Result; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; +import org.apache.druid.query.groupby.GroupingEngine; +import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesQueryEngine; +import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.query.topn.TopNQuery; +import org.apache.druid.query.topn.TopNQueryBuilder; +import org.apache.druid.query.topn.TopNQueryEngine; +import org.apache.druid.query.topn.TopNResultValue; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.utils.CloseableUtils; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.nio.ByteBuffer; +import java.util.List; + +public class CursorHolderPreaggTest extends InitializedNullHandlingTest +{ + private CloseableStupidPool bufferPool; + private GroupingEngine groupingEngine; + private TopNQueryEngine topNQueryEngine; + private TimeseriesQueryEngine timeseriesQueryEngine; + + private CursorFactory cursorFactory; + private Segment segment; + + @Rule + public final CloserRule closer = new CloserRule(false); + + @Before + public void setup() + { + bufferPool = closer.closeLater( + new CloseableStupidPool<>( + "CursorHolderPreaggTest-bufferPool", + () -> ByteBuffer.allocate(50000) + ) + ); + topNQueryEngine = new TopNQueryEngine(bufferPool); + timeseriesQueryEngine = new TimeseriesQueryEngine(bufferPool); + groupingEngine = new GroupingEngine( + new DruidProcessingConfig(), + GroupByQueryConfig::new, + new GroupByResourcesReservationPool( + closer.closeLater( + new CloseableDefaultBlockingPool<>( + () -> ByteBuffer.allocate(50000), + 4 + ) + ), + new GroupByQueryConfig() + ), + TestHelper.makeJsonMapper(), + TestHelper.makeSmileMapper(), + (query, future) -> { + } + ); + + this.cursorFactory = new CursorFactory() + { + private final RowSignature rowSignature = RowSignature.builder() + .add("a", ColumnType.STRING) + .add("b", ColumnType.STRING) + .add("cnt", ColumnType.LONG) + .build(); + + private final Pair cursorAndCloser = IterableRowsCursorHelper.getCursorFromIterable( + ImmutableList.of( + new Object[]{"a", "aa", 5L}, + new Object[]{"a", "aa", 6L}, + new Object[]{"b", "bb", 7L} + ), + rowSignature + ); + + @Override + public CursorHolder makeCursorHolder(CursorBuildSpec spec) + { + return new CursorHolder() + { + @Nullable + @Override + public Cursor asCursor() + { + return cursorAndCloser.lhs; + } + + @Override + public boolean isPreAggregated() + { + return true; + } + + @Override + public void close() + { + CloseableUtils.closeAndWrapExceptions(cursorAndCloser.rhs); + } + }; + } + + @Override + public RowSignature getRowSignature() + { + return rowSignature; + } + + @Override + @Nullable + public ColumnCapabilities getColumnCapabilities(String column) + { + return rowSignature.getColumnCapabilities(column); + } + }; + + segment = new Segment() + { + @Override + public SegmentId getId() + { + return SegmentId.dummy("test"); + } + + @Override + public Interval getDataInterval() + { + return Intervals.ETERNITY; + } + + @Nullable + @Override + public QueryableIndex asQueryableIndex() + { + return null; + } + + @Override + public CursorFactory asCursorFactory() + { + return cursorFactory; + } + + @Override + public void close() + { + + } + }; + } + + @Test + public void testTopn() + { + final TopNQuery topNQuery = new TopNQueryBuilder().dataSource("test") + .granularity(Granularities.ALL) + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .dimension("a") + .aggregators(new CountAggregatorFactory("cnt")) + .metric("cnt") + .threshold(10) + .build(); + Sequence> results = topNQueryEngine.query( + topNQuery, + segment, + null + ); + + List> rows = results.toList(); + Assert.assertEquals(1, rows.size()); + // the cnt column is treated as pre-aggregated, so the values of the rows are summed + Assert.assertEquals(2, rows.get(0).getValue().getValue().size()); + Assert.assertEquals(11L, rows.get(0).getValue().getValue().get(0).getLongMetric("cnt").longValue()); + Assert.assertEquals(7L, rows.get(0).getValue().getValue().get(1).getLongMetric("cnt").longValue()); + } + + @Test + public void testGroupBy() + { + final GroupByQuery query = GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addDimension("b") + .addAggregator(new CountAggregatorFactory("cnt")) + .build(); + + Sequence results = groupingEngine.process( + query, + cursorFactory, + null, + bufferPool, + null + ); + List rows = results.toList(); + Assert.assertEquals(2, rows.size()); + // the cnt column is treated as pre-aggregated, so the values of the rows are summed + Assert.assertArrayEquals(new Object[]{"a", "aa", 11L}, rows.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bb", 7L}, rows.get(1).getArray()); + } + + @Test + public void testTimeseries() + { + TimeseriesQuery timeseriesQuery = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .granularity(Granularities.ALL) + .aggregators(new CountAggregatorFactory("cnt")) + .build(); + Sequence> results = timeseriesQueryEngine.process( + timeseriesQuery, + cursorFactory, + null, + null + ); + List> rows = results.toList(); + Assert.assertEquals(1, rows.size()); + // the cnt column is treated as pre-aggregated, so the values of the rows are summed + Assert.assertEquals(18L, (long) rows.get(0).getValue().getLongMetric("cnt")); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java index 9dbd866654c..9c88ab0dc8f 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java @@ -394,7 +394,7 @@ public class IndexBuilder { // Determine row signature by building an mmapped index first. try (final QueryableIndex index = buildMMappedIndex()) { - final RowSignature signature = new QueryableIndexStorageAdapter(index).getRowSignature(); + final RowSignature signature = new QueryableIndexCursorFactory(index).getRowSignature(); return new RowBasedSegment<>( SegmentId.dummy("IndexBuilder"), @@ -409,8 +409,8 @@ public class IndexBuilder { // Build mmapped index first, then copy over. try (final QueryableIndex index = buildMMappedIndex()) { - return FrameTestUtil.adapterToFrameSegment( - new QueryableIndexStorageAdapter(index), + return FrameTestUtil.cursorFactoryToFrameSegment( + new QueryableIndexCursorFactory(index), frameType, SegmentId.dummy("IndexBuilder") ); diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java index b8c9721b5a0..5c519f88458 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -65,6 +65,7 @@ import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Rule; @@ -262,7 +263,7 @@ public class IndexMergerTestBase extends InitializedNullHandlingTest ImmutableList.of("3", "4", timestamp, 1L), ImmutableList.of("3", "4", timestamp + 1, 1L) ), - FrameTestUtil.readRowsFromAdapter(new QueryableIndexStorageAdapter(index), null, false).toList() + FrameTestUtil.readRowsFromCursorFactory(new QueryableIndexCursorFactory(index)).toList() ); } @@ -330,7 +331,7 @@ public class IndexMergerTestBase extends InitializedNullHandlingTest ImmutableList.of("3", "4", timestamp, 2L), ImmutableList.of("3", "4", timestamp + 1, 1L) ), - FrameTestUtil.readRowsFromAdapter(new QueryableIndexStorageAdapter(index), null, false).toList() + FrameTestUtil.readRowsFromCursorFactory(new QueryableIndexCursorFactory(index)).toList() ); } @@ -1705,10 +1706,14 @@ public class IndexMergerTestBase extends InitializedNullHandlingTest indexSpec, -1 ); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(indexIO.loadIndex( - merged))); - Assert.assertEquals(ImmutableSet.of("A", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); - + final QueryableIndexSegment segment = new QueryableIndexSegment( + closer.closeLater(indexIO.loadIndex(merged)), + SegmentId.dummy("test") + ); + Assert.assertEquals( + ImmutableSet.of("A", "C"), + Arrays.stream(segment.as(PhysicalSegmentInspector.class).getMetadata().getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toSet()) + ); } @Test @@ -1777,9 +1782,14 @@ public class IndexMergerTestBase extends InitializedNullHandlingTest indexSpec, -1 ); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(indexIO.loadIndex( - merged))); - Assert.assertEquals(ImmutableSet.of("A", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); + final QueryableIndexSegment segment = new QueryableIndexSegment( + closer.closeLater(indexIO.loadIndex(merged)), + SegmentId.dummy("test") + ); + Assert.assertEquals( + ImmutableSet.of("A", "C"), + Arrays.stream(segment.as(PhysicalSegmentInspector.class).getMetadata().getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toSet()) + ); } @@ -1844,10 +1854,14 @@ public class IndexMergerTestBase extends InitializedNullHandlingTest ); // Since D was not present in any of the indices, it is not present in the output - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(indexIO.loadIndex( - merged))); - Assert.assertEquals(ImmutableSet.of("A", "B", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); - + final QueryableIndexSegment segment = new QueryableIndexSegment( + closer.closeLater(indexIO.loadIndex(merged)), + SegmentId.dummy("test") + ); + Assert.assertEquals( + ImmutableSet.of("A", "B", "C"), + Arrays.stream(segment.as(PhysicalSegmentInspector.class).getMetadata().getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toSet()) + ); } @Test(expected = IAE.class) @@ -1888,10 +1902,14 @@ public class IndexMergerTestBase extends InitializedNullHandlingTest indexSpec, -1 ); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter( - closer.closeLater(indexIO.loadIndex(merged)) + final QueryableIndexSegment segment = new QueryableIndexSegment( + closer.closeLater(indexIO.loadIndex(merged)), + SegmentId.dummy("test") + ); + Assert.assertEquals( + ImmutableSet.of("A", "B", "C"), + Arrays.stream(segment.as(PhysicalSegmentInspector.class).getMetadata().getAggregators()).map(AggregatorFactory::getName).collect(Collectors.toSet()) ); - Assert.assertEquals(ImmutableSet.of("A", "B", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/ListCursor.java b/processing/src/test/java/org/apache/druid/segment/ListCursor.java index 2ed9e843150..5d0454314e8 100644 --- a/processing/src/test/java/org/apache/druid/segment/ListCursor.java +++ b/processing/src/test/java/org/apache/druid/segment/ListCursor.java @@ -36,7 +36,7 @@ import java.util.List; * This is used to test the base cursor of an UnnestCursor. * Usages can be found in tests of {@link UnnestColumnValueSelectorCursor} in {@link UnnestColumnValueSelectorCursorTest} * However this cannot help with {@link UnnestDimensionCursor}. - * Tests for {@link UnnestDimensionCursor} are done alongside tests for {@link UnnestStorageAdapterTest} + * Tests for {@link UnnestDimensionCursor} are done alongside tests for {@link UnnestCursorFactoryTest} */ public class ListCursor implements Cursor { diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java index f13b6c5632a..9fc9fc0f578 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java @@ -26,14 +26,12 @@ import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.BuiltInTypesModule; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.nested.StructuredData; @@ -146,9 +144,9 @@ public class NestedDataColumnIndexerV4Test extends InitializedNullHandlingTest index.add(makeInputRow(minTimestamp + 4, true, STRING_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, STRING_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(STRING_COL, STRING_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -198,8 +196,8 @@ public class NestedDataColumnIndexerV4Test extends InitializedNullHandlingTest index.add(makeInputRow(minTimestamp + 4, true, LONG_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, LONG_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(LONG_COL, LONG_COL, ColumnType.LONG); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -279,8 +277,8 @@ public class NestedDataColumnIndexerV4Test extends InitializedNullHandlingTest index.add(makeInputRow(minTimestamp + 4, true, DOUBLE_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, DOUBLE_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(DOUBLE_COL, DOUBLE_COL, ColumnType.DOUBLE); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -360,8 +358,8 @@ public class NestedDataColumnIndexerV4Test extends InitializedNullHandlingTest index.add(makeInputRow(minTimestamp + 4, true, STRING_ARRAY_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, STRING_ARRAY_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec( STRING_ARRAY_COL, @@ -403,8 +401,8 @@ public class NestedDataColumnIndexerV4Test extends InitializedNullHandlingTest index.add(makeInputRow(minTimestamp + 4, true, VARIANT_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, VARIANT_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(VARIANT_COL, VARIANT_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -446,8 +444,8 @@ public class NestedDataColumnIndexerV4Test extends InitializedNullHandlingTest index.add(makeInputRow(minTimestamp + 4, true, NESTED_COL, null)); index.add(makeInputRow(minTimestamp + 5, false, NESTED_COL, null)); - IncrementalIndexStorageAdapter storageAdapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); final DimensionSpec dimensionSpec = new DefaultDimensionSpec(NESTED_COL, NESTED_COL, ColumnType.STRING); ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -478,15 +476,16 @@ public class NestedDataColumnIndexerV4Test extends InitializedNullHandlingTest { IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( - new IncrementalIndexSchema( - minTimestamp, - new TimestampSpec(TIME_COL, "millis", null), - Granularities.NONE, - VirtualColumns.EMPTY, - DimensionsSpec.builder().useSchemaDiscovery(true).build(), - new AggregatorFactory[0], - false - ) + IncrementalIndexSchema.builder() + .withMinTimestamp(minTimestamp) + .withTimestampSpec(new TimestampSpec(TIME_COL, "millis", null)) + .withDimensionsSpec( + DimensionsSpec.builder() + .useSchemaDiscovery(true) + .build() + ) + .withRollup(false) + .build() ) .setMaxRowCount(1000) .build(); diff --git a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorFactoryTest.java similarity index 94% rename from processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorFactoryTest.java index 4cde2a79658..05de6845e53 100644 --- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexCursorFactoryTest.java @@ -40,7 +40,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collection; -public class QueryableIndexStorageAdapterTest +public class QueryableIndexCursorFactoryTest { @RunWith(Parameterized.class) public static class DimensionDictionarySelectorTest extends InitializedNullHandlingTest @@ -68,9 +68,9 @@ public class QueryableIndexStorageAdapterTest public void setUp() { final QueryableIndex index = TestIndex.getMMappedTestIndex(); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); if (vectorize) { - final VectorCursor cursor = closer.register(adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)).asVectorCursor(); + final VectorCursor cursor = closer.register(cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)).asVectorCursor(); final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -81,7 +81,7 @@ public class QueryableIndexStorageAdapterTest partialNullSelector = columnSelectorFactory.makeSingleValueDimensionSelector(DefaultDimensionSpec.of("partial_null_column")); } else { - final CursorHolder cursorHolder = closer.register(adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)); + final CursorHolder cursorHolder = closer.register(cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)); final Cursor cursor = cursorHolder.asCursor(); final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); @@ -225,8 +225,8 @@ public class QueryableIndexStorageAdapterTest public void setUp() { final QueryableIndex index = TestIndex.getMMappedTestIndex(); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); - final CursorHolder cursorHolder = closer.register(adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)); + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); + final CursorHolder cursorHolder = closer.register(cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)); cursor = cursorHolder.asCursor(); columnSelectorFactory = cursor.getColumnSelectorFactory(); } diff --git a/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java b/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java index b1f65cfc8d1..774af2c30c2 100644 --- a/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java @@ -43,7 +43,7 @@ public class ReferenceCountingSegmentTest private final SegmentId segmentId = SegmentId.dummy("test_segment"); private final Interval dataInterval = new Interval(DateTimes.nowUtc().minus(Days.days(1)), DateTimes.nowUtc()); private QueryableIndex index; - private StorageAdapter adapter; + private CursorFactory cursorFactory; private IndexedTable indexedTable; private int underlyingSegmentClosedCount; @@ -52,7 +52,7 @@ public class ReferenceCountingSegmentTest { underlyingSegmentClosedCount = 0; index = EasyMock.createNiceMock(QueryableIndex.class); - adapter = EasyMock.createNiceMock(StorageAdapter.class); + cursorFactory = EasyMock.createNiceMock(CursorFactory.class); indexedTable = EasyMock.createNiceMock(IndexedTable.class); segment = ReferenceCountingSegment.wrapRootGenerationSegment( @@ -77,9 +77,9 @@ public class ReferenceCountingSegmentTest } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return adapter; + return cursorFactory; } @Override @@ -87,8 +87,8 @@ public class ReferenceCountingSegmentTest { if (clazz.equals(QueryableIndex.class)) { return (T) asQueryableIndex(); - } else if (clazz.equals(StorageAdapter.class)) { - return (T) asStorageAdapter(); + } else if (clazz.equals(CursorFactory.class)) { + return (T) asCursorFactory(); } else if (clazz.equals(IndexedTable.class)) { return (T) indexedTable; } @@ -172,14 +172,14 @@ public class ReferenceCountingSegmentTest Assert.assertEquals(segmentId, segment.getId()); Assert.assertEquals(dataInterval, segment.getDataInterval()); Assert.assertEquals(index, segment.asQueryableIndex()); - Assert.assertEquals(adapter, segment.asStorageAdapter()); + Assert.assertEquals(cursorFactory, segment.asCursorFactory()); } @Test public void testSegmentAs() { Assert.assertSame(index, segment.as(QueryableIndex.class)); - Assert.assertSame(adapter, segment.as(StorageAdapter.class)); + Assert.assertSame(cursorFactory, segment.as(CursorFactory.class)); Assert.assertSame(indexedTable, segment.as(IndexedTable.class)); Assert.assertNull(segment.as(String.class)); } diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedCursorFactoryTest.java similarity index 82% rename from processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/segment/RowBasedCursorFactoryTest.java index 6c46050c697..aab86309eaa 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedCursorFactoryTest.java @@ -51,7 +51,6 @@ import org.junit.Test; import java.math.RoundingMode; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.concurrent.atomic.AtomicLong; @@ -61,9 +60,7 @@ import java.util.function.Supplier; import java.util.function.ToLongFunction; import java.util.stream.Collectors; -import static org.junit.Assert.assertThrows; - -public class RowBasedStorageAdapterTest +public class RowBasedCursorFactoryTest { private static final String UNKNOWN_TYPE_NAME = "unknownType"; @@ -246,14 +243,14 @@ public class RowBasedStorageAdapterTest public final AtomicLong numCloses = new AtomicLong(); - private RowBasedStorageAdapter createIntAdapter(final int... ints) + private RowBasedCursorFactory createIntAdapter(final int... ints) { return createIntAdapter(ROW_ADAPTER, ints); } - private RowBasedStorageAdapter createIntAdapter(RowAdapter adapter, final int... ints) + private RowBasedCursorFactory createIntAdapter(RowAdapter adapter, final int... ints) { - return new RowBasedStorageAdapter<>( + return new RowBasedCursorFactory<>( Sequences.simple(Arrays.stream(ints).boxed().collect(Collectors.toList())) .withBaggage(numCloses::incrementAndGet), adapter, @@ -261,113 +258,17 @@ public class RowBasedStorageAdapterTest ); } - @Test - public void test_getInterval() - { - final RowBasedStorageAdapter adapter = createIntAdapter(); - Assert.assertEquals(Intervals.ETERNITY, adapter.getInterval()); - } - - @Test - public void test_getAvailableDimensions() - { - final RowBasedStorageAdapter adapter = createIntAdapter(); - - // Sort them for comparison purposes. - Assert.assertEquals( - ROW_SIGNATURE.getColumnNames().stream().sorted().collect(Collectors.toList()), - Lists.newArrayList(adapter.getAvailableDimensions()).stream().sorted().collect(Collectors.toList()) - ); - } - - @Test - public void test_getAvailableMetrics() - { - final RowBasedStorageAdapter adapter = createIntAdapter(); - - Assert.assertEquals( - Collections.emptyList(), - Lists.newArrayList(adapter.getAvailableMetrics()) - ); - } - @Test public void test_getRowSignature() { - final RowBasedStorageAdapter adapter = createIntAdapter(); + final RowBasedCursorFactory adapter = createIntAdapter(); Assert.assertEquals(ROW_SIGNATURE, adapter.getRowSignature()); } - @Test - public void test_getDimensionCardinality_knownColumns() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - - // Row based adapters don't know cardinality (they don't walk their Iterables until makeCursor is called). - for (String column : ROW_SIGNATURE.getColumnNames()) { - Assert.assertEquals(DimensionDictionarySelector.CARDINALITY_UNKNOWN, adapter.getDimensionCardinality(column)); - } - } - - @Test - public void test_getDimensionCardinality_unknownColumn() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - Assert.assertEquals(DimensionDictionarySelector.CARDINALITY_UNKNOWN, adapter.getDimensionCardinality("unknown")); - } - - @Test - public void test_getDimensionCardinality_timeColumn() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - Assert.assertEquals(DimensionDictionarySelector.CARDINALITY_UNKNOWN, adapter.getDimensionCardinality("__time")); - } - - @Test - public void test_getMinValue() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - - // Row based adapters don't know min/max values, so they always return null. - // Test both known and unknown columns. - final List columns = - ImmutableList.builder().addAll(ROW_SIGNATURE.getColumnNames()).add("unknown", "__time").build(); - - for (String column : columns) { - Assert.assertNull(column, adapter.getMinValue(column)); - } - } - - @Test - public void test_getMaxValue() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - - // Row based adapters don't know min/max values, so they always return null. - // Test both known and unknown columns. - final List columns = - ImmutableList.builder().addAll(ROW_SIGNATURE.getColumnNames()).add("unknown", "__time").build(); - - for (String column : columns) { - Assert.assertNull(column, adapter.getMaxValue(column)); - } - } - - @Test - public void test_getCapabilities() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - - // Row based adapters don't know cardinality (they don't walk their Iterables until makeCursor is called). - for (String column : ROW_SIGNATURE.getColumnNames()) { - Assert.assertEquals(DimensionDictionarySelector.CARDINALITY_UNKNOWN, adapter.getDimensionCardinality(column)); - } - } - @Test public void test_getColumnCapabilities_float() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.FLOAT.name()); Assert.assertEquals(ValueType.FLOAT, capabilities.getType()); @@ -377,7 +278,7 @@ public class RowBasedStorageAdapterTest @Test public void test_getColumnCapabilities_double() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.DOUBLE.name()); Assert.assertEquals(ValueType.DOUBLE, capabilities.getType()); @@ -387,7 +288,7 @@ public class RowBasedStorageAdapterTest @Test public void test_getColumnCapabilities_long() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.LONG.name()); Assert.assertEquals(ValueType.LONG, capabilities.getType()); @@ -397,7 +298,7 @@ public class RowBasedStorageAdapterTest @Test public void test_getColumnCapabilities_string() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.STRING.name()); Assert.assertEquals(ValueType.STRING, capabilities.getType()); @@ -411,7 +312,7 @@ public class RowBasedStorageAdapterTest @Test public void test_getColumnCapabilities_complex() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.COMPLEX.name()); @@ -425,7 +326,7 @@ public class RowBasedStorageAdapterTest @Test public void test_getColumnCapabilities_unknownType() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final ColumnCapabilities capabilities = adapter.getColumnCapabilities(UNKNOWN_TYPE_NAME); Assert.assertNull(capabilities); @@ -434,14 +335,14 @@ public class RowBasedStorageAdapterTest @Test public void test_getColumnCapabilities_nonexistent() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); Assert.assertNull(adapter.getColumnCapabilities("nonexistent")); } @Test public void test_getColumnTypeString() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); for (String columnName : ROW_SIGNATURE.getColumnNames()) { if (UNKNOWN_TYPE_NAME.equals(columnName)) { @@ -456,24 +357,10 @@ public class RowBasedStorageAdapterTest } } - @Test - public void test_getNumRows() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - assertThrows(UnsupportedOperationException.class, () -> adapter.getMetadata()); - } - - @Test - public void test_getMetadata() - { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); - assertThrows(UnsupportedOperationException.class, () -> adapter.getMetadata()); - } - @Test public void test_makeCursor() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .build(); @@ -495,7 +382,7 @@ public class RowBasedStorageAdapterTest @Test public void test_makeCursor_filterOnLong() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(new SelectorDimFilter(ValueType.LONG.name(), "1.0", null).toFilter()) @@ -518,7 +405,7 @@ public class RowBasedStorageAdapterTest @Test public void test_makeCursor_filterOnNonexistentColumnEqualsNull() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(new SelectorDimFilter("nonexistent", null, null).toFilter()) @@ -541,7 +428,7 @@ public class RowBasedStorageAdapterTest @Test public void test_makeCursor_filterOnVirtualColumn() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(new SelectorDimFilter("vc", "2", null).toFilter()) @@ -576,7 +463,7 @@ public class RowBasedStorageAdapterTest @Test public void test_makeCursor_descending() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setPreferredOrdering(Cursors.descendingTimeOrder()) @@ -599,7 +486,7 @@ public class RowBasedStorageAdapterTest @Test public void test_makeCursor_intervalDoesNotMatch() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("2000/P1D")) @@ -618,7 +505,7 @@ public class RowBasedStorageAdapterTest @Test public void test_makeCursor_intervalPartiallyMatches() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 2); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 2); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("1970-01-01T01/PT1H")) @@ -639,7 +526,7 @@ public class RowBasedStorageAdapterTest @Test public void test_makeCursor_hourGranularity() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 1, 2, 3); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("1970/1971")) @@ -663,7 +550,7 @@ public class RowBasedStorageAdapterTest @Test public void test_makeCursor_hourGranularityWithInterval() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 1, 2, 3); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("1970-01-01T01/PT2H")) @@ -686,7 +573,7 @@ public class RowBasedStorageAdapterTest @Test public void test_makeCursor_hourGranularityWithIntervalDescending() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1, 1, 2, 3); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1, 1, 2, 3); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.of("1970-01-01T01/PT2H")) @@ -710,7 +597,7 @@ public class RowBasedStorageAdapterTest @Test public void test_makeCursor_allProcessors() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1); try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { final Cursor cursor = cursorHolder.asCursor(); @@ -815,7 +702,7 @@ public class RowBasedStorageAdapterTest @Test public void test_makeCursor_filterOnNonexistentColumnEqualsNonnull() { - final RowBasedStorageAdapter adapter = createIntAdapter(0, 1); + final RowBasedCursorFactory adapter = createIntAdapter(0, 1); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(new SelectorDimFilter("nonexistent", "abc", null).toFilter()) diff --git a/processing/src/test/java/org/apache/druid/segment/TestSegmentForAs.java b/processing/src/test/java/org/apache/druid/segment/TestSegmentForAs.java index 2a910d0d42a..59bde904e3c 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestSegmentForAs.java +++ b/processing/src/test/java/org/apache/druid/segment/TestSegmentForAs.java @@ -61,9 +61,9 @@ public class TestSegmentForAs implements Segment } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return as(StorageAdapter.class); + return as(CursorFactory.class); } @SuppressWarnings("unchecked") diff --git a/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java deleted file mode 100644 index 93266894c69..00000000000 --- a/processing/src/test/java/org/apache/druid/segment/TombstoneSegmentStorageAdapterTest.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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.segment; - -import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.data.Indexed; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Test; - -import javax.annotation.Nullable; - - -public class TombstoneSegmentStorageAdapterTest -{ - @Test - public void testTombstoneDefaultInterface() - { - StorageAdapter sa = new StorageAdapter() - { - @Override - public CursorHolder makeCursorHolder(CursorBuildSpec spec) - { - return new CursorHolder() - { - @Nullable - @Override - public Cursor asCursor() - { - return null; - } - }; - } - - @Override - public Interval getInterval() - { - return null; - } - - @Override - public int getNumRows() - { - return 0; - } - - @Override - public Indexed getAvailableDimensions() - { - return null; - } - - @Override - public Iterable getAvailableMetrics() - { - return null; - } - - @Override - public int getDimensionCardinality(String column) - { - return 0; - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - return null; - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - return null; - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - return null; - } - - @Nullable - @Override - public Metadata getMetadata() - { - return null; - } - }; - - Assert.assertFalse(sa.isFromTombstone()); - } - -} diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestCursorFactoryTest.java similarity index 82% rename from processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/segment/UnnestCursorFactoryTest.java index 96a06e3523b..32770f91f2e 100644 --- a/processing/src/test/java/org/apache/druid/segment/UnnestStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/UnnestCursorFactoryTest.java @@ -29,6 +29,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.NestedDataTestUtils; import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.SelectorDimFilter; @@ -44,8 +45,8 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.join.PostJoinCursor; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -74,18 +75,18 @@ import static org.apache.druid.segment.filter.FilterTestUtils.selector; import static org.apache.druid.segment.filter.Filters.and; import static org.apache.druid.segment.filter.Filters.or; -public class UnnestStorageAdapterTest extends InitializedNullHandlingTest +public class UnnestCursorFactoryTest extends InitializedNullHandlingTest { @ClassRule public static TemporaryFolder tmp = new TemporaryFolder(); private static Closer CLOSER; private static IncrementalIndex INCREMENTAL_INDEX; - private static IncrementalIndexStorageAdapter INCREMENTAL_INDEX_STORAGE_ADAPTER; + private static IncrementalIndexCursorFactory INCREMENTAL_INDEX_CURSOR_FACTORY; private static QueryableIndex QUERYABLE_INDEX; - private static UnnestStorageAdapter UNNEST_STORAGE_ADAPTER; - private static UnnestStorageAdapter UNNEST_STORAGE_ADAPTER1; - private static UnnestStorageAdapter UNNEST_ARRAYS; - private static List ADAPTERS; + private static UnnestCursorFactory UNNEST_CURSOR_FACTORY; + private static UnnestCursorFactory UNNEST_CURSOR_FACTORY1; + private static UnnestCursorFactory UNNEST_ARRAYS; + private static List CURSOR_FACTORIES; private static String INPUT_COLUMN_NAME = "multi-string1"; private static String OUTPUT_COLUMN_NAME = "unnested-multi-string1"; private static String OUTPUT_COLUMN_NAME1 = "unnested-multi-string1-again"; @@ -110,21 +111,21 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest INCREMENTAL_INDEX = CLOSER.register( segmentGenerator.generateIncrementalIndex(dataSegment, schemaInfo, Granularities.HOUR, numRows) ); - INCREMENTAL_INDEX_STORAGE_ADAPTER = new IncrementalIndexStorageAdapter(INCREMENTAL_INDEX); - UNNEST_STORAGE_ADAPTER = new UnnestStorageAdapter( - INCREMENTAL_INDEX_STORAGE_ADAPTER, + INCREMENTAL_INDEX_CURSOR_FACTORY = new IncrementalIndexCursorFactory(INCREMENTAL_INDEX); + UNNEST_CURSOR_FACTORY = new UnnestCursorFactory( + INCREMENTAL_INDEX_CURSOR_FACTORY, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + INPUT_COLUMN_NAME + "\"", null, ExprMacroTable.nil()), null ); - UNNEST_STORAGE_ADAPTER1 = new UnnestStorageAdapter( - UNNEST_STORAGE_ADAPTER, + UNNEST_CURSOR_FACTORY1 = new UnnestCursorFactory( + UNNEST_CURSOR_FACTORY, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME1, "\"" + INPUT_COLUMN_NAME + "\"", null, ExprMacroTable.nil()), null ); final InputSource inputSource = ResourceInputSource.of( - UnnestStorageAdapterTest.class.getClassLoader(), + UnnestCursorFactoryTest.class.getClassLoader(), NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE ); IndexBuilder bob = IndexBuilder.create() @@ -144,15 +145,15 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest .transform(TransformSpec.NONE) .inputTmpDir(tmp.newFolder()); QUERYABLE_INDEX = CLOSER.register(bob.buildMMappedIndex()); - UNNEST_ARRAYS = new UnnestStorageAdapter( - new QueryableIndexStorageAdapter(QUERYABLE_INDEX), + UNNEST_ARRAYS = new UnnestCursorFactory( + new QueryableIndexCursorFactory(QUERYABLE_INDEX), new ExpressionVirtualColumn("u", "\"arrayLongNulls\"", ColumnType.LONG, ExprMacroTable.nil()), null ); - ADAPTERS = ImmutableList.of( - UNNEST_STORAGE_ADAPTER, - UNNEST_STORAGE_ADAPTER1 + CURSOR_FACTORIES = ImmutableList.of( + UNNEST_CURSOR_FACTORY, + UNNEST_CURSOR_FACTORY1 ); } @@ -164,24 +165,22 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest } @Test - public void test_group_of_unnest_adapters_methods() + public void test_capabilities() { String colName = "multi-string1"; - for (StorageAdapter adapter : ADAPTERS) { - adapter.getColumnCapabilities(colName); - Assert.assertEquals(adapter.getNumRows(), 0); - Assert.assertNotNull(adapter.getMetadata()); + for (UnnestCursorFactory cursorFactory : CURSOR_FACTORIES) { + cursorFactory.getColumnCapabilities(colName); Assert.assertEquals( - adapter.getColumnCapabilities(colName).toColumnType(), - INCREMENTAL_INDEX_STORAGE_ADAPTER.getColumnCapabilities(colName).toColumnType() + cursorFactory.getColumnCapabilities(colName).toColumnType(), + INCREMENTAL_INDEX_CURSOR_FACTORY.getColumnCapabilities(colName).toColumnType() ); - assertColumnReadsIdentifier(((UnnestStorageAdapter) adapter).getUnnestColumn(), colName); + assertColumnReadsIdentifier(cursorFactory.getUnnestColumn(), colName); } } @Test - public void test_unnest_adapter_column_capabilities() + public void test_unnest_factory_column_capabilities() { String colName = "multi-string1"; List columnsInTable = Arrays.asList( @@ -200,27 +199,27 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest ValueType.STRING, ValueType.STRING ); - UnnestStorageAdapter adapter = UNNEST_STORAGE_ADAPTER; + UnnestCursorFactory cursorFactory = UNNEST_CURSOR_FACTORY; for (int i = 0; i < columnsInTable.size(); i++) { - ColumnCapabilities capabilities = adapter.getColumnCapabilities(columnsInTable.get(i)); + ColumnCapabilities capabilities = cursorFactory.getColumnCapabilities(columnsInTable.get(i)); Assert.assertEquals(capabilities.getType(), valueTypes.get(i)); } - assertColumnReadsIdentifier(adapter.getUnnestColumn(), colName); + assertColumnReadsIdentifier(cursorFactory.getUnnestColumn(), colName); Assert.assertEquals( - adapter.getColumnCapabilities(OUTPUT_COLUMN_NAME).isDictionaryEncoded(), + cursorFactory.getColumnCapabilities(OUTPUT_COLUMN_NAME).isDictionaryEncoded(), ColumnCapabilities.Capable.TRUE // passed through from dict-encoded input ); Assert.assertEquals( - adapter.getColumnCapabilities(OUTPUT_COLUMN_NAME).hasMultipleValues(), + cursorFactory.getColumnCapabilities(OUTPUT_COLUMN_NAME).hasMultipleValues(), ColumnCapabilities.Capable.FALSE ); } @Test - public void test_unnest_adapters_basic() + public void test_unnest_factory_basic() { - try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = UNNEST_CURSOR_FACTORY.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -254,7 +253,7 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest } @Test - public void test_unnest_adapters_basic_array_column() + public void test_unnest_factory_basic_array_column() { try (final CursorHolder cursorHolder = UNNEST_ARRAYS.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); @@ -286,10 +285,10 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest } @Test - public void test_unnest_adapters_basic_row_based_array_column() + public void test_unnest_factory_basic_row_based_array_column() { - StorageAdapter adapter = new UnnestStorageAdapter( - new RowBasedStorageAdapter<>( + UnnestCursorFactory cursorFactory = new UnnestCursorFactory( + new RowBasedCursorFactory<>( Sequences.simple( Arrays.asList( new Object[]{1L, new Object[]{1L, 2L}}, @@ -321,7 +320,7 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest new ExpressionVirtualColumn("u", "\"a\"", ColumnType.LONG, ExprMacroTable.nil()), null ); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -351,9 +350,9 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest } @Test - public void test_two_levels_of_unnest_adapters() + public void test_two_levels_of_unnest() { - try (final CursorHolder cursorHolder = UNNEST_STORAGE_ADAPTER1.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = UNNEST_CURSOR_FACTORY1.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -384,17 +383,18 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest } @Test - public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest_adapters() + public void test_pushdown_or_filters_unnested_and_original_dimension_with_unnest() { - final UnnestStorageAdapter unnestStorageAdapter = new UnnestStorageAdapter( - new TestStorageAdapter(INCREMENTAL_INDEX), + final TestCursorFactory testCursorFactory = new TestCursorFactory(INCREMENTAL_INDEX); + final UnnestCursorFactory cursorFactory = new UnnestCursorFactory( + testCursorFactory, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + INPUT_COLUMN_NAME + "\"", null, ExprMacroTable.nil()), null ); - final VirtualColumn vc = unnestStorageAdapter.getUnnestColumn(); + final VirtualColumn vc = cursorFactory.getUnnestColumn(); - final String inputColumn = unnestStorageAdapter.getUnnestInputIfDirectAccess(vc); + final String inputColumn = cursorFactory.getUnnestInputIfDirectAccess(vc); final OrFilter baseFilter = new OrFilter(ImmutableList.of( selector(OUTPUT_COLUMN_NAME, "1"), @@ -409,10 +409,9 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(baseFilter) .build(); - try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); + final Filter pushDownFilter = testCursorFactory.getPushDownFilter(); Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); @@ -423,17 +422,18 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest } @Test - public void test_nested_filters_unnested_and_original_dimension_with_unnest_adapters() + public void test_nested_filters_unnested_and_original_dimension_with_unnest() { - final UnnestStorageAdapter unnestStorageAdapter = new UnnestStorageAdapter( - new TestStorageAdapter(INCREMENTAL_INDEX), + final TestCursorFactory testCursorFactory = new TestCursorFactory(INCREMENTAL_INDEX); + final UnnestCursorFactory cursorFactory = new UnnestCursorFactory( + testCursorFactory, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + INPUT_COLUMN_NAME + "\"", null, ExprMacroTable.nil()), null ); - final VirtualColumn vc = unnestStorageAdapter.getUnnestColumn(); + final VirtualColumn vc = cursorFactory.getUnnestColumn(); - final String inputColumn = unnestStorageAdapter.getUnnestInputIfDirectAccess(vc); + final String inputColumn = cursorFactory.getUnnestInputIfDirectAccess(vc); final OrFilter baseFilter = new OrFilter(ImmutableList.of( selector(OUTPUT_COLUMN_NAME, "1"), @@ -454,10 +454,9 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(baseFilter) .build(); - try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); + final Filter pushDownFilter = testCursorFactory.getPushDownFilter(); Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); @@ -714,30 +713,31 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest } @Test - public void test_pushdown_filters_unnested_dimension_with_unnest_adapters() + public void test_pushdown_filters_unnested_dimension_with_unnest() { - final UnnestStorageAdapter unnestStorageAdapter = new UnnestStorageAdapter( - new TestStorageAdapter(INCREMENTAL_INDEX), + final TestCursorFactory testCursorFactory = new TestCursorFactory(INCREMENTAL_INDEX); + final DimFilter filter = new SelectorDimFilter(OUTPUT_COLUMN_NAME, "1", null); + final UnnestCursorFactory unnestCursorFactory = new UnnestCursorFactory( + testCursorFactory, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + INPUT_COLUMN_NAME + "\"", null, ExprMacroTable.nil()), - new SelectorDimFilter(OUTPUT_COLUMN_NAME, "1", null) + filter ); - final VirtualColumn vc = unnestStorageAdapter.getUnnestColumn(); + final VirtualColumn vc = unnestCursorFactory.getUnnestColumn(); - final String inputColumn = unnestStorageAdapter.getUnnestInputIfDirectAccess(vc); + final String inputColumn = unnestCursorFactory.getUnnestInputIfDirectAccess(vc); final Filter expectedPushDownFilter = selector(inputColumn, "1"); - try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = unnestCursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); + final Filter pushDownFilter = testCursorFactory.getPushDownFilter(); Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); final Filter postFilter = ((PostJoinCursor) cursor).getPostJoinFilter(); - Assert.assertEquals(unnestStorageAdapter.getUnnestFilter(), postFilter); + Assert.assertEquals(filter.toFilter(), postFilter); int count = 0; while (!cursor.isDone()) { @@ -752,15 +752,16 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest @Test public void test_pushdown_filters_unnested_dimension_outside() { - final UnnestStorageAdapter unnestStorageAdapter = new UnnestStorageAdapter( - new TestStorageAdapter(INCREMENTAL_INDEX), + final TestCursorFactory testCursorFactory = new TestCursorFactory(INCREMENTAL_INDEX); + final UnnestCursorFactory unnestCursorFactory = new UnnestCursorFactory( + testCursorFactory, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + INPUT_COLUMN_NAME + "\"", null, ExprMacroTable.nil()), null ); - final VirtualColumn vc = unnestStorageAdapter.getUnnestColumn(); + final VirtualColumn vc = unnestCursorFactory.getUnnestColumn(); - final String inputColumn = unnestStorageAdapter.getUnnestInputIfDirectAccess(vc); + final String inputColumn = unnestCursorFactory.getUnnestInputIfDirectAccess(vc); final Filter expectedPushDownFilter = selector(inputColumn, "1"); @@ -770,10 +771,9 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest .setFilter(queryFilter) .build(); - try (final CursorHolder cursorHolder = unnestStorageAdapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = unnestCursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); - final TestStorageAdapter base = (TestStorageAdapter) unnestStorageAdapter.getBaseAdapter(); - final Filter pushDownFilter = base.getPushDownFilter(); + final Filter pushDownFilter = testCursorFactory.getPushDownFilter(); Assert.assertEquals(expectedPushDownFilter, pushDownFilter); Assert.assertEquals(cursor.getClass(), PostJoinCursor.class); @@ -807,14 +807,14 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest IncrementalIndex index = CLOSER.register( segmentGenerator.generateIncrementalIndex(dataSegment, schemaInfo, Granularities.HOUR, 100) ); - IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - UnnestStorageAdapter withNullsStorageAdapter = new UnnestStorageAdapter( - adapter, + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + UnnestCursorFactory unnestCursorFactory = new UnnestCursorFactory( + cursorFactory, new ExpressionVirtualColumn(OUTPUT_COLUMN_NAME, "\"" + inputColumn + "\"", null, ExprMacroTable.nil()), null ); - try (final CursorHolder cursorHolder = withNullsStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = unnestCursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -843,7 +843,7 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest ) { testComputeBaseAndPostUnnestFilters( - UNNEST_STORAGE_ADAPTER, + UNNEST_CURSOR_FACTORY, testQueryFilter, expectedBasePushDown, expectedPostUnnest @@ -851,20 +851,20 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest } public void testComputeBaseAndPostUnnestFilters( - UnnestStorageAdapter adapter, + UnnestCursorFactory cursorFactory, Filter testQueryFilter, String expectedBasePushDown, String expectedPostUnnest ) { - final String inputColumn = adapter.getUnnestInputIfDirectAccess(adapter.getUnnestColumn()); - final VirtualColumn vc = adapter.getUnnestColumn(); - Pair filterPair = adapter.computeBaseAndPostUnnestFilters( + final String inputColumn = cursorFactory.getUnnestInputIfDirectAccess(cursorFactory.getUnnestColumn()); + final VirtualColumn vc = cursorFactory.getUnnestColumn(); + Pair filterPair = cursorFactory.computeBaseAndPostUnnestFilters( testQueryFilter, null, VirtualColumns.EMPTY, inputColumn, - vc.capabilities(adapter, inputColumn) + vc.capabilities(cursorFactory, inputColumn) ); Filter actualPushDownFilter = filterPair.lhs; Filter actualPostUnnestFilter = filterPair.rhs; @@ -889,16 +889,16 @@ public class UnnestStorageAdapterTest extends InitializedNullHandlingTest /** * Class to test the flow of pushing down filters into the base cursor - * while using the UnnestStorageAdapter. This class keeps a reference of the filter + * while using the {@link UnnestCursorFactory}. This class keeps a reference of the filter * which is pushed down to the cursor which serves as a checkpoint to validate * if the right filter is being pushed down */ -class TestStorageAdapter extends IncrementalIndexStorageAdapter +class TestCursorFactory extends IncrementalIndexCursorFactory { private Filter pushDownFilter; - public TestStorageAdapter(IncrementalIndex index) + public TestCursorFactory(IncrementalIndex index) { super(index); } diff --git a/processing/src/test/java/org/apache/druid/segment/VirtualColumnsTest.java b/processing/src/test/java/org/apache/druid/segment/VirtualColumnsTest.java index 71d9adbd017..9f0e8c5ea31 100644 --- a/processing/src/test/java/org/apache/druid/segment/VirtualColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/VirtualColumnsTest.java @@ -43,7 +43,6 @@ import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.MockitoJUnit; @@ -62,9 +61,6 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest { private static final String REAL_COLUMN_NAME = "real_column"; - @Rule - public ExpectedException expectedException = ExpectedException.none(); - @Rule public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); @@ -217,10 +213,11 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest { final VirtualColumns virtualColumns = makeVirtualColumns(); - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("No such virtual column[bar]"); - - virtualColumns.makeColumnValueSelector("bar", baseColumnSelectorFactory); + Throwable t = Assert.assertThrows( + IllegalArgumentException.class, + () -> virtualColumns.makeColumnValueSelector("bar", baseColumnSelectorFactory) + ); + Assert.assertEquals("No such virtual column[bar]", t.getMessage()); } @Test @@ -321,10 +318,11 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest TestExprMacroTable.INSTANCE ); - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("virtualColumn name[__time] not allowed"); - - VirtualColumns.create(ImmutableList.of(expr)); + Throwable t = Assert.assertThrows( + IllegalArgumentException.class, + () -> VirtualColumns.create(ImmutableList.of(expr)) + ); + Assert.assertEquals("virtualColumn name[__time] not allowed", t.getMessage()); } @Test @@ -344,10 +342,11 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest TestExprMacroTable.INSTANCE ); - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Duplicate virtualColumn name[expr]"); - - VirtualColumns.create(ImmutableList.of(expr, expr2)); + Throwable t = Assert.assertThrows( + IllegalArgumentException.class, + () -> VirtualColumns.create(ImmutableList.of(expr, expr2)) + ); + Assert.assertEquals("Duplicate virtualColumn name[expr]", t.getMessage()); } @Test @@ -367,10 +366,11 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest TestExprMacroTable.INSTANCE ); - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("Self-referential column[expr]"); - - VirtualColumns.create(ImmutableList.of(expr, expr2)); + Throwable t = Assert.assertThrows( + IllegalArgumentException.class, + () -> VirtualColumns.create(ImmutableList.of(expr, expr2)) + ); + Assert.assertEquals("Self-referential column[expr]", t.getMessage()); } @Test @@ -417,6 +417,42 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest Assert.assertNotEquals(VirtualColumns.EMPTY.hashCode(), virtualColumns.hashCode()); } + @Test + public void testEquivalence() + { + final VirtualColumn v0 = new ExpressionVirtualColumn( + "expr", + "x + y", + ColumnType.FLOAT, + TestExprMacroTable.INSTANCE + ); + final VirtualColumns virtualColumns = VirtualColumns.create(ImmutableList.of(v0)); + + final VirtualColumn v1 = new ExpressionVirtualColumn( + "differentNameExpr", + "x + y", + ColumnType.FLOAT, + TestExprMacroTable.INSTANCE + ); + final VirtualColumn v2 = new ExpressionVirtualColumn( + "differentNameTypeExpr", + "x + y", + ColumnType.DOUBLE, + TestExprMacroTable.INSTANCE + ); + final VirtualColumn v3 = new ExpressionVirtualColumn( + "expr", + "x + y", + ColumnType.DOUBLE, + TestExprMacroTable.INSTANCE + ); + + Assert.assertEquals(v0, virtualColumns.findEquivalent(v0)); + Assert.assertEquals(v0, virtualColumns.findEquivalent(v1)); + Assert.assertNull(virtualColumns.findEquivalent(v2)); + Assert.assertNull(virtualColumns.findEquivalent(v3)); + } + @Test public void testSerde() throws Exception { diff --git a/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java index 2481a6e8aad..35185967f66 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java @@ -34,8 +34,8 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Test; import org.junit.runner.RunWith; @@ -69,7 +69,7 @@ public class AndFilterTest extends BaseFilterTest public AndFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java index f45885790dc..6830040bab8 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ArrayContainsElementFilterTests.java @@ -34,8 +34,8 @@ import org.apache.druid.query.filter.ArrayContainsElementFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.AfterClass; @@ -56,7 +56,7 @@ public class ArrayContainsElementFilterTests public ArrayContainsElementFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index 1491fbd9d38..2e3ae0b633f 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -24,7 +24,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.guava.SettableSupplier; @@ -40,7 +39,8 @@ import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.segment.FrameSegment; -import org.apache.druid.frame.segment.FrameStorageAdapter; +import org.apache.druid.frame.segment.columnar.ColumnarFrameCursorFactory; +import org.apache.druid.frame.segment.row.RowFrameCursorFactory; import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -69,16 +69,16 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.RowAdapters; import org.apache.druid.segment.RowBasedColumnSelectorFactory; -import org.apache.druid.segment.RowBasedStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.RowBasedCursorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; @@ -91,8 +91,8 @@ import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.filter.cnf.CNFFilterExplosionException; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -404,7 +404,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest private final List rows; protected final IndexBuilder indexBuilder; - protected final Function> finisher; + protected final Function> finisher; protected final boolean cnf; protected final boolean optimize; protected final String testName; @@ -414,22 +414,22 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest // In other words, numeric null values will be treated as nulls instead of the default value protected final boolean canTestNumericNullsAsDefaultValues; - protected StorageAdapter adapter; + protected CursorFactory cursorFactory; protected VirtualColumns virtualColumns; // JUnit creates a new test instance for every test method call. // For filter tests, the test setup creates a segment. - // Creating a new segment for every test method call is pretty slow, so cache the StorageAdapters. + // Creating a new segment for every test method call is pretty slow, so cache the CursorFactory. // Each thread gets its own map. - private static ThreadLocal>> adapterCache = + private static ThreadLocal>> adapterCache = ThreadLocal.withInitial(HashMap::new); public BaseFilterTest( String testName, List rows, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) @@ -449,18 +449,18 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest { BuiltInTypesModule.registerHandlersAndSerde(); String className = getClass().getName(); - Map adaptersForClass = adapterCache.get().get(className); + Map adaptersForClass = adapterCache.get().get(className); if (adaptersForClass == null) { adaptersForClass = new HashMap<>(); adapterCache.get().put(className, adaptersForClass); } - AdapterStuff adapterStuff = adaptersForClass.get(testName); - if (adapterStuff == null) { - Pair pair = finisher.apply( + CursorStuff cursorStuff = adaptersForClass.get(testName); + if (cursorStuff == null) { + Pair pair = finisher.apply( indexBuilder.tmpDir(temporaryFolder.newFolder()).rows(rows) ); - adapterStuff = new AdapterStuff( + cursorStuff = new CursorStuff( pair.lhs, VirtualColumns.create( Arrays.stream(VIRTUAL_COLUMNS.getVirtualColumns()) @@ -469,19 +469,19 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest ), pair.rhs ); - adaptersForClass.put(testName, adapterStuff); + adaptersForClass.put(testName, cursorStuff); } - this.adapter = adapterStuff.adapter; - this.virtualColumns = adapterStuff.virtualColumns; + this.cursorFactory = cursorStuff.cursorFactory; + this.virtualColumns = cursorStuff.virtualColumns; } public static void tearDown(String className) throws Exception { - Map adaptersForClass = adapterCache.get().get(className); + Map adaptersForClass = adapterCache.get().get(className); if (adaptersForClass != null) { - for (Map.Entry entry : adaptersForClass.entrySet()) { + for (Map.Entry entry : adaptersForClass.entrySet()) { entry.getValue().closeable.close(); } adapterCache.get().put(className, null); @@ -508,13 +508,13 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest "off-heap memory segment write-out medium", OffHeapMemorySegmentWriteOutMediumFactory.instance() ); - final Map>> finishers = - ImmutableMap.>>builder() + final Map>> finishers = + ImmutableMap.>>builder() .put( "incremental", input -> { final IncrementalIndex index = input.buildIncrementalIndex(); - return Pair.of(new IncrementalIndexStorageAdapter(index), index); + return Pair.of(new IncrementalIndexCursorFactory(index), index); } ) .put( @@ -542,7 +542,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest ) ); final IncrementalIndex index = input.buildIncrementalIndex(); - return Pair.of(new IncrementalIndexStorageAdapter(index), index); + return Pair.of(new IncrementalIndexCursorFactory(index), index); } ) .put( @@ -570,7 +570,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest ) ); final QueryableIndex index = input.buildMMappedIndex(); - return Pair.of(new QueryableIndexStorageAdapter(index), index); + return Pair.of(new QueryableIndexCursorFactory(index), index); } ) .put( @@ -605,21 +605,21 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest .intermediaryPersistSize(3) .buildMMappedIndex(); - return Pair.of(new QueryableIndexStorageAdapter(index), index); + return Pair.of(new QueryableIndexCursorFactory(index), index); } ) .put( "mmapped", input -> { final QueryableIndex index = input.buildMMappedIndex(); - return Pair.of(new QueryableIndexStorageAdapter(index), index); + return Pair.of(new QueryableIndexCursorFactory(index), index); } ) .put( "mmappedMerged", input -> { final QueryableIndex index = input.buildMMappedMergedIndex(); - return Pair.of(new QueryableIndexStorageAdapter(index), index); + return Pair.of(new QueryableIndexCursorFactory(index), index); } ) .put( @@ -640,7 +640,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest Assert.assertTrue(NullHandling.replaceWithDefault()); try { final QueryableIndex index = input.getIndexIO().loadIndex(file); - return Pair.of(new QueryableIndexStorageAdapter(index), index); + return Pair.of(new QueryableIndexCursorFactory(index), index); } catch (IOException e) { throw new RuntimeException(e); @@ -649,11 +649,11 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest ) .put( "rowBasedWithoutTypeSignature", - input -> Pair.of(input.buildRowBasedSegmentWithoutTypeSignature().asStorageAdapter(), () -> {}) + input -> Pair.of(input.buildRowBasedSegmentWithoutTypeSignature().asCursorFactory(), () -> {}) ) .put( "rowBasedWithTypeSignature", - input -> Pair.of(input.buildRowBasedSegmentWithTypeSignature().asStorageAdapter(), () -> {}) + input -> Pair.of(input.buildRowBasedSegmentWithTypeSignature().asCursorFactory(), () -> {}) ) .put("frame (row-based)", input -> { // remove variant type columns from row frames since they aren't currently supported @@ -676,7 +676,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest ) ); final FrameSegment segment = input.buildFrameSegment(FrameType.ROW_BASED); - return Pair.of(segment.asStorageAdapter(), segment); + return Pair.of(segment.asCursorFactory(), segment); }) .put("frame (columnar)", input -> { // remove array type columns from columnar frames since they aren't currently supported @@ -699,7 +699,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest ) ); final FrameSegment segment = input.buildFrameSegment(FrameType.COLUMNAR); - return Pair.of(segment.asStorageAdapter(), segment); + return Pair.of(segment.asCursorFactory(), segment); }) .build(); @@ -711,7 +711,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest for (Map.Entry bitmapSerdeFactoryEntry : bitmapSerdeFactories.entrySet()) { for (Map.Entry segmentWriteOutMediumFactoryEntry : segmentWriteOutMediumFactories.entrySet()) { - for (Map.Entry>> finisherEntry : + for (Map.Entry>> finisherEntry : finishers.entrySet()) { for (boolean cnf : ImmutableList.of(false, true)) { for (boolean optimize : ImmutableList.of(false, true)) { @@ -815,7 +815,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest private VectorCursor makeVectorCursor(final Filter filter) { final CursorBuildSpec buildSpec = makeVectorCursorBuildSpec(filter); - return adapter.makeCursorHolder(buildSpec).asVectorCursor(); + return cursorFactory.makeCursorHolder(buildSpec).asVectorCursor(); } /** @@ -823,7 +823,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest */ private List selectColumnValuesMatchingFilter(final DimFilter filter, final String selectColumn) { - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(makeFilter(filter)))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(makeFilter(filter)))) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() @@ -843,7 +843,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest private long selectCountUsingFilteredAggregator(final DimFilter filter) { - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(null))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(null))) { final Cursor cursor = cursorHolder.asCursor(); Aggregator agg = new FilteredAggregatorFactory( new CountAggregatorFactory("count"), @@ -861,13 +861,13 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest private long selectCountUsingVectorizedFilteredAggregator(final DimFilter dimFilter) { Preconditions.checkState( - makeFilter(dimFilter).canVectorizeMatcher(adapter), + makeFilter(dimFilter).canVectorizeMatcher(cursorFactory), "Cannot vectorize filter: %s", dimFilter ); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(null))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeVectorCursorBuildSpec(null))) { final VectorCursor cursor = cursorHolder.asVectorCursor(); final FilteredAggregatorFactory aggregatorFactory = new FilteredAggregatorFactory( new CountAggregatorFactory("count"), @@ -934,7 +934,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest } }; - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeCursorBuildSpec(postFilteringFilter))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(postFilteringFilter))) { final Cursor cursor = cursorHolder.asCursor(); final DimensionSelector selector = cursor .getColumnSelectorFactory() @@ -994,7 +994,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest } }; - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(postFilteringFilter))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeVectorCursorBuildSpec(postFilteringFilter))) { final VectorCursor cursor = cursorHolder.asVectorCursor(); final SingleValueDimensionVectorSelector selector = cursor .getColumnSelectorFactory() @@ -1019,7 +1019,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest final String selectColumn ) { - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(makeFilter(filter)))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeVectorCursorBuildSpec(makeFilter(filter)))) { final VectorCursor cursor = cursorHolder.asVectorCursor(); final SingleValueDimensionVectorSelector selector = cursor .getColumnSelectorFactory() @@ -1046,7 +1046,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest ) { final Expr parsedIdentifier = Parser.parse(selectColumn, TestExprMacroTable.INSTANCE); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(makeVectorCursorBuildSpec(makeFilter(filter)))) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeVectorCursorBuildSpec(makeFilter(filter)))) { final VectorCursor cursor = cursorHolder.asVectorCursor(); final ExpressionType outputType = parsedIdentifier.getOutputType(cursor.getColumnSelectorFactory()); @@ -1102,10 +1102,6 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest ) { // Generate rowSignature - final RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); - for (String columnName : Iterables.concat(adapter.getAvailableDimensions(), adapter.getAvailableMetrics())) { - rowSignatureBuilder.add(columnName, adapter.getColumnCapabilities(columnName).toColumnType()); - } // Perform test final SettableSupplier rowSupplier = new SettableSupplier<>(); @@ -1114,7 +1110,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest RowBasedColumnSelectorFactory.create( RowAdapters.standardRow(), rowSupplier::get, - rowSignatureBuilder.build(), + cursorFactory.getRowSignature(), false, false ) @@ -1136,12 +1132,13 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest ) { // IncrementalIndex, RowBasedSegment cannot vectorize. - // Columnar FrameStorageAdapter *can* vectorize, but the tests won't pass, because the vectorizable cases - // differ from QueryableIndexStorageAdapter due to frames not having indexes. So, skip these too. + // ColumnarFrameCursorFactory *can* vectorize, but the tests won't pass, because the vectorizable cases + // differ from QueryableIndexCursorFactory due to frames not having indexes. So, skip these too. final boolean testVectorized = - !(adapter instanceof IncrementalIndexStorageAdapter) - && !(adapter instanceof RowBasedStorageAdapter) - && !(adapter instanceof FrameStorageAdapter); + !(cursorFactory instanceof IncrementalIndexCursorFactory) + && !(cursorFactory instanceof RowBasedCursorFactory) + && !(cursorFactory instanceof RowFrameCursorFactory) + && !(cursorFactory instanceof ColumnarFrameCursorFactory); assertFilterMatches(filter, expectedRows, testVectorized); // test double inverted @@ -1156,12 +1153,13 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest ) { // IncrementalIndex, RowBasedSegment cannot vectorize. - // Columnar FrameStorageAdapter *can* vectorize, but the tests won't pass, because the vectorizable cases - // differ from QueryableIndexStorageAdapter due to frames not having indexes. So, skip these too. + // ColumnarFrameCursorHolderFactory *can* vectorize, but the tests won't pass, because the vectorizable cases + // differ from QueryableIndexCursorFactory due to frames not having indexes. So, skip these too. final boolean testVectorized = - !(adapter instanceof IncrementalIndexStorageAdapter) - && !(adapter instanceof RowBasedStorageAdapter) - && !(adapter instanceof FrameStorageAdapter); + !(cursorFactory instanceof IncrementalIndexCursorFactory) + && !(cursorFactory instanceof RowBasedCursorFactory) + && !(cursorFactory instanceof RowFrameCursorFactory) + && !(cursorFactory instanceof ColumnarFrameCursorFactory); if (isAutoSchema()) { Throwable t = Assert.assertThrows( @@ -1246,19 +1244,19 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest } } - private static class AdapterStuff + private static class CursorStuff { - private final StorageAdapter adapter; + private final CursorFactory cursorFactory; private final VirtualColumns virtualColumns; private final Closeable closeable; - private AdapterStuff( - StorageAdapter adapter, + private CursorStuff( + CursorFactory cursorFactory, VirtualColumns virtualColumns, Closeable closeable ) { - this.adapter = adapter; + this.cursorFactory = cursorFactory; this.virtualColumns = virtualColumns; this.closeable = closeable; } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java index 036abc13cd6..69d80b3f3a2 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java @@ -33,8 +33,8 @@ import org.apache.druid.query.extraction.JavaScriptExtractionFn; import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Rule; @@ -62,7 +62,7 @@ public class BoundFilterTest extends BaseFilterTest public BoundFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java index 8a46d313d2d..3525e120c3e 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java @@ -38,8 +38,8 @@ import org.apache.druid.query.extraction.MapLookupExtractor; import org.apache.druid.query.filter.ColumnComparisonDimFilter; import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Test; import org.junit.runner.RunWith; @@ -78,7 +78,7 @@ public class ColumnComparisonFilterTest extends BaseFilterTest public ColumnComparisonFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java index 97be448e61a..0259a2c3432 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java @@ -40,8 +40,8 @@ import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.IsFalseDimFilter; import org.apache.druid.query.filter.IsTrueDimFilter; import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.AfterClass; @@ -62,7 +62,7 @@ public class EqualityFilterTests public EqualityFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterNonStrictBooleansTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterNonStrictBooleansTest.java index e9fea68c0df..8b60360b6c1 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterNonStrictBooleansTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterNonStrictBooleansTest.java @@ -25,8 +25,8 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.Pair; import org.apache.druid.math.expr.ExpressionProcessing; import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -40,7 +40,7 @@ public class ExpressionFilterNonStrictBooleansTest extends ExpressionFilterTest public ExpressionFilterNonStrictBooleansTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java index 1bade62c5c8..aa2d21ef9d3 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java @@ -41,8 +41,8 @@ import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.ExpressionDimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -108,7 +108,7 @@ public class ExpressionFilterTest extends BaseFilterTest public ExpressionFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FilterBundleTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FilterBundleTest.java index aa764535444..9bc86ae3900 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FilterBundleTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FilterBundleTest.java @@ -43,7 +43,12 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; +@RunWith(Parameterized.class) public class FilterBundleTest extends InitializedNullHandlingTest { private Closer closer; @@ -53,6 +58,15 @@ public class FilterBundleTest extends InitializedNullHandlingTest @Rule public TemporaryFolder tmpDir = new TemporaryFolder(); + @Parameters + public static Object[] flags() + { + return new Object[]{false, true}; + } + + @Parameter + public boolean cursorAutoArrangeFilters; + @Before public void setUp() { @@ -317,8 +331,7 @@ public class FilterBundleTest extends InitializedNullHandlingTest protected FilterBundle makeFilterBundle(final Filter filter) { - return filter.makeFilterBundle( - indexSelector, + return new FilterBundle.Builder(filter, indexSelector, cursorAutoArrangeFilters).build( new DefaultBitmapResultFactory(bitmapFactory), indexSelector.getNumRows(), indexSelector.getNumRows(), diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java index f95bd2ea8e9..d0d2941277a 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java @@ -39,8 +39,8 @@ import org.apache.druid.query.filter.DruidPredicateMatch; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.filter.cnf.CNFFilterExplosionException; import org.apache.druid.segment.index.BitmapColumnIndex; import org.junit.AfterClass; @@ -160,7 +160,7 @@ public class FilterPartitionTest extends BaseFilterTest public FilterPartitionTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java index c9717f18628..b1112e061fd 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java @@ -45,8 +45,8 @@ import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.search.ContainsSearchQuerySpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.junit.AfterClass; import org.junit.Test; @@ -93,7 +93,7 @@ public class FloatAndDoubleFilteringTest extends BaseFilterTest public FloatAndDoubleFilteringTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTests.java index 6f5c4b72eb1..fb0ebbfb031 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTests.java @@ -47,9 +47,9 @@ import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.TypedInFilter; import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.AfterClass; @@ -87,7 +87,7 @@ public class InFilterTests public InFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java index 54267591697..36702eedb1f 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java @@ -35,8 +35,8 @@ import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.junit.AfterClass; import org.junit.Test; @@ -79,7 +79,7 @@ public class InvalidFilteringTest extends BaseFilterTest public InvalidFilteringTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java index 98b9579a907..9e337fe6ece 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java @@ -32,8 +32,8 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.JavaScriptDimFilter; import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Rule; @@ -51,7 +51,7 @@ public class JavaScriptFilterTest extends BaseFilterTest public JavaScriptFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java index 5cc37bf4fa0..d3cc658d6ae 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java @@ -39,8 +39,8 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.LikeDimFilter; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.lookup.LookupExtractionFn; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.junit.AfterClass; @@ -85,7 +85,7 @@ public class LikeFilterTest extends BaseFilterTest public LikeFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java index 3fd1c55ebab..5011f337909 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java @@ -43,8 +43,8 @@ import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.search.ContainsSearchQuerySpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.junit.AfterClass; import org.junit.Test; @@ -88,7 +88,7 @@ public class LongFilteringTest extends BaseFilterTest public LongFilteringTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/NotFilterEvaluateTest.java b/processing/src/test/java/org/apache/druid/segment/filter/NotFilterEvaluateTest.java index ed1be8acde3..ddc21c6d374 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/NotFilterEvaluateTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/NotFilterEvaluateTest.java @@ -32,8 +32,8 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Test; import org.junit.runner.RunWith; @@ -67,7 +67,7 @@ public class NotFilterEvaluateTest extends BaseFilterTest public NotFilterEvaluateTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTests.java index 767f450fc2e..26d0ca547f4 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTests.java @@ -32,8 +32,8 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.NullFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Test; @@ -52,7 +52,7 @@ public class NullFilterTests public NullFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/OrFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/OrFilterTest.java index 691339fe498..54689d30d9d 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/OrFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/OrFilterTest.java @@ -38,8 +38,8 @@ import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.filter.TrueDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Test; import org.junit.runner.RunWith; @@ -73,7 +73,7 @@ public class OrFilterTest extends BaseFilterTest public OrFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java index e2c0be5909c..d55d71b792d 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java @@ -41,8 +41,8 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.RangeFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -117,7 +117,7 @@ public class RangeFilterTests public RangeFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java index a6ec0168177..744232ad6fc 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java @@ -31,8 +31,8 @@ import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.extraction.JavaScriptExtractionFn; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.RegexDimFilter; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Rule; @@ -49,7 +49,7 @@ public class RegexFilterTest extends BaseFilterTest public RegexFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java index 8aef0ef27ab..4c620eec8d8 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java @@ -33,8 +33,8 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.SearchQueryDimFilter; import org.apache.druid.query.search.ContainsSearchQuerySpec; import org.apache.druid.query.search.SearchQuerySpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Rule; @@ -51,7 +51,7 @@ public class SearchQueryFilterTest extends BaseFilterTest public SearchQueryFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java index c51b94fc59a..ed082c4598e 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java @@ -33,8 +33,8 @@ import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Test; @@ -55,7 +55,7 @@ public class SelectorFilterTest extends BaseFilterTest public SelectorFilterTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java index da4abbe587a..48c02ad71d7 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java @@ -47,8 +47,8 @@ import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.search.ContainsSearchQuerySpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnHolder; import org.junit.AfterClass; import org.junit.Test; @@ -88,7 +88,7 @@ public class TimeFilteringTest extends BaseFilterTest public TimeFilteringTest( String testName, IndexBuilder indexBuilder, - Function> finisher, + Function> finisher, boolean cnf, boolean optimize ) diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java index 77e0470c548..e1a7319cab1 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java @@ -25,9 +25,12 @@ import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorAndSize; import org.apache.druid.query.aggregation.LongMaxAggregator; import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; import org.apache.druid.segment.CloserRule; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.testing.InitializedNullHandlingTest; import org.easymock.EasyMock; import org.junit.Rule; @@ -69,22 +72,39 @@ public class IncrementalIndexIngestionTest extends InitializedNullHandlingTest { // Prepare the mocks & set close() call count expectation to 1 Aggregator mockedAggregator = EasyMock.createMock(LongMaxAggregator.class); + EasyMock.expect(mockedAggregator.aggregateWithSize()).andReturn(0L).anyTimes(); mockedAggregator.close(); EasyMock.expectLastCall().times(1); - final IncrementalIndex genericIndex = indexCreator.createIndex( + + EasyMock.replay(mockedAggregator); + + final IncrementalIndex incrementalIndex = indexCreator.createIndex( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.MINUTE) - .withMetrics(new LongMaxAggregatorFactory("max", "max")) + .withMetrics(new LongMaxAggregatorFactory("max", "max") + { + @Override + protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) + { + return mockedAggregator; + } + + @Override + public AggregatorAndSize factorizeWithSize(ColumnSelectorFactory metricFactory) + { + return new AggregatorAndSize(mockedAggregator, Long.BYTES); + } + }) .build() ); // This test is specific to the on-heap index - if (!(genericIndex instanceof OnheapIncrementalIndex)) { + if (!(incrementalIndex instanceof OnheapIncrementalIndex)) { return; } - final OnheapIncrementalIndex index = (OnheapIncrementalIndex) genericIndex; + final OnheapIncrementalIndex index = (OnheapIncrementalIndex) incrementalIndex; index.add(new MapBasedInputRow( 0, @@ -92,11 +112,7 @@ public class IncrementalIndexIngestionTest extends InitializedNullHandlingTest ImmutableMap.of("billy", 1, "max", 1) )); - // override the aggregators with the mocks - index.concurrentGet(0)[0] = mockedAggregator; - // close the indexer and validate the expectations - EasyMock.replay(mockedAggregator); index.close(); EasyMock.verify(mockedAggregator); } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java index 80c8207ed60..f5779bf7362 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java @@ -28,10 +28,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.BuiltInTypesModule; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.CloserRule; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Rule; @@ -80,15 +77,11 @@ public class IncrementalIndexMultiValueSpecTest extends InitializedNullHandlingT new StringDimensionSchema("string3", DimensionSchema.MultiValueHandling.SORTED_SET, true) ) ); - IncrementalIndexSchema schema = new IncrementalIndexSchema( - 0, - new TimestampSpec("ds", "auto", null), - Granularities.ALL, - VirtualColumns.EMPTY, - dimensionsSpec, - new AggregatorFactory[0], - false - ); + IncrementalIndexSchema schema = IncrementalIndexSchema.builder() + .withTimestampSpec(new TimestampSpec("ds", "auto", null)) + .withDimensionsSpec(dimensionsSpec) + .withRollup(false) + .build(); Map map = new HashMap() { @Override diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index bab827130e4..87c60cb7420 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -64,17 +64,19 @@ import org.apache.druid.segment.CloserRule; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.SelectorFilter; import org.apache.druid.segment.index.AllTrueBitmapColumnIndex; import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; @@ -182,18 +184,17 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT .addOrderByColumn("billy") .build(); final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); - final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); try ( CloseableStupidPool pool = new CloseableStupidPool<>( "GroupByQueryEngine-bufferPool", () -> ByteBuffer.allocate(50000) ); ResourceHolder processingBuffer = pool.take(); - final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec) + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec) ) { final Sequence rows = GroupByQueryEngine.process( query, - adapter, new IncrementalIndexTimeBoundaryInspector(index), cursorHolder, buildSpec, @@ -258,7 +259,7 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT ) .addOrderByColumn("billy") .build(); - final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); try ( CloseableStupidPool pool = new CloseableStupidPool<>( @@ -266,11 +267,10 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT () -> ByteBuffer.allocate(50000) ); ResourceHolder processingBuffer = pool.take(); - final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec) + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec) ) { final Sequence rows = GroupByQueryEngine.process( query, - adapter, new IncrementalIndexTimeBoundaryInspector(index), cursorHolder, buildSpec, @@ -320,7 +320,7 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT ) ); - IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); for (boolean descending : Arrays.asList(false, true)) { final CursorBuildSpec buildSpec = CursorBuildSpec @@ -330,7 +330,7 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT .setPreferredOrdering(descending ? Cursors.descendingTimeOrder() : Cursors.ascendingTimeOrder()) .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); DimensionSelector dimSelector; @@ -390,8 +390,7 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT .threshold(10) .aggregators(new LongSumAggregatorFactory("cnt", "cnt")) .build(), - new IncrementalIndexStorageAdapter(index), - new IncrementalIndexTimeBoundaryInspector(index), + new IncrementalIndexSegment(index, SegmentId.dummy("test")), null ) .toList(); @@ -430,7 +429,7 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) .setDimFilter(DimFilters.dimEquals("sally", (String) null)) .build(); - final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); + final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); try ( @@ -439,11 +438,10 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT () -> ByteBuffer.allocate(50000) ); ResourceHolder processingBuffer = pool.take(); - final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec) + final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec) ) { final Sequence rows = GroupByQueryEngine.process( query, - adapter, new IncrementalIndexTimeBoundaryInspector(index), cursorHolder, buildSpec, @@ -478,12 +476,12 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT ); } - final StorageAdapter sa = new IncrementalIndexStorageAdapter(index); + final CursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) .build(); - try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); DimensionSelector dimSelector = cursor .getColumnSelectorFactory() @@ -534,13 +532,13 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT ); } - final StorageAdapter sa = new IncrementalIndexStorageAdapter(index); + final CursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(new DictionaryRaceTestFilter(index, timestamp)) .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) .build(); - try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); DimensionSelector dimSelector = cursor .getColumnSelectorFactory() @@ -579,12 +577,12 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT index.add(rows.get(i)); } - final StorageAdapter sa = new IncrementalIndexStorageAdapter(index); + final CursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setInterval(Intervals.utc(timestamp - 60_000, timestamp + 60_000)) .build(); - try (final CursorHolder cursorHolder = sa.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { Cursor cursor = cursorHolder.asCursor(); DimensionSelector dimSelector1A = cursor diff --git a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentCursorFactoryTest.java similarity index 95% rename from processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentCursorFactoryTest.java index c3a59659afc..105b2ae7e3c 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentCursorFactoryTest.java @@ -28,6 +28,7 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; @@ -54,7 +55,7 @@ import java.io.IOException; import java.util.Collections; import java.util.List; -public class BaseHashJoinSegmentStorageAdapterTest extends InitializedNullHandlingTest +public class BaseHashJoinSegmentCursorFactoryTest extends InitializedNullHandlingTest { public static JoinFilterRewriteConfig DEFAULT_JOIN_FILTER_REWRITE_CONFIG = new JoinFilterRewriteConfig( true, @@ -223,17 +224,18 @@ public class BaseHashJoinSegmentStorageAdapterTest extends InitializedNullHandli /** * Creates a fact-to-country join segment without a {@link JoinFilterPreAnalysis}. This means it cannot - * have {@link org.apache.druid.segment.StorageAdapter#makeCursorHolder(CursorBuildSpec)} called on it. + * have {@link org.apache.druid.segment.CursorFactory#makeCursorHolder(CursorBuildSpec)} called on it. */ - protected HashJoinSegmentStorageAdapter makeFactToCountrySegment() + protected HashJoinSegment makeFactToCountrySegment() { return makeFactToCountrySegment(JoinType.LEFT); } - protected HashJoinSegmentStorageAdapter makeFactToCountrySegment(JoinType joinType) + protected HashJoinSegment makeFactToCountrySegment(JoinType joinType) { - return new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + return new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + null, ImmutableList.of(factToCountryOnIsoCode(joinType)), null ); diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java similarity index 86% rename from processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java index 7ae04d9fcb1..1aedc1cd3c3 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentCursorFactoryTest.java @@ -32,6 +32,8 @@ import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.TopNOptimizationInspector; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ValueType; @@ -47,14 +49,14 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorageAdapterTest +public class HashJoinSegmentCursorFactoryTest extends BaseHashJoinSegmentCursorFactoryTest { @Test public void test_getInterval_factToCountry() { Assert.assertEquals( Intervals.of("2015-09-12/2015-09-12T05:21:00.060Z"), - makeFactToCountrySegment().getInterval() + makeFactToCountrySegment().getDataInterval() ); } @@ -79,133 +81,14 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag "c1.countryIsoCode", "c1.countryName" ), - Lists.newArrayList(makeFactToCountrySegment().getRowSignature().getColumnNames()) + Lists.newArrayList(makeFactToCountrySegment().asCursorFactory().getRowSignature().getColumnNames()) ); } - @Test - public void test_getAvailableDimensions_factToCountry() - { - Assert.assertEquals( - ImmutableList.of( - "channel", - "regionIsoCode", - "countryNumber", - "countryIsoCode", - "user", - "isRobot", - "isAnonymous", - "namespace", - "page", - "delta", - "c1.countryNumber", - "c1.countryIsoCode", - "c1.countryName" - ), - Lists.newArrayList(makeFactToCountrySegment().getAvailableDimensions().iterator()) - ); - } - - @Test - public void test_getAvailableMetrics_factToCountry() - { - Assert.assertEquals( - ImmutableList.of("channel_uniques"), - Lists.newArrayList(makeFactToCountrySegment().getAvailableMetrics().iterator()) - ); - } - - @Test - public void test_getDimensionCardinality_factToCountryFactColumn() - { - Assert.assertEquals( - 18, - makeFactToCountrySegment().getDimensionCardinality("countryIsoCode") - ); - } - - @Test - public void test_getDimensionCardinality_factToCountryJoinColumn() - { - Assert.assertEquals( - 19, - makeFactToCountrySegment().getDimensionCardinality(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName") - ); - } - - @Test - public void test_getDimensionCardinality_factToCountryNonexistentFactColumn() - { - Assert.assertEquals( - 1, - makeFactToCountrySegment().getDimensionCardinality("nonexistent") - ); - } - - @Test - public void test_getDimensionCardinality_factToCountryNonexistentJoinColumn() - { - Assert.assertEquals( - 1, - makeFactToCountrySegment().getDimensionCardinality(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent") - ); - } - - @Test - public void test_getMinValue_factToCountryFactColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMinValue("countryIsoCode")); - } - - @Test - public void test_getMinValue_factToCountryJoinColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMinValue(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode")); - } - - @Test - public void test_getMinValue_factToCountryNonexistentFactColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMinValue("nonexistent")); - } - - @Test - public void test_getMinValue_factToCountryNonexistentJoinColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMinValue(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent")); - } - - @Test - public void test_getMaxValue_factToCountryFactColumn() - { - Assert.assertEquals( - "US", - makeFactToCountrySegment().getMaxValue("countryIsoCode") - ); - } - - @Test - public void test_getMaxValue_factToCountryJoinColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMaxValue(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName")); - } - - @Test - public void test_getMaxValue_factToCountryNonexistentFactColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMaxValue("nonexistent")); - } - - @Test - public void test_getMaxValue_factToCountryNonexistentJoinColumn() - { - Assert.assertNull(makeFactToCountrySegment().getMaxValue(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent")); - } - @Test public void test_getColumnCapabilities_factToCountryFactColumn() { - final ColumnCapabilities capabilities = makeFactToCountrySegment().getColumnCapabilities("countryIsoCode"); + final ColumnCapabilities capabilities = makeFactToCountrySegment().asCursorFactory().getColumnCapabilities("countryIsoCode"); Assert.assertEquals(ValueType.STRING, capabilities.getType()); Assert.assertTrue(capabilities.hasBitmapIndexes()); @@ -217,7 +100,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag @Test public void test_getColumnCapabilities_factToCountryJoinColumn() { - final ColumnCapabilities capabilities = makeFactToCountrySegment().getColumnCapabilities( + final ColumnCapabilities capabilities = makeFactToCountrySegment().asCursorFactory().getColumnCapabilities( FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode" ); @@ -231,7 +114,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag @Test public void test_getColumnCapabilities_factToCountryNonexistentFactColumn() { - final ColumnCapabilities capabilities = makeFactToCountrySegment() + final ColumnCapabilities capabilities = makeFactToCountrySegment().asCursorFactory() .getColumnCapabilities("nonexistent"); Assert.assertNull(capabilities); @@ -240,7 +123,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag @Test public void test_getColumnCapabilities_factToCountryNonexistentJoinColumn() { - final ColumnCapabilities capabilities = makeFactToCountrySegment() + final ColumnCapabilities capabilities = makeFactToCountrySegment().asCursorFactory() .getColumnCapabilities(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "nonexistent"); Assert.assertNull(capabilities); @@ -251,7 +134,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag { Assert.assertEquals( "hyperUnique", - makeFactToCountrySegment().getColumnCapabilities("channel_uniques").getComplexTypeName() + makeFactToCountrySegment().asCursorFactory().getColumnCapabilities("channel_uniques").getComplexTypeName() ); } @@ -260,7 +143,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag { Assert.assertEquals( "COMPLEX", - makeFactToCountrySegment().getColumnCapabilities("channel_uniques").asTypeString() + makeFactToCountrySegment().asCursorFactory().getColumnCapabilities("channel_uniques").asTypeString() ); } @@ -269,29 +152,12 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag { Assert.assertEquals( "STRING", - makeFactToCountrySegment().getColumnCapabilities(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName") + makeFactToCountrySegment().asCursorFactory() + .getColumnCapabilities(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName") .asTypeString() ); } - @Test - public void test_getNumRows_factToCountry() - { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("Cannot retrieve number of rows from join segment"); - - makeFactToCountrySegment().getNumRows(); - } - - @Test - public void test_getMetadata_factToCountry() - { - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("Cannot retrieve metadata from join segment"); - - makeFactToCountrySegment().getMetadata(); - } - @Test public void test_makeCursor_factToCountryLeft() { @@ -303,8 +169,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -360,8 +227,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -415,8 +283,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -465,8 +334,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -517,8 +387,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -577,8 +448,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -633,8 +505,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -664,8 +537,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -693,8 +567,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -708,7 +583,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L}, @@ -727,8 +602,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -741,7 +617,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"}, @@ -760,8 +636,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -775,7 +652,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis", 14L}, @@ -794,8 +671,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -808,7 +686,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{null, null, NullHandling.sqlCompatible() ? null : 0L, "AX", "Atlantis"}, @@ -833,8 +711,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -870,8 +749,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -907,8 +787,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -944,8 +825,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -994,8 +876,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1065,8 +948,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1119,8 +1003,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1176,8 +1061,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1241,8 +1127,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -1252,7 +1139,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{"Diskussion:Sebastian Schulz", "Australia"}, @@ -1302,8 +1189,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1342,8 +1230,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -1353,7 +1242,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{"Diskussion:Sebastian Schulz", "Australia"}, @@ -1403,8 +1292,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1448,8 +1338,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag virtualColumns ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1499,8 +1390,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag virtualColumns ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1547,8 +1439,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1592,8 +1485,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1638,8 +1532,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -1652,7 +1547,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); CursorBuildSpec buildSpec = CursorBuildSpec.builder().setFilter(filter).build(); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(buildSpec), + cursorFactory.makeCursorHolder(buildSpec), columns, ImmutableList.of( new Object[]{"Giusy Ferreri discography", "VA", "IT", "Provincia di Varese", "IT"}, @@ -1691,8 +1586,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); JoinTestHelper.readCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1726,8 +1622,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); JoinTestHelper.readCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1761,8 +1658,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); JoinTestHelper.readCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1796,8 +1694,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag ); JoinTestHelper.readCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder(CursorBuildSpec.FULL_SCAN), @@ -1817,8 +1716,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1846,8 +1746,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1874,8 +1775,9 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ).makeCursorHolder( @@ -1899,8 +1801,8 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), baseFilter, joinableClauses, joinFilterPreAnalysis @@ -1936,8 +1838,8 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag VirtualColumns.EMPTY ); JoinTestHelper.verifyCursor( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), baseFilter, joinableClauses, joinFilterPreAnalysis @@ -1971,8 +1873,8 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), baseFilter, joinableClauses, joinFilterPreAnalysis @@ -1985,7 +1887,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN), + cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN), columns, ImmutableList.of( new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L}, @@ -2026,8 +1928,8 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), baseFilter, joinableClauses, joinFilterPreAnalysis @@ -2040,7 +1942,7 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN), + cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN), columns, ImmutableList.of( new Object[]{"Didier Leclair", "CA", "CA", "Canada", 1L}, @@ -2071,111 +1973,100 @@ public class HashJoinSegmentStorageAdapterTest extends BaseHashJoinSegmentStorag @Test public void test_hasBuiltInFiltersForSingleJoinableClauseWithVariousJoinTypes() { - Assert.assertTrue(makeFactToCountrySegment(JoinType.INNER).hasBuiltInFilters()); - Assert.assertFalse(makeFactToCountrySegment(JoinType.LEFT).hasBuiltInFilters()); - Assert.assertFalse(makeFactToCountrySegment(JoinType.RIGHT).hasBuiltInFilters()); - Assert.assertFalse(makeFactToCountrySegment(JoinType.FULL).hasBuiltInFilters()); + Assert.assertFalse(makeFactToCountrySegment(JoinType.INNER).as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); + Assert.assertTrue(makeFactToCountrySegment(JoinType.LEFT).as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); + Assert.assertFalse(makeFactToCountrySegment(JoinType.RIGHT).as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); + Assert.assertTrue(makeFactToCountrySegment(JoinType.FULL).as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); // cross join - Assert.assertFalse( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - new JoinableClause( + HashJoinSegment segment = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + null, + ImmutableList.of( + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.INNER, + JoinConditionAnalysis.forExpression( + "'true'", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - new IndexedTableJoinable(countriesTable), - JoinType.INNER, - JoinConditionAnalysis.forExpression( - "'true'", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - ExprMacroTable.nil() - ) + ExprMacroTable.nil() ) - ), - null - ).hasBuiltInFilters() - ); - } - - @Test - public void test_hasBuiltInFiltersForEmptyJoinableClause() - { - Assert.assertFalse( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of(), - null - ).hasBuiltInFilters() + ) + ), + null ); + TopNOptimizationInspector inspector = segment.as(TopNOptimizationInspector.class); + Assert.assertTrue(inspector.areAllDictionaryIdsPresent()); } @Test public void test_hasBuiltInFiltersForConvertedJoin() { - Assert.assertTrue( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - new InDimFilter("dim", ImmutableSet.of("foo", "bar")), - ImmutableList.of(), - null - ).hasBuiltInFilters() + final HashJoinSegment segment = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + new InDimFilter("dim", ImmutableSet.of("foo", "bar")), + ImmutableList.of(), + null ); + final TopNOptimizationInspector inspector = segment.as(TopNOptimizationInspector.class); + Assert.assertFalse(inspector.areAllDictionaryIdsPresent()); } @Test public void test_hasBuiltInFiltersForMultipleJoinableClausesWithVariousJoinTypes() { - Assert.assertTrue( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.INNER), - regionToCountry(JoinType.LEFT) - ), - null - ).hasBuiltInFilters() + final HashJoinSegment segment = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + null, + ImmutableList.of( + factToRegion(JoinType.INNER), + regionToCountry(JoinType.LEFT) + ), + null ); + Assert.assertFalse(segment.as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); - Assert.assertTrue( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.RIGHT), - regionToCountry(JoinType.INNER), - factToCountryOnNumber(JoinType.FULL) - ), - null - ).hasBuiltInFilters() + final HashJoinSegment segment2 = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + null, + ImmutableList.of( + factToRegion(JoinType.RIGHT), + regionToCountry(JoinType.INNER), + factToCountryOnNumber(JoinType.FULL) + ), + null ); + Assert.assertFalse(segment2.as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); - Assert.assertFalse( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - regionToCountry(JoinType.LEFT) - ), - null - ).hasBuiltInFilters() + final HashJoinSegment segment3 = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + null, + ImmutableList.of( + factToRegion(JoinType.LEFT), + regionToCountry(JoinType.LEFT) + ), + null ); + Assert.assertTrue(segment3.as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); - Assert.assertFalse( - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - ImmutableList.of( - factToRegion(JoinType.LEFT), - new JoinableClause( + final HashJoinSegment segment4 = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(factSegment), + null, + ImmutableList.of( + factToRegion(JoinType.LEFT), + new JoinableClause( + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.INNER, + JoinConditionAnalysis.forExpression( + "'true'", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - new IndexedTableJoinable(countriesTable), - JoinType.INNER, - JoinConditionAnalysis.forExpression( - "'true'", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX, - ExprMacroTable.nil() - ) + ExprMacroTable.nil() ) - ), - null - ).hasBuiltInFilters() + ) + ), + null ); + Assert.assertTrue(segment4.as(TopNOptimizationInspector.class).areAllDictionaryIdsPresent()); } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java index 9f9905e47d4..55e245bdb1c 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentTest.java @@ -23,12 +23,12 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.MaxIngestedEventTimeInspector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -163,9 +163,9 @@ public class HashJoinSegmentTest extends InitializedNullHandlingTest } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return referencedSegment.asStorageAdapter(); + return referencedSegment.asCursorFactory(); } @Nullable @@ -234,14 +234,15 @@ public class HashJoinSegmentTest extends InitializedNullHandlingTest public void test_asQueryableIndex() { Assert.assertNull(hashJoinSegment.asQueryableIndex()); + Assert.assertNull(hashJoinSegment.as(QueryableIndex.class)); } @Test public void test_asStorageAdapter() { Assert.assertThat( - hashJoinSegment.asStorageAdapter(), - CoreMatchers.instanceOf(HashJoinSegmentStorageAdapter.class) + hashJoinSegment.asCursorFactory(), + CoreMatchers.instanceOf(HashJoinSegmentCursorFactory.class) ); } diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java index 656cbdb2ba2..e043e40fa48 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java @@ -58,7 +58,7 @@ import java.util.Arrays; import java.util.List; import java.util.Set; -public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTest +public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentCursorFactoryTest { @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() @@ -75,14 +75,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -147,14 +148,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -198,14 +200,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -260,14 +263,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -326,14 +330,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -391,14 +396,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes virtualColumns ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder() .setFilter(originalFilter) .setVirtualColumns(virtualColumns) @@ -463,14 +469,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses.getJoinableClauses(), virtualColumns ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses.getJoinableClauses(), joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder() .setFilter(originalFilter) .setVirtualColumns(virtualColumns) @@ -565,14 +572,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -687,14 +695,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -777,13 +786,14 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes expectedException.expectMessage( "Cannot build hash-join matcher on non-equi-join condition: \"r1.regionIsoCode\" == regionIsoCode && reverse(\"r1.countryIsoCode\") == countryIsoCode"); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -821,14 +831,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -918,14 +929,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( @@ -1003,14 +1015,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( @@ -1072,14 +1085,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1125,14 +1139,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1177,14 +1192,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1228,14 +1244,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1280,14 +1297,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1330,8 +1348,9 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -1339,7 +1358,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1389,8 +1408,9 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -1398,7 +1418,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes // In non-SQL-compatible mode, we get an extra row, since the 'null' countryNumber for "Talk:Oswald Tilghman" // is interpreted as 0 (a.k.a. Australia). JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1447,14 +1467,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1497,14 +1518,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1546,14 +1568,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( @@ -1599,14 +1622,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(filter).build() ), ImmutableList.of( @@ -1651,14 +1675,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1702,14 +1727,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1764,14 +1790,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1830,14 +1857,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1898,8 +1926,9 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -1913,7 +1942,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -1962,14 +1991,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses.getJoinableClauses(), joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -2037,14 +2067,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses.getJoinableClauses(), joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -2206,8 +2237,9 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes joinableClauses, VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); @@ -2242,7 +2274,7 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes } JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -2364,14 +2396,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( @@ -2439,14 +2472,15 @@ public class JoinFilterAnalyzerTest extends BaseHashJoinSegmentStorageAdapterTes ) ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), + HashJoinSegmentCursorFactory cursorFactory = new HashJoinSegmentCursorFactory( + factSegment.asCursorFactory(), + null, joinableClauses, joinFilterPreAnalysis ); JoinTestHelper.verifyCursor( - adapter.makeCursorHolder( + cursorFactory.makeCursorHolder( CursorBuildSpec.builder().setFilter(originalFilter).build() ), ImmutableList.of( diff --git a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java index 1a5e6dc63de..364e12ccd2d 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/PostJoinCursorTest.java @@ -27,12 +27,15 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.timeline.SegmentId; import org.junit.Test; @@ -47,7 +50,7 @@ import static java.lang.Thread.sleep; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -public class PostJoinCursorTest extends BaseHashJoinSegmentStorageAdapterTest +public class PostJoinCursorTest extends BaseHashJoinSegmentCursorFactoryTest { public QueryableIndexSegment infiniteFactSegment; @@ -58,45 +61,58 @@ public class PostJoinCursorTest extends BaseHashJoinSegmentStorageAdapterTest */ private static class TestInfiniteQueryableIndexSegment extends QueryableIndexSegment { - - private static class InfiniteQueryableIndexStorageAdapter extends QueryableIndexStorageAdapter + private static class InfiniteCursorFactory implements CursorFactory { + final CursorFactory delegate; CountDownLatch countDownLatch; - public InfiniteQueryableIndexStorageAdapter(QueryableIndex index, CountDownLatch countDownLatch) + public InfiniteCursorFactory(CursorFactory delegate, CountDownLatch countDownLatch) { - super(index); + this.delegate = delegate; this.countDownLatch = countDownLatch; } @Override public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - final CursorHolder delegate = super.makeCursorHolder(spec); + final CursorHolder holder = delegate.makeCursorHolder(spec); return new CursorHolder() { @Nullable @Override public Cursor asCursor() { - return new CursorNoAdvance(delegate.asCursor(), countDownLatch); + return new CursorNoAdvance(holder.asCursor(), countDownLatch); } @Nullable @Override public List getOrdering() { - return delegate.getOrdering(); + return holder.getOrdering(); } @Override public void close() { - delegate.close(); + holder.close(); } }; } + @Override + public RowSignature getRowSignature() + { + return delegate.getRowSignature(); + } + + @Override + @Nullable + public ColumnCapabilities getColumnCapabilities(String column) + { + return delegate.getColumnCapabilities(column); + } + private static class CursorNoAdvance implements Cursor { Cursor cursor; @@ -149,18 +165,18 @@ public class PostJoinCursorTest extends BaseHashJoinSegmentStorageAdapterTest } } - private final StorageAdapter testStorageAdaptor; + private final CursorFactory cursorFactory; public TestInfiniteQueryableIndexSegment(QueryableIndex index, SegmentId segmentId, CountDownLatch countDownLatch) { super(index, segmentId); - testStorageAdaptor = new InfiniteQueryableIndexStorageAdapter(index, countDownLatch); + cursorFactory = new InfiniteCursorFactory(new QueryableIndexCursorFactory(index), countDownLatch); } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return testStorageAdaptor; + return cursorFactory; } } @@ -184,6 +200,17 @@ public class PostJoinCursorTest extends BaseHashJoinSegmentStorageAdapterTest @Test public void testAdvanceWithInterruption() throws IOException, InterruptedException + { + testAdvance(true); + } + + @Test + public void testAdvanceWithoutInterruption() throws IOException, InterruptedException + { + testAdvance(false); + } + + private void testAdvance(boolean withInterruption) throws IOException, InterruptedException { final int rowsBeforeInterrupt = 1000; @@ -198,7 +225,7 @@ public class PostJoinCursorTest extends BaseHashJoinSegmentStorageAdapterTest countriesTable = JoinTestHelper.createCountriesIndexedTable(); - Thread joinCursorThread = new Thread(() -> makeCursorAndAdvance()); + Thread joinCursorThread = new Thread(() -> makeCursorAndAdvance(withInterruption)); ExceptionHandler exceptionHandler = new ExceptionHandler(); joinCursorThread.setUncaughtExceptionHandler(exceptionHandler); joinCursorThread.start(); @@ -218,7 +245,7 @@ public class PostJoinCursorTest extends BaseHashJoinSegmentStorageAdapterTest fail(); } - public void makeCursorAndAdvance() + public void makeCursorAndAdvance(boolean withInterruption) { List joinableClauses = ImmutableList.of( @@ -231,13 +258,14 @@ public class PostJoinCursorTest extends BaseHashJoinSegmentStorageAdapterTest VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter hashJoinSegmentStorageAdapter = new HashJoinSegmentStorageAdapter( - infiniteFactSegment.asStorageAdapter(), + HashJoinSegment hashJoinSegment = new HashJoinSegment( + ReferenceCountingSegment.wrapRootGenerationSegment(infiniteFactSegment), + null, joinableClauses, joinFilterPreAnalysis ); - try (final CursorHolder cursorHolder = hashJoinSegmentStorageAdapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + try (final CursorHolder cursorHolder = hashJoinSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ((PostJoinCursor) cursor).setValueMatcher(new ValueMatcher() @@ -255,7 +283,11 @@ public class PostJoinCursorTest extends BaseHashJoinSegmentStorageAdapterTest } }); - cursor.advance(); + if (withInterruption) { + cursor.advance(); + } else { + cursor.advanceUninterruptibly(); + } } } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java index 9b8f7f4d916..8df2bffb2fd 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTableTest.java @@ -42,8 +42,10 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.PhysicalSegmentInspector; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.TestIndex; @@ -142,8 +144,8 @@ public class BroadcastSegmentIndexedTableTest extends InitializedNullHandlingTes segment.getTotalSpace() ); backingSegment = (QueryableIndexSegment) factory.factorize(dataSegment, segment, false, SegmentLazyLoadFailCallback.NOOP); - columnNames = - new QueryableIndexStorageAdapter(backingSegment.asQueryableIndex()).getRowSignature().getColumnNames(); + columnNames = new QueryableIndexCursorFactory(backingSegment.as(QueryableIndex.class)).getRowSignature() + .getColumnNames(); broadcastTable = new BroadcastSegmentIndexedTable(backingSegment, keyColumns, dataSegment.getVersion()); } @@ -287,11 +289,11 @@ public class BroadcastSegmentIndexedTableTest extends InitializedNullHandlingTes checkColumnSelectorFactory(columnName); try (final Closer closer = Closer.create()) { final int columnIndex = columnNames.indexOf(columnName); - final int numRows = backingSegment.asStorageAdapter().getNumRows(); + final int numRows = backingSegment.as(PhysicalSegmentInspector.class).getNumRows(); final IndexedTable.Reader reader = broadcastTable.columnReader(columnIndex); closer.register(reader); final SimpleAscendingOffset offset = new SimpleAscendingOffset(numRows); - final BaseColumn theColumn = backingSegment.asQueryableIndex() + final BaseColumn theColumn = backingSegment.as(QueryableIndex.class) .getColumnHolder(columnName) .getColumn(); closer.register(theColumn); @@ -317,10 +319,10 @@ public class BroadcastSegmentIndexedTableTest extends InitializedNullHandlingTes private void checkColumnSelectorFactory(String columnName) { try (final Closer closer = Closer.create()) { - final int numRows = backingSegment.asStorageAdapter().getNumRows(); + final int numRows = backingSegment.as(PhysicalSegmentInspector.class).getNumRows(); final SimpleAscendingOffset offset = new SimpleAscendingOffset(numRows); - final BaseColumn theColumn = backingSegment.asQueryableIndex() + final BaseColumn theColumn = backingSegment.as(QueryableIndex.class) .getColumnHolder(columnName) .getColumn(); closer.register(theColumn); diff --git a/processing/src/test/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactoryTest.java index 557bf4b8dfc..8a877d6f8ed 100644 --- a/processing/src/test/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/loading/TombstoneSegmentizerFactoryTest.java @@ -21,20 +21,27 @@ package org.apache.druid.segment.loading; import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.TombstoneShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; +import javax.annotation.Nullable; + import static org.junit.Assert.assertThrows; public class TombstoneSegmentizerFactoryTest { - @Test public void testSegmentCreation() { @@ -50,11 +57,11 @@ public class TombstoneSegmentizerFactoryTest .build(); Segment segment = factory.factorize(tombstone, null, true, null); - Assert.assertNotNull(segment.asStorageAdapter()); + Assert.assertNotNull(segment.asCursorFactory()); Assert.assertEquals("foo_2021-01-01T00:00:00.000Z_2022-01-01T00:00:00.000Z_1", segment.getId().toString()); Assert.assertEquals(expectedInterval, segment.getDataInterval()); - QueryableIndex queryableIndex = segment.asQueryableIndex(); + QueryableIndex queryableIndex = segment.as(QueryableIndex.class); Assert.assertNotNull(queryableIndex); assertThrows(UnsupportedOperationException.class, queryableIndex::getNumRows); assertThrows(UnsupportedOperationException.class, queryableIndex::getAvailableDimensions); @@ -63,10 +70,77 @@ public class TombstoneSegmentizerFactoryTest assertThrows(UnsupportedOperationException.class, queryableIndex::getDimensionHandlers); assertThrows(UnsupportedOperationException.class, queryableIndex::getColumnNames); assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getColumnHolder(null)); + Assert.assertTrue(segment.isTombstone()); - StorageAdapter storageAdapter = segment.asStorageAdapter(); - Assert.assertNotNull(storageAdapter); - Assert.assertTrue(storageAdapter.isFromTombstone()); + Assert.assertNotNull(segment.asCursorFactory()); } + + @Test + public void testDefaultSegmentIsTombstoneFalse() + { + Segment segment = new Segment() + { + @Override + public SegmentId getId() + { + return null; + } + + @Override + public Interval getDataInterval() + { + return null; + } + + @Nullable + @Override + public QueryableIndex asQueryableIndex() + { + return null; + } + + @Override + public CursorFactory asCursorFactory() + { + return new CursorFactory() + { + @Override + public CursorHolder makeCursorHolder(CursorBuildSpec spec) + { + return new CursorHolder() + { + @Nullable + @Override + public Cursor asCursor() + { + return null; + } + }; + } + + @Override + public RowSignature getRowSignature() + { + return RowSignature.empty(); + } + + @Override + @Nullable + public ColumnCapabilities getColumnCapabilities(String column) + { + return null; + } + }; + } + + @Override + public void close() + { + + } + }; + + Assert.assertFalse(segment.isTombstone()); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java index fd89e083441..377db0f4f54 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java @@ -37,7 +37,6 @@ import org.apache.druid.segment.DoubleColumnSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.LongColumnSelector; import org.apache.druid.segment.Segment; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.transform.TransformSpec; @@ -344,11 +343,11 @@ public class NestedFieldColumnSelectorsTest extends InitializedNullHandlingTest IndexSpec.DEFAULT ); Assert.assertEquals(1, segments.size()); - StorageAdapter storageAdapter = segments.get(0).asStorageAdapter(); + Segment segment = segments.get(0); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setVirtualColumns(virtualColumns) .build(); - final CursorHolder cursorHolder = closer.register(storageAdapter.makeCursorHolder(buildSpec)); + final CursorHolder cursorHolder = closer.register(segment.asCursorFactory().makeCursorHolder(buildSpec)); final Cursor cursor = cursorHolder.asCursor(); return cursor.getColumnSelectorFactory(); } @@ -369,9 +368,9 @@ public class NestedFieldColumnSelectorsTest extends InitializedNullHandlingTest IndexSpec.DEFAULT ); Assert.assertEquals(1, segments.size()); - StorageAdapter storageAdapter = segments.get(0).asStorageAdapter(); + Segment segment = segments.get(0); final CursorBuildSpec buildSpec = CursorBuildSpec.builder().setVirtualColumns(virtualColumns).build(); - VectorCursor cursor = closer.register(storageAdapter.makeCursorHolder(buildSpec)).asVectorCursor(); + VectorCursor cursor = closer.register(segment.asCursorFactory().makeCursorHolder(buildSpec)).asVectorCursor(); return cursor.getColumnSelectorFactory(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/vector/NilVectorSelectorTest.java b/processing/src/test/java/org/apache/druid/segment/vector/NilVectorSelectorTest.java index 2f4c93e862e..704ef9bf11a 100644 --- a/processing/src/test/java/org/apache/druid/segment/vector/NilVectorSelectorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/vector/NilVectorSelectorTest.java @@ -21,7 +21,7 @@ package org.apache.druid.segment.vector; import org.apache.druid.collections.bitmap.WrappedRoaringBitmap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.query.QueryContexts; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -35,7 +35,7 @@ public class NilVectorSelectorTest extends InitializedNullHandlingTest @Test public void testDefaultSizedVector() { - testVectorSize(QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE); + testVectorSize(QueryContexts.DEFAULT_VECTOR_SIZE); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java index 13f82e2be8e..a3060f078a2 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java @@ -25,7 +25,6 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.guava.SettableSupplier; import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; @@ -44,13 +43,12 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestObjectColumnSelector; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; @@ -58,8 +56,8 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -79,10 +77,10 @@ public class ExpressionSelectorsTest extends InitializedNullHandlingTest { private static Closer CLOSER; private static QueryableIndex QUERYABLE_INDEX; - private static QueryableIndexStorageAdapter QUERYABLE_INDEX_STORAGE_ADAPTER; + private static QueryableIndexCursorFactory QUERYABLE_INDEX_CURSOR_FACTORY; private static IncrementalIndex INCREMENTAL_INDEX; - private static IncrementalIndexStorageAdapter INCREMENTAL_INDEX_STORAGE_ADAPTER; - private static List ADAPTERS; + private static IncrementalIndexCursorFactory INCREMENTAL_INDEX_CURSOR_FACTORY; + private static List CURSOR_FACTORIES; private static final ColumnCapabilities SINGLE_VALUE = new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setDictionaryEncoded(true) @@ -116,14 +114,17 @@ public class ExpressionSelectorsTest extends InitializedNullHandlingTest INCREMENTAL_INDEX = CLOSER.register( segmentGenerator.generateIncrementalIndex(dataSegment, schemaInfo, Granularities.HOUR, numRows) ); - INCREMENTAL_INDEX_STORAGE_ADAPTER = new IncrementalIndexStorageAdapter(INCREMENTAL_INDEX); + INCREMENTAL_INDEX_CURSOR_FACTORY = new IncrementalIndexCursorFactory(INCREMENTAL_INDEX); QUERYABLE_INDEX = CLOSER.register( segmentGenerator.generate(dataSegment, schemaInfo, Granularities.HOUR, numRows) ); - QUERYABLE_INDEX_STORAGE_ADAPTER = new QueryableIndexStorageAdapter(QUERYABLE_INDEX); + QUERYABLE_INDEX_CURSOR_FACTORY = new QueryableIndexCursorFactory(QUERYABLE_INDEX); - ADAPTERS = ImmutableList.of(INCREMENTAL_INDEX_STORAGE_ADAPTER, QUERYABLE_INDEX_STORAGE_ADAPTER); + CURSOR_FACTORIES = ImmutableList.of( + INCREMENTAL_INDEX_CURSOR_FACTORY, + QUERYABLE_INDEX_CURSOR_FACTORY + ); } @AfterClass @@ -137,7 +138,7 @@ public class ExpressionSelectorsTest extends InitializedNullHandlingTest public void test_single_value_string_bindings() { final String columnName = "string3"; - for (StorageAdapter adapter : ADAPTERS) { + for (CursorFactory adapter : CURSOR_FACTORIES) { try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); @@ -202,7 +203,7 @@ public class ExpressionSelectorsTest extends InitializedNullHandlingTest public void test_multi_value_string_bindings() { final String columnName = "multi-string3"; - for (StorageAdapter adapter : ADAPTERS) { + for (CursorFactory adapter : CURSOR_FACTORIES) { try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -279,7 +280,7 @@ public class ExpressionSelectorsTest extends InitializedNullHandlingTest public void test_long_bindings() { final String columnName = "long3"; - for (StorageAdapter adapter : ADAPTERS) { + for (CursorFactory adapter : CURSOR_FACTORIES) { try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -324,7 +325,7 @@ public class ExpressionSelectorsTest extends InitializedNullHandlingTest public void test_double_bindings() { final String columnName = "double3"; - for (StorageAdapter adapter : ADAPTERS) { + for (CursorFactory adapter : CURSOR_FACTORIES) { try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); ColumnSelectorFactory factory = cursor.getColumnSelectorFactory(); @@ -617,15 +618,10 @@ public class ExpressionSelectorsTest extends InitializedNullHandlingTest // underlying dimension selector. // This occurred during schemaless ingestion with spare dimension values and no explicit null rows, so the // conditions are replicated by this test. See https://github.com/apache/druid/pull/10248 for details - IncrementalIndexSchema schema = new IncrementalIndexSchema( - 0, - new TimestampSpec("time", "millis", DateTimes.nowUtc()), - Granularities.NONE, - VirtualColumns.EMPTY, - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new CountAggregatorFactory("count")}, - true - ); + IncrementalIndexSchema schema = IncrementalIndexSchema.builder() + .withTimestampSpec(new TimestampSpec("time", "millis", DateTimes.nowUtc())) + .withMetrics(new AggregatorFactory[]{new CountAggregatorFactory("count")}) + .build(); IncrementalIndex index = new OnheapIncrementalIndex.Builder().setMaxRowCount(100).setIndexSchema(schema).build(); index.add( @@ -643,8 +639,8 @@ public class ExpressionSelectorsTest extends InitializedNullHandlingTest ) ); - IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { + IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) { Cursor cursor = cursorHolder.asCursor(); DimensionSelector xExprSelector = ExpressionSelectors.makeDimensionSelector( cursor.getColumnSelectorFactory(), diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsCastTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsCastTest.java index 6fbee67734f..e103413ae9c 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsCastTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsCastTest.java @@ -26,7 +26,7 @@ import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; @@ -116,11 +116,11 @@ public class ExpressionVectorSelectorsCastTest ) ) ); - final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(index); + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setVirtualColumns(virtualColumns) .build(); - VectorCursor cursor = closer.register(storageAdapter.makeCursorHolder(buildSpec)).asVectorCursor(); + VectorCursor cursor = closer.register(cursorFactory.makeCursorHolder(buildSpec)).asVectorCursor(); ColumnCapabilities capabilities = INDEX.getColumnCapabilities(column); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java index 5d284a74129..142230593db 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java @@ -40,7 +40,7 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DeprecatedQueryableIndexColumnSelector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.StringEncodingStrategy; @@ -234,15 +234,15 @@ public class ExpressionVectorSelectorsTest extends InitializedNullHandlingTest ) ) ); - final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(index); + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setVirtualColumns(virtualColumns) .build(); - try (final CursorHolder cursorHolder = storageAdapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final VectorCursor cursor = cursorHolder.asVectorCursor(); Assert.assertNotNull(cursor); - ColumnCapabilities capabilities = virtualColumns.getColumnCapabilitiesWithFallback(storageAdapter, "v"); + ColumnCapabilities capabilities = virtualColumns.getColumnCapabilitiesWithFallback(cursorFactory, "v"); int rowCount = 0; if (capabilities.isDictionaryEncoded().isTrue()) { diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumnTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumnTest.java index 581c8674da9..62f26588009 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumnTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumnTest.java @@ -86,11 +86,23 @@ public class NestedFieldVirtualColumnTest ); } + @Test + public void testEquivalence() + { + NestedFieldVirtualColumn v1 = new NestedFieldVirtualColumn("nested", "$.x.y.z", "v0", ColumnType.LONG); + NestedFieldVirtualColumn v2 = new NestedFieldVirtualColumn("nested", "$.x.y.z", "v1", ColumnType.LONG); + NestedFieldVirtualColumn v3 = new NestedFieldVirtualColumn("nested", "$.x.y.z[0]", "v0", ColumnType.LONG); + Assert.assertNotEquals(v1, v2); + Assert.assertEquals(v1.getEquivalanceKey(), v2.getEquivalanceKey()); + Assert.assertNotEquals(v1, v3); + Assert.assertNotEquals(v1.getEquivalanceKey(), v3.getEquivalanceKey()); + } + @Test public void testEqualsAndHashcode() { EqualsVerifier.forClass(NestedFieldVirtualColumn.class) - .withNonnullFields("columnName", "outputName") + .withNonnullFields("fieldSpec", "outputName") .withIgnoredFields("hasNegativeArrayIndex") .usingGetClass() .verify(); diff --git a/quidem-ut/pom.xml b/quidem-ut/pom.xml index dfcb65f6406..dd88d7fc84f 100644 --- a/quidem-ut/pom.xml +++ b/quidem-ut/pom.xml @@ -30,7 +30,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT diff --git a/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java b/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java index 9654b0dc23f..74149cefcb2 100644 --- a/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java +++ b/quidem-ut/src/main/java/org/apache/druid/quidem/ExposedAsBrokerQueryComponentSupplierWrapper.java @@ -336,4 +336,10 @@ public class ExposedAsBrokerQueryComponentSupplierWrapper implements QueryCompon } ); } + + @Override + public Boolean isExplainSupported() + { + return delegate.isExplainSupported(); + } } diff --git a/server/pom.xml b/server/pom.xml index 6e6e709bcf3..6cfeee55f57 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT @@ -140,8 +140,19 @@ jsr305 - io.tesla.aether - tesla-aether + org.apache.maven.resolver + maven-resolver-connector-basic + 1.3.1 + + + org.apache.maven.resolver + maven-resolver-transport-http + 1.3.1 + + + org.apache.maven + maven-resolver-provider + 3.6.0 com.amazonaws @@ -500,7 +511,9 @@ - io.tesla.aether:tesla-aether + org.apache.maven:maven-resolver-provider + org.apache.maven.resolver:maven-resolver-transport-http + org.apache.maven.resolver:maven-resolver-connector-basic org.xerial.snappy:snappy-java diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/HttpInputSourceDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/HttpInputSourceDefn.java index b4a3cf3425a..110974d18ca 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/HttpInputSourceDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/HttpInputSourceDefn.java @@ -19,6 +19,8 @@ package org.apache.druid.catalog.model.table; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableMap; import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.ColumnSpec; @@ -27,6 +29,7 @@ import org.apache.druid.catalog.model.table.TableFunction.ParameterDefn; import org.apache.druid.catalog.model.table.TableFunction.ParameterType; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.HttpInputSource; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.metadata.DefaultPasswordProvider; @@ -93,6 +96,7 @@ public class HttpInputSourceDefn extends FormattedInputSourceDefn public static final String PASSWORD_PARAMETER = "password"; public static final String PASSWORD_ENV_VAR_PARAMETER = "passwordEnvVar"; + public static final String HEADERS = "headers"; private static final List URI_PARAMS = Collections.singletonList( new Parameter(URIS_PARAMETER, ParameterType.VARCHAR_ARRAY, true) ); @@ -103,10 +107,15 @@ public class HttpInputSourceDefn extends FormattedInputSourceDefn new Parameter(PASSWORD_ENV_VAR_PARAMETER, ParameterType.VARCHAR, true) ); + private static final List HEADERS_PARAMS = Collections.singletonList( + new Parameter(HEADERS, ParameterType.VARCHAR, true) + ); + // Field names in the HttpInputSource protected static final String URIS_FIELD = "uris"; protected static final String PASSWORD_FIELD = "httpAuthenticationPassword"; protected static final String USERNAME_FIELD = "httpAuthenticationUsername"; + protected static final String HEADERS_FIELD = "requestHeaders"; @Override public String typeValue() @@ -201,7 +210,7 @@ public class HttpInputSourceDefn extends FormattedInputSourceDefn @Override protected List adHocTableFnParameters() { - return CatalogUtils.concatLists(URI_PARAMS, USER_PWD_PARAMS); + return CatalogUtils.concatLists(URI_PARAMS, CatalogUtils.concatLists(USER_PWD_PARAMS, HEADERS_PARAMS)); } @Override @@ -210,6 +219,7 @@ public class HttpInputSourceDefn extends FormattedInputSourceDefn jsonMap.put(InputSource.TYPE_PROPERTY, HttpInputSource.TYPE_KEY); convertUriArg(jsonMap, args); convertUserPasswordArgs(jsonMap, args); + convertHeaderArg(jsonMap, args); } @Override @@ -228,6 +238,10 @@ public class HttpInputSourceDefn extends FormattedInputSourceDefn params = CatalogUtils.concatLists(params, USER_PWD_PARAMS); } + if (!sourceMap.containsKey(HEADERS_FIELD)) { + params = CatalogUtils.concatLists(params, HEADERS_PARAMS); + } + // Does the table define a format? if (table.inputFormatMap == null) { params = addFormatParameters(params); @@ -255,6 +269,9 @@ public class HttpInputSourceDefn extends FormattedInputSourceDefn if (!sourceMap.containsKey(USERNAME_FIELD) && !sourceMap.containsKey(PASSWORD_FIELD)) { convertUserPasswordArgs(sourceMap, args); } + if (!sourceMap.containsKey(HEADERS_FIELD)) { + convertHeaderArg(sourceMap, args); + } return convertPartialFormattedTable(table, args, columns, sourceMap); } @@ -283,6 +300,26 @@ public class HttpInputSourceDefn extends FormattedInputSourceDefn } } + /** + * URIs in SQL is in the form of a string that contains a comma-delimited + * set of URIs. Done since SQL doesn't support array scalars. + */ + private void convertHeaderArg(Map jsonMap, Map args) + { + String requestHeaders = CatalogUtils.getString(args, HEADERS); + Map headersMap; + if (requestHeaders != null) { + try { + headersMap = DefaultObjectMapper.INSTANCE.readValue(requestHeaders, new TypeReference>(){}); + } + catch (JsonProcessingException e) { + throw new ISE("Failed read map from headers json"); + } + jsonMap.put(HEADERS_FIELD, headersMap); + } + + } + /** * Convert the user name and password. All are SQL strings. Passwords must be in * the form of a password provider, so do the needed conversion. HTTP provides diff --git a/server/src/main/java/org/apache/druid/guice/BrokerProcessingModule.java b/server/src/main/java/org/apache/druid/guice/BrokerProcessingModule.java index d70a2157e15..bc12e929219 100644 --- a/server/src/main/java/org/apache/druid/guice/BrokerProcessingModule.java +++ b/server/src/main/java/org/apache/druid/guice/BrokerProcessingModule.java @@ -20,16 +20,13 @@ package org.apache.druid.guice; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.ProvisionException; -import org.apache.druid.client.cache.BackgroundCachePopulator; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulator; import org.apache.druid.client.cache.CachePopulatorStats; -import org.apache.druid.client.cache.ForegroundCachePopulator; import org.apache.druid.collections.BlockingPool; import org.apache.druid.collections.DefaultBlockingPool; import org.apache.druid.collections.NonBlockingPool; @@ -43,25 +40,22 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.offheap.OffheapBufferGenerator; import org.apache.druid.query.BrokerParallelMergeConfig; import org.apache.druid.query.DruidProcessingConfig; -import org.apache.druid.query.ExecutorServiceMonitor; import org.apache.druid.query.ForwardingQueryProcessingPool; import org.apache.druid.query.QueryProcessingPool; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByResourcesReservationPool; -import org.apache.druid.server.metrics.MetricsModule; import org.apache.druid.utils.JvmUtils; import java.nio.ByteBuffer; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.ForkJoinPool; /** * This module is used to fulfill dependency injection of query processing and caching resources: buffer pools and * thread pools on Broker. Broker does not need to be allocated an intermediate results pool. * This is separated from DruidProcessingModule to separate the needs of the broker from the historicals + * + * @see DruidProcessingModule */ - public class BrokerProcessingModule implements Module { private static final Logger log = new Logger(BrokerProcessingModule.class); @@ -69,9 +63,8 @@ public class BrokerProcessingModule implements Module @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.processing.merge", BrokerParallelMergeConfig.class); - JsonConfigProvider.bind(binder, "druid.processing", DruidProcessingConfig.class); - MetricsModule.register(binder, ExecutorServiceMonitor.class); + JsonConfigProvider.bind(binder, DruidProcessingModule.PROCESSING_PROPERTY_PREFIX + ".merge", BrokerParallelMergeConfig.class); + DruidProcessingModule.registerConfigsAndMonitor(binder); } @Provides @@ -82,20 +75,7 @@ public class BrokerProcessingModule implements Module CacheConfig cacheConfig ) { - if (cacheConfig.getNumBackgroundThreads() > 0) { - final ExecutorService exec = Executors.newFixedThreadPool( - cacheConfig.getNumBackgroundThreads(), - new ThreadFactoryBuilder() - .setNameFormat("background-cacher-%d") - .setDaemon(true) - .setPriority(Thread.MIN_PRIORITY) - .build() - ); - - return new BackgroundCachePopulator(exec, smileMapper, cachePopulatorStats, cacheConfig.getMaxEntrySize()); - } else { - return new ForegroundCachePopulator(smileMapper, cachePopulatorStats, cacheConfig.getMaxEntrySize()); - } + return DruidProcessingModule.createCachePopulator(smileMapper, cachePopulatorStats, cacheConfig); } @Provides @@ -113,7 +93,6 @@ public class BrokerProcessingModule implements Module public NonBlockingPool getIntermediateResultsPool(DruidProcessingConfig config) { verifyDirectMemory(config); - return new StupidPool<>( "intermediate processing pool", new OffheapBufferGenerator("intermediate processing", config.intermediateComputeSizeBytes()), diff --git a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java index a2daa25e214..4879b5cd3c7 100644 --- a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java @@ -59,13 +59,14 @@ import java.util.concurrent.Executors; */ public class DruidProcessingModule implements Module { + public static final String PROCESSING_PROPERTY_PREFIX = "druid.processing"; + private static final Logger log = new Logger(DruidProcessingModule.class); @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.processing", DruidProcessingConfig.class); - MetricsModule.register(binder, ExecutorServiceMonitor.class); + registerConfigsAndMonitor(binder); } @Provides @@ -75,6 +76,59 @@ public class DruidProcessingModule implements Module CachePopulatorStats cachePopulatorStats, CacheConfig cacheConfig ) + { + return createCachePopulator(smileMapper, cachePopulatorStats, cacheConfig); + } + + @Provides + @ManageLifecycle + public QueryProcessingPool getProcessingExecutorPool( + DruidProcessingConfig config, + ExecutorServiceMonitor executorServiceMonitor, + Lifecycle lifecycle + ) + { + return createProcessingExecutorPool(config, executorServiceMonitor, lifecycle); + } + + @Provides + @LazySingleton + @Global + public NonBlockingPool getIntermediateResultsPool(DruidProcessingConfig config) + { + return createIntermediateResultsPool(config); + } + + @Provides + @LazySingleton + @Merging + public BlockingPool getMergeBufferPool(DruidProcessingConfig config) + { + return createMergeBufferPool(config); + } + + @Provides + @LazySingleton + @Merging + public GroupByResourcesReservationPool getGroupByResourcesReservationPool( + @Merging BlockingPool mergeBufferPool, + GroupByQueryConfig groupByQueryConfig + ) + { + return new GroupByResourcesReservationPool(mergeBufferPool, groupByQueryConfig); + } + + public static void registerConfigsAndMonitor(Binder binder) + { + JsonConfigProvider.bind(binder, PROCESSING_PROPERTY_PREFIX, DruidProcessingConfig.class); + MetricsModule.register(binder, ExecutorServiceMonitor.class); + } + + public static CachePopulator createCachePopulator( + ObjectMapper smileMapper, + CachePopulatorStats cachePopulatorStats, + CacheConfig cacheConfig + ) { if (cacheConfig.getNumBackgroundThreads() > 0) { final ExecutorService exec = Executors.newFixedThreadPool( @@ -92,9 +146,7 @@ public class DruidProcessingModule implements Module } } - @Provides - @ManageLifecycle - public QueryProcessingPool getProcessingExecutorPool( + public static QueryProcessingPool createProcessingExecutorPool( DruidProcessingConfig config, ExecutorServiceMonitor executorServiceMonitor, Lifecycle lifecycle @@ -109,10 +161,7 @@ public class DruidProcessingModule implements Module ); } - @Provides - @LazySingleton - @Global - public NonBlockingPool getIntermediateResultsPool(DruidProcessingConfig config) + public static NonBlockingPool createIntermediateResultsPool(final DruidProcessingConfig config) { verifyDirectMemory(config); return new StupidPool<>( @@ -123,10 +172,7 @@ public class DruidProcessingModule implements Module ); } - @Provides - @LazySingleton - @Merging - public BlockingPool getMergeBufferPool(DruidProcessingConfig config) + public static BlockingPool createMergeBufferPool(final DruidProcessingConfig config) { verifyDirectMemory(config); return new DefaultBlockingPool<>( @@ -135,18 +181,7 @@ public class DruidProcessingModule implements Module ); } - @Provides - @LazySingleton - @Merging - public GroupByResourcesReservationPool getGroupByResourcesReservationPool( - @Merging BlockingPool mergeBufferPool, - GroupByQueryConfig groupByQueryConfig - ) - { - return new GroupByResourcesReservationPool(mergeBufferPool, groupByQueryConfig); - } - - private void verifyDirectMemory(DruidProcessingConfig config) + private static void verifyDirectMemory(DruidProcessingConfig config) { try { final long maxDirectMemory = JvmUtils.getRuntimeInfo().getDirectMemorySizeBytes(); diff --git a/server/src/main/java/org/apache/druid/guice/RouterProcessingModule.java b/server/src/main/java/org/apache/druid/guice/RouterProcessingModule.java index f76b5ed940d..85357a7fa04 100644 --- a/server/src/main/java/org/apache/druid/guice/RouterProcessingModule.java +++ b/server/src/main/java/org/apache/druid/guice/RouterProcessingModule.java @@ -28,15 +28,12 @@ import org.apache.druid.collections.DummyNonBlockingPool; import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.guice.annotations.Global; import org.apache.druid.guice.annotations.Merging; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.DruidProcessingConfig; -import org.apache.druid.query.ExecutorServiceMonitor; -import org.apache.druid.query.ForwardingQueryProcessingPool; +import org.apache.druid.query.NoopQueryProcessingPool; import org.apache.druid.query.QueryProcessingPool; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByResourcesReservationPool; -import org.apache.druid.server.metrics.MetricsModule; import java.nio.ByteBuffer; @@ -46,6 +43,8 @@ import java.nio.ByteBuffer; * {@link org.apache.druid.query.QueryToolChest}s, and they couple query type aspects not related to processing and * caching, which Router uses, and related to processing and caching, which Router doesn't use, but they inject the * resources. + * + * @see DruidProcessingModule */ public class RouterProcessingModule implements Module { @@ -54,8 +53,7 @@ public class RouterProcessingModule implements Module @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.processing", DruidProcessingConfig.class); - MetricsModule.register(binder, ExecutorServiceMonitor.class); + DruidProcessingModule.registerConfigsAndMonitor(binder); } @Provides @@ -65,7 +63,7 @@ public class RouterProcessingModule implements Module if (config.isNumThreadsConfigured()) { log.warn("numThreads[%d] configured, that is ignored on Router", config.getNumThreads()); } - return new ForwardingQueryProcessingPool(Execs.dummy()); + return NoopQueryProcessingPool.instance(); } @Provides diff --git a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java index bda88401881..22ee4ec4102 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java @@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Multiset; @@ -64,6 +63,17 @@ import java.util.stream.Collectors; public class DataSchema { private static final Logger log = new Logger(DataSchema.class); + + public static Builder builder() + { + return new Builder(); + } + + public static Builder builder(DataSchema schema) + { + return new Builder(schema); + } + private final String dataSource; private final AggregatorFactory[] aggregators; private final GranularitySpec granularitySpec; @@ -126,33 +136,6 @@ public class DataSchema } } - @VisibleForTesting - public DataSchema( - String dataSource, - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, - AggregatorFactory[] aggregators, - GranularitySpec granularitySpec, - TransformSpec transformSpec - ) - { - this(dataSource, timestampSpec, dimensionsSpec, aggregators, granularitySpec, transformSpec, null, null); - } - - // old constructor for backward compatibility - @Deprecated - public DataSchema( - String dataSource, - Map parserMap, - AggregatorFactory[] aggregators, - GranularitySpec granularitySpec, - TransformSpec transformSpec, - ObjectMapper objectMapper - ) - { - this(dataSource, null, null, aggregators, granularitySpec, transformSpec, parserMap, objectMapper); - } - private static void validateDatasourceName(String dataSource) { IdUtils.validateId("dataSource", dataSource); @@ -403,44 +386,17 @@ public class DataSchema public DataSchema withGranularitySpec(GranularitySpec granularitySpec) { - return new DataSchema( - dataSource, - timestampSpec, - dimensionsSpec, - aggregators, - granularitySpec, - transformSpec, - parserMap, - objectMapper - ); + return builder(this).withGranularity(granularitySpec).build(); } public DataSchema withTransformSpec(TransformSpec transformSpec) { - return new DataSchema( - dataSource, - timestampSpec, - dimensionsSpec, - aggregators, - granularitySpec, - transformSpec, - parserMap, - objectMapper - ); + return builder(this).withTransform(transformSpec).build(); } public DataSchema withDimensionsSpec(DimensionsSpec dimensionsSpec) { - return new DataSchema( - dataSource, - timestampSpec, - dimensionsSpec, - aggregators, - granularitySpec, - transformSpec, - parserMap, - objectMapper - ); + return builder(this).withDimensions(dimensionsSpec).build(); } @Override @@ -457,4 +413,110 @@ public class DataSchema ", inputRowParser=" + inputRowParser + '}'; } + + public static class Builder + { + private String dataSource; + private AggregatorFactory[] aggregators; + private GranularitySpec granularitySpec; + private TransformSpec transformSpec; + private Map parserMap; + private ObjectMapper objectMapper; + + // The below fields can be initialized lazily from parser for backward compatibility. + private TimestampSpec timestampSpec; + private DimensionsSpec dimensionsSpec; + + public Builder() + { + + } + + public Builder(DataSchema schema) + { + this.dataSource = schema.dataSource; + this.aggregators = schema.aggregators; + this.granularitySpec = schema.granularitySpec; + this.transformSpec = schema.transformSpec; + this.parserMap = schema.parserMap; + this.objectMapper = schema.objectMapper; + this.timestampSpec = schema.timestampSpec; + this.dimensionsSpec = schema.dimensionsSpec; + } + + public Builder withDataSource(String dataSource) + { + this.dataSource = dataSource; + return this; + } + + public Builder withTimestamp(TimestampSpec timestampSpec) + { + this.timestampSpec = timestampSpec; + return this; + } + + public Builder withDimensions(DimensionsSpec dimensionsSpec) + { + this.dimensionsSpec = dimensionsSpec; + return this; + } + + public Builder withDimensions(List dimensions) + { + this.dimensionsSpec = DimensionsSpec.builder().setDimensions(dimensions).build(); + return this; + } + + public Builder withDimensions(DimensionSchema... dimensions) + { + return withDimensions(Arrays.asList(dimensions)); + } + + public Builder withAggregators(AggregatorFactory... aggregators) + { + this.aggregators = aggregators; + return this; + } + + public Builder withGranularity(GranularitySpec granularitySpec) + { + this.granularitySpec = granularitySpec; + return this; + } + + public Builder withTransform(TransformSpec transformSpec) + { + this.transformSpec = transformSpec; + return this; + } + + @Deprecated + public Builder withObjectMapper(ObjectMapper objectMapper) + { + this.objectMapper = objectMapper; + return this; + } + + @Deprecated + public Builder withParserMap(Map parserMap) + { + this.parserMap = parserMap; + return this; + } + + public DataSchema build() + { + return new DataSchema( + dataSource, + timestampSpec, + dimensionsSpec, + aggregators, + granularitySpec, + transformSpec, + parserMap, + objectMapper + ); + } + } } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java index d918ec5e3f2..99d965ec643 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java @@ -102,6 +102,13 @@ import java.util.stream.StreamSupport; *

    * This class has an abstract method {@link #refresh(Set, Set)} which the child class must override * with the logic to build and cache table schema. + *

    + * Note on handling tombstone segments: + * These segments lack data or column information. + * Additionally, segment metadata queries, which are not yet implemented for tombstone segments + * (see: https://github.com/apache/druid/pull/12137) do not provide metadata for tombstones, + * leading to indefinite refresh attempts for these segments. + * Therefore, these segments are never added to the set of segments being refreshed. * * @param The type of information associated with the data source, which must extend {@link DataSourceInformation}. */ @@ -478,13 +485,6 @@ public abstract class AbstractSegmentMetadataCache columnTypes = new LinkedHashMap<>(); if (segmentsMap != null && !segmentsMap.isEmpty()) { - for (SegmentId segmentId : segmentsMap.keySet()) { + for (Map.Entry entry : segmentsMap.entrySet()) { + SegmentId segmentId = entry.getKey(); Optional optionalSchema = segmentSchemaCache.getSchemaForSegment(segmentId); if (optionalSchema.isPresent()) { RowSignature rowSignature = optionalSchema.get().getSchemaPayload().getRowSignature(); mergeRowSignature(columnTypes, rowSignature); } else { - // mark it for refresh, however, this case shouldn't arise by design - markSegmentAsNeedRefresh(segmentId); - log.debug("SchemaMetadata for segmentId [%s] is absent.", segmentId); + markSegmentForRefreshIfNeeded(entry.getValue().getSegment()); } } } else { @@ -869,4 +864,32 @@ public class CoordinatorSegmentMetadataCache extends AbstractSegmentMetadataCach return Optional.empty(); } } + + /** + * A segment schema can go missing. To ensure smooth functioning, segment is marked for refresh. + * It need not be refreshed in the following scenarios: + * - Tombstone segment, since they do not have any schema. + * - Unused segment which hasn't been yet removed from the cache. + * Any other scenario needs investigation. + */ + private void markSegmentForRefreshIfNeeded(DataSegment segment) + { + SegmentId id = segment.getId(); + + log.debug("SchemaMetadata for segmentId [%s] is absent.", id); + + if (segment.isTombstone()) { + log.debug("Skipping refresh for tombstone segment [%s].", id); + return; + } + + ImmutableDruidDataSource druidDataSource = + sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(segment.getDataSource()); + + if (druidDataSource != null && druidDataSource.getSegment(id) != null) { + markSegmentAsNeedRefresh(id); + } else { + log.debug("Skipping refresh for unused segment [%s].", id); + } + } } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/FingerprintGenerator.java b/server/src/main/java/org/apache/druid/segment/metadata/FingerprintGenerator.java index f1cb8ea0a50..7d35b180c62 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/FingerprintGenerator.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/FingerprintGenerator.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.metadata; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.common.hash.Hasher; import com.google.common.hash.Hashing; import com.google.common.io.BaseEncoding; @@ -30,11 +31,17 @@ import org.apache.druid.guice.LazySingleton; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.SchemaPayload; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; /** - * Utility to generate fingerprint for an object. + * Utility to generate schema fingerprint which is used to ensure schema uniqueness in the metadata database. + * Note, that the generated fingerprint is independent of the column order. */ @LazySingleton public class FingerprintGenerator @@ -53,12 +60,20 @@ public class FingerprintGenerator * Generates fingerprint or hash string for an object using SHA-256 hash algorithm. */ @SuppressWarnings("UnstableApiUsage") - public String generateFingerprint(SchemaPayload schemaPayload, String dataSource, int version) + public String generateFingerprint(final SchemaPayload schemaPayload, final String dataSource, final int version) { + // Sort the column names in lexicographic order + // The aggregator factories are column order independent since they are stored in a hashmap + // This ensures that all permutations of a given columns would result in the same fingerprint + // thus avoiding schema explosion in the metadata database + // Note that this signature is not persisted anywhere, it is only used for fingerprint computation + final RowSignature sortedSignature = getLexicographicallySortedSignature(schemaPayload.getRowSignature()); + final SchemaPayload updatedPayload = new SchemaPayload(sortedSignature, schemaPayload.getAggregatorFactories()); try { + final Hasher hasher = Hashing.sha256().newHasher(); - hasher.putBytes(objectMapper.writeValueAsBytes(schemaPayload)); + hasher.putBytes(objectMapper.writeValueAsBytes(updatedPayload)); // add delimiter, inspired from org.apache.druid.metadata.PendingSegmentRecord.computeSequenceNamePrevIdSha1 hasher.putByte((byte) 0xff); @@ -82,4 +97,21 @@ public class FingerprintGenerator ); } } + + @VisibleForTesting + protected RowSignature getLexicographicallySortedSignature(final RowSignature rowSignature) + { + final List columns = new ArrayList<>(rowSignature.getColumnNames()); + + Collections.sort(columns); + + final RowSignature.Builder sortedSignature = RowSignature.builder(); + + for (String column : columns) { + ColumnType type = rowSignature.getColumnType(column).orElse(null); + sortedSignature.add(column, type); + } + + return sortedSignature.build(); + } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java b/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java index b051fa72724..e9f6e3a2ade 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/FireHydrant.java @@ -20,14 +20,15 @@ package org.apache.druid.segment.realtime; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Iterables; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.segment.IncrementalIndexSegment; +import org.apache.druid.segment.Metadata; +import org.apache.druid.segment.PhysicalSegmentInspector; +import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -76,10 +77,16 @@ public class FireHydrant public int getSegmentNumDimensionColumns() { - final Segment segment = adapter.get().getBaseSegment(); - if (segment != null) { - final StorageAdapter storageAdapter = segment.asStorageAdapter(); - return storageAdapter.getAvailableDimensions().size(); + if (hasSwapped()) { + final Segment segment = adapter.get().getBaseSegment(); + if (segment != null) { + QueryableIndex queryableIndex = segment.as(QueryableIndex.class); + if (queryableIndex != null) { + return queryableIndex.getAvailableDimensions().size(); + } + } + } else { + return index.getDimensions().size(); } return 0; } @@ -88,8 +95,9 @@ public class FireHydrant { final Segment segment = adapter.get().getBaseSegment(); if (segment != null) { - final StorageAdapter storageAdapter = segment.asStorageAdapter(); - return Iterables.size(storageAdapter.getAvailableMetrics()); + final PhysicalSegmentInspector segmentInspector = segment.as(PhysicalSegmentInspector.class); + final Metadata metadata = segmentInspector == null ? null : segmentInspector.getMetadata(); + return metadata != null && metadata.getAggregators() != null ? metadata.getAggregators().length : 0; } return 0; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/WindowedStorageAdapter.java b/server/src/main/java/org/apache/druid/segment/realtime/WindowedCursorFactory.java similarity index 74% rename from server/src/main/java/org/apache/druid/segment/realtime/WindowedStorageAdapter.java rename to server/src/main/java/org/apache/druid/segment/realtime/WindowedCursorFactory.java index 9b4bd1e4ea7..63eb32a0253 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/WindowedStorageAdapter.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/WindowedCursorFactory.java @@ -19,23 +19,23 @@ package org.apache.druid.segment.realtime; -import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.CursorFactory; import org.joda.time.Interval; -public class WindowedStorageAdapter +public class WindowedCursorFactory { - private final StorageAdapter adapter; + private final CursorFactory cursorFactory; private final Interval interval; - public WindowedStorageAdapter(StorageAdapter adapter, Interval interval) + public WindowedCursorFactory(CursorFactory cursorFactory, Interval interval) { - this.adapter = adapter; + this.cursorFactory = cursorFactory; this.interval = interval; } - public StorageAdapter getAdapter() + public CursorFactory getCursorFactory() { - return adapter; + return cursorFactory; } public Interval getInterval() @@ -46,8 +46,8 @@ public class WindowedStorageAdapter @Override public String toString() { - return "WindowedStorageAdapter{" + - "adapter=" + adapter + + return "WindowedCursorFactory{" + + "cursorFactory=" + cursorFactory + ", interval=" + interval + '}'; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index 979d8cb92b3..bb5acec7b8f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -816,7 +816,7 @@ public class BatchAppenderator implements Appenderator try { for (FireHydrant fireHydrant : sink) { Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); - final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex(); + final QueryableIndex queryableIndex = segmentAndCloseable.lhs.as(QueryableIndex.class); if (queryableIndex != null) { rowsinMergedSegment += queryableIndex.getNumRows(); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index a25d6b7acd7..0d7d01253c8 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -934,7 +934,7 @@ public class StreamAppenderator implements Appenderator try { for (FireHydrant fireHydrant : sink) { Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); - final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex(); + final QueryableIndex queryableIndex = segmentAndCloseable.lhs.as(QueryableIndex.class); log.debug("Segment[%s] adding hydrant[%s]", identifier, fireHydrant); indexes.add(queryableIndex); closer.register(segmentAndCloseable.rhs); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TaskSegmentSchemaUtil.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TaskSegmentSchemaUtil.java index 8897390a746..980f9bf4d8e 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TaskSegmentSchemaUtil.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TaskSegmentSchemaUtil.java @@ -22,10 +22,9 @@ package org.apache.druid.segment.realtime.appenderator; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.SchemaPayload; import org.apache.druid.segment.SchemaPayloadPlus; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.RowSignature; import java.io.File; @@ -41,10 +40,9 @@ public class TaskSegmentSchemaUtil public static SchemaPayloadPlus getSegmentSchema(File segmentFile, IndexIO indexIO) throws IOException { final QueryableIndex queryableIndex = indexIO.loadIndex(segmentFile); - final StorageAdapter storageAdapter = new QueryableIndexStorageAdapter(queryableIndex); - final RowSignature rowSignature = storageAdapter.getRowSignature(); - final long numRows = storageAdapter.getNumRows(); - final AggregatorFactory[] aggregatorFactories = storageAdapter.getMetadata().getAggregators(); + final RowSignature rowSignature = new QueryableIndexCursorFactory(queryableIndex).getRowSignature(); + final long numRows = queryableIndex.getNumRows(); + final AggregatorFactory[] aggregatorFactories = queryableIndex.getMetadata().getAggregators(); Map aggregatorFactoryMap = new HashMap<>(); if (null != aggregatorFactories) { for (AggregatorFactory aggregatorFactory : aggregatorFactories) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java index 5e59c0e9b15..98c5e11fa0b 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.realtime.sink; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterators; @@ -33,10 +34,9 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.Query; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnFormat; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -44,7 +44,6 @@ import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.realtime.FireHydrant; @@ -157,8 +156,8 @@ public class Sink implements Iterable, Overshadowable maxCount = hydrant.getCount(); ReferenceCountingSegment segment = hydrant.getIncrementedSegment(); try { - QueryableIndex index = segment.asQueryableIndex(); - overwriteIndexDimensions(new QueryableIndexStorageAdapter(index)); + overwriteIndexDimensions(segment); + QueryableIndex index = segment.as(QueryableIndex.class); numRowsExcludingCurrIndex.addAndGet(index.getNumRows()); } finally { @@ -287,7 +286,7 @@ public class Sink implements Iterable, Overshadowable return 0; } - return currHydrant.getIndex().size(); + return index.size(); } } @@ -299,7 +298,7 @@ public class Sink implements Iterable, Overshadowable return 0; } - return currHydrant.getIndex().getBytesInMemory().get(); + return index.getBytesInMemory().get(); } } @@ -354,11 +353,11 @@ public class Sink implements Iterable, Overshadowable if (lastHydrant.hasSwapped()) { oldFormat = new HashMap<>(); - ReferenceCountingSegment segment = lastHydrant.getIncrementedSegment(); + final ReferenceCountingSegment segment = lastHydrant.getIncrementedSegment(); try { - QueryableIndex oldIndex = segment.asQueryableIndex(); - overwriteIndexDimensions(new QueryableIndexStorageAdapter(oldIndex)); + overwriteIndexDimensions(segment); if (variableDimensions) { + final QueryableIndex oldIndex = Preconditions.checkNotNull(segment.as(QueryableIndex.class)); for (String dim : oldIndex.getAvailableDimensions()) { dimOrder.add(dim); oldFormat.put(dim, oldIndex.getColumnHolder(dim).getColumnFormat()); @@ -369,9 +368,9 @@ public class Sink implements Iterable, Overshadowable segment.decrement(); } } else { - IncrementalIndex oldIndex = lastHydrant.getIndex(); - overwriteIndexDimensions(new IncrementalIndexStorageAdapter(oldIndex)); + overwriteIndexDimensions(lastHydrant.getHydrantSegment()); if (variableDimensions) { + IncrementalIndex oldIndex = lastHydrant.getIndex(); dimOrder.addAll(oldIndex.getDimensionOrder()); oldFormat = oldIndex.getColumnFormats(); } @@ -399,9 +398,9 @@ public class Sink implements Iterable, Overshadowable * Merge the column from the index with the existing columns. */ @GuardedBy("hydrantLock") - private void overwriteIndexDimensions(StorageAdapter storageAdapter) + private void overwriteIndexDimensions(Segment segment) { - RowSignature rowSignature = storageAdapter.getRowSignature(); + RowSignature rowSignature = segment.asCursorFactory().getRowSignature(); for (String dim : rowSignature.getColumnNames()) { columnsExcludingCurrIndex.add(dim); rowSignature.getColumnType(dim).ifPresent(type -> columnTypeExcludingCurrIndex.put(dim, type)); @@ -422,9 +421,7 @@ public class Sink implements Iterable, Overshadowable } // Add columns from the currHydrant that do not yet exist in columnsExcludingCurrIndex. - IncrementalIndexStorageAdapter currStorageAdapter = - new IncrementalIndexStorageAdapter(currHydrant.getIndex()); - RowSignature currSignature = currStorageAdapter.getRowSignature(); + RowSignature currSignature = currHydrant.getHydrantSegment().asCursorFactory().getRowSignature(); for (String dim : currSignature.getColumnNames()) { if (!columnsExcludingCurrIndex.contains(dim)) { diff --git a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java index 990878eda6e..37ae14f56c3 100644 --- a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java @@ -56,6 +56,7 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.query.ResultLevelCachingQueryRunner; +import org.apache.druid.query.ResultSerializationMode; import org.apache.druid.query.RetryQueryRunner; import org.apache.druid.query.RetryQueryRunnerConfig; import org.apache.druid.query.SegmentDescriptor; @@ -359,7 +360,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker * @param dryRun if true, does not actually execute any subqueries, but will inline empty result sets. */ @SuppressWarnings({"rawtypes", "unchecked"}) // Subquery, toolchest, runner handling all use raw types - private DataSource inlineIfNecessary( + private DataSource inlineIfNecessary( final DataSource dataSource, @Nullable final QueryToolChest toolChestIfOutermost, final AtomicInteger subqueryRowLimitAccumulator, @@ -434,11 +435,17 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker if (dryRun) { queryResults = Sequences.empty(); } else { - final QueryRunner subqueryRunner = subQuery.getRunner(this); - queryResults = subqueryRunner.run( - QueryPlus.wrap(subQuery), - DirectDruidClient.makeResponseContextForQuery() + Query subQueryWithSerialization = subQuery.withOverriddenContext( + Collections.singletonMap( + ResultSerializationMode.CTX_SERIALIZATION_PARAMETER, + ClientQuerySegmentWalkerUtils.getLimitType(maxSubqueryMemory, cannotMaterializeToFrames.get()) + .serializationMode() + .toString() + ) ); + queryResults = subQueryWithSerialization + .getRunner(this) + .run(QueryPlus.wrap(subQueryWithSerialization), DirectDruidClient.makeResponseContextForQuery()); } return toInlineDataSource( @@ -647,14 +654,11 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker .collect(Collectors.toList())); } - /** - */ /** * * Convert the results of a particular query into a materialized (List-based) InlineDataSource. * * @param query the query - * @param results query results * @param toolChest toolchest for the query * @param limitAccumulator an accumulator for tracking the number of accumulated rows in all subqueries for a * particular master query @@ -671,7 +675,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker */ private static > DataSource toInlineDataSource( final QueryType query, - final Sequence results, + final Sequence queryResults, final QueryToolChest toolChest, final AtomicInteger limitAccumulator, final AtomicLong memoryLimitAccumulator, @@ -697,7 +701,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker subqueryStatsProvider.incrementSubqueriesWithRowLimit(); dataSource = materializeResultsAsArray( query, - results, + queryResults, toolChest, limitAccumulator, limit, @@ -713,7 +717,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker } Optional maybeDataSource = materializeResultsAsFrames( query, - results, + queryResults, toolChest, limitAccumulator, memoryLimitAccumulator, @@ -734,7 +738,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker subqueryStatsProvider.incrementSubqueriesFallingBackToRowLimit(); dataSource = materializeResultsAsArray( query, - results, + queryResults, toolChest, limitAccumulator, limit, @@ -770,11 +774,9 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker final ServiceEmitter emitter ) { - Optional> framesOptional; - boolean startedAccumulating = false; try { - framesOptional = toolChest.resultsAsFrames( + Optional> framesOptional = toolChest.resultsAsFrames( query, results, new ArenaMemoryAllocatorFactory(FRAME_SIZE), @@ -912,5 +914,4 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker QueryContexts.MAX_SUBQUERY_ROWS_KEY ); } - } diff --git a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalkerUtils.java b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalkerUtils.java index 6667cd96112..0435ed23193 100644 --- a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalkerUtils.java +++ b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalkerUtils.java @@ -19,6 +19,8 @@ package org.apache.druid.server; +import org.apache.druid.query.ResultSerializationMode; + /** * Utilities for {@link ClientQuerySegmentWalker} */ @@ -35,7 +37,13 @@ public class ClientQuerySegmentWalkerUtils * walker ensures that the cumulative number of rows of the results of subqueries of the given query donot exceed * the limit specified in the context or as the server default */ - ROW_LIMIT, + ROW_LIMIT { + @Override + public ResultSerializationMode serializationMode() + { + return ResultSerializationMode.ROWS; + } + }, /** * Subqueries limited by the BYTE_LIMIT are materialized as {@link org.apache.druid.frame.Frame}s on heap. Frames @@ -44,10 +52,18 @@ public class ClientQuerySegmentWalkerUtils * Frames in the broker memory) of a given query do not exceed the limit specified in the context or as the server * default */ - MEMORY_LIMIT + MEMORY_LIMIT { + @Override + public ResultSerializationMode serializationMode() + { + return ResultSerializationMode.FRAMES; + } + }; + + public abstract ResultSerializationMode serializationMode(); } - /** + /** * Returns the limit type to be used for a given subquery. * It returns MEMORY_LIMIT only if: * 1. The user has enabled the 'maxSubqueryBytes' explicitly in the query context or as the server default diff --git a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java index e0bb9875240..a91959ca20b 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLifecycle.java +++ b/server/src/main/java/org/apache/druid/server/QueryLifecycle.java @@ -19,7 +19,7 @@ package org.apache.druid.server; -import com.fasterxml.jackson.databind.ObjectWriter; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Iterables; @@ -62,7 +62,6 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; - import java.util.HashSet; import java.util.LinkedHashMap; import java.util.Map; @@ -434,7 +433,7 @@ public class QueryLifecycle || (!shouldFinalize && queryContext.isSerializeDateTimeAsLongInner(false)); } - public ObjectWriter newOutputWriter(ResourceIOReaderWriter ioReaderWriter) + public ObjectMapper newOutputWriter(ResourceIOReaderWriter ioReaderWriter) { return ioReaderWriter.getResponseWriter().newOutputWriter( getToolChest(), diff --git a/server/src/main/java/org/apache/druid/server/QueryResource.java b/server/src/main/java/org/apache/druid/server/QueryResource.java index 2db205ca0be..61696dd5cec 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResource.java +++ b/server/src/main/java/org/apache/druid/server/QueryResource.java @@ -19,11 +19,12 @@ package org.apache.druid.server; +import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParseException; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.ObjectWriter; -import com.fasterxml.jackson.databind.SequenceWriter; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.databind.SerializerProvider; import com.fasterxml.jackson.databind.module.SimpleModule; import com.fasterxml.jackson.datatype.joda.ser.DateTimeSerializer; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; @@ -37,6 +38,7 @@ import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Self; import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.BadJsonQueryException; import org.apache.druid.query.Query; @@ -374,7 +376,7 @@ public class QueryResource implements QueryCountStatsProvider return responseType; } - ObjectWriter newOutputWriter( + ObjectMapper newOutputWriter( @Nullable QueryToolChest> toolChest, @Nullable Query query, boolean serializeDateTimeAsLong @@ -387,7 +389,7 @@ public class QueryResource implements QueryCountStatsProvider } else { decoratedMapper = mapper; } - return isPretty ? decoratedMapper.writerWithDefaultPrettyPrinter() : decoratedMapper.writer(); + return isPretty ? decoratedMapper.copy().enable(SerializationFeature.INDENT_OUTPUT) : decoratedMapper; } Response ok(Object object) throws IOException @@ -531,35 +533,7 @@ public class QueryResource implements QueryCountStatsProvider @Override public Writer makeWriter(OutputStream out) throws IOException { - final ObjectWriter objectWriter = queryLifecycle.newOutputWriter(io); - final SequenceWriter sequenceWriter = objectWriter.writeValuesAsArray(out); - return new Writer() - { - - @Override - public void writeResponseStart() - { - // Do nothing - } - - @Override - public void writeRow(Object obj) throws IOException - { - sequenceWriter.write(obj); - } - - @Override - public void writeResponseEnd() - { - // Do nothing - } - - @Override - public void close() throws IOException - { - sequenceWriter.close(); - } - }; + return new NativeQueryWriter(queryLifecycle.newOutputWriter(io), out); } @Override @@ -585,8 +559,49 @@ public class QueryResource implements QueryCountStatsProvider @Override public void writeException(Exception e, OutputStream out) throws IOException { - final ObjectWriter objectWriter = queryLifecycle.newOutputWriter(io); - out.write(objectWriter.writeValueAsBytes(e)); + final ObjectMapper objectMapper = queryLifecycle.newOutputWriter(io); + out.write(objectMapper.writeValueAsBytes(e)); + } + } + + static class NativeQueryWriter implements QueryResultPusher.Writer + { + private final SerializerProvider serializers; + private final JsonGenerator jsonGenerator; + + public NativeQueryWriter(final ObjectMapper responseMapper, final OutputStream out) throws IOException + { + // Don't use objectWriter.writeValuesAsArray(out), because that causes an end array ] to be written when the + // writer is closed, even if it's closed in case of an exception. This causes valid JSON to be emitted in case + // of an exception, which makes it difficult for callers to detect problems. Note: this means that if an error + // occurs on a Historical (or other data server) after it started to push results to the Broker, the Broker + // will experience that as "JsonEOFException: Unexpected end-of-input: expected close marker for Array". + this.serializers = responseMapper.getSerializerProviderInstance(); + this.jsonGenerator = responseMapper.createGenerator(out); + } + + @Override + public void writeResponseStart() throws IOException + { + jsonGenerator.writeStartArray(); + } + + @Override + public void writeRow(Object obj) throws IOException + { + JacksonUtils.writeObjectUsingSerializerProvider(jsonGenerator, serializers, obj); + } + + @Override + public void writeResponseEnd() throws IOException + { + jsonGenerator.writeEndArray(); + } + + @Override + public void close() throws IOException + { + jsonGenerator.close(); } } } diff --git a/server/src/main/java/org/apache/druid/server/SegmentManager.java b/server/src/main/java/org/apache/druid/server/SegmentManager.java index 672e4c23aa8..baf1c298348 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -29,9 +29,9 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.segment.PhysicalSegmentInspector; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.join.table.IndexedTable; import org.apache.druid.segment.join.table.ReferenceCountingIndexedTable; import org.apache.druid.segment.loading.SegmentCacheManager; @@ -343,8 +343,13 @@ public class SegmentManager dataSegment.getVersion(), dataSegment.getShardSpec().createChunk(segment) ); - final StorageAdapter storageAdapter = segment.asStorageAdapter(); - final long numOfRows = (dataSegment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows(); + final PhysicalSegmentInspector countInspector = segment.as(PhysicalSegmentInspector.class); + final long numOfRows; + if (dataSegment.isTombstone() || countInspector == null) { + numOfRows = 0; + } else { + numOfRows = countInspector.getNumRows(); + } dataSourceState.addSegment(dataSegment, numOfRows); pageCacheLoadFunction.accept(dataSegment); @@ -387,8 +392,13 @@ public class SegmentManager if (oldQueryable != null) { try (final Closer closer = Closer.create()) { - StorageAdapter storageAdapter = oldQueryable.asStorageAdapter(); - long numOfRows = (segment.isTombstone() || storageAdapter == null) ? 0 : storageAdapter.getNumRows(); + final PhysicalSegmentInspector countInspector = oldQueryable.as(PhysicalSegmentInspector.class); + final long numOfRows; + if (segment.isTombstone() || countInspector == null) { + numOfRows = 0; + } else { + numOfRows = countInspector.getNumRows(); + } dataSourceState.removeSegment(segment, numOfRows); closer.register(oldQueryable); diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index 9e9a199917a..ffbdd44bce6 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java @@ -223,11 +223,27 @@ public class CompactionStatus partitionsSpecFromTuningConfig.getMaxRowsPerSegment(), ((DynamicPartitionsSpec) partitionsSpecFromTuningConfig).getMaxTotalRowsOr(Long.MAX_VALUE) ); + } else if (partitionsSpecFromTuningConfig instanceof DimensionRangePartitionsSpec) { + return getEffectiveRangePartitionsSpec((DimensionRangePartitionsSpec) partitionsSpecFromTuningConfig); } else { return partitionsSpecFromTuningConfig; } } + /** + * Converts to have only the effective maxRowsPerSegment to avoid false positives when targetRowsPerSegment is set but + * effectively translates to the same maxRowsPerSegment. + */ + static DimensionRangePartitionsSpec getEffectiveRangePartitionsSpec(DimensionRangePartitionsSpec partitionsSpec) + { + return new DimensionRangePartitionsSpec( + null, + partitionsSpec.getMaxRowsPerSegment(), + partitionsSpec.getPartitionDimensions(), + partitionsSpec.isAssumeGrouped() + ); + } + /** * Evaluates {@link #CHECKS} to determine the compaction status. */ @@ -286,10 +302,14 @@ public class CompactionStatus private CompactionStatus partitionsSpecIsUpToDate() { + PartitionsSpec existingPartionsSpec = lastCompactionState.getPartitionsSpec(); + if (existingPartionsSpec instanceof DimensionRangePartitionsSpec) { + existingPartionsSpec = getEffectiveRangePartitionsSpec((DimensionRangePartitionsSpec) existingPartionsSpec); + } return CompactionStatus.completeIfEqual( "partitionsSpec", findPartitionsSpecFromConfig(tuningConfig), - lastCompactionState.getPartitionsSpec(), + existingPartionsSpec, CompactionStatus::asString ); } diff --git a/server/src/main/java/org/apache/druid/server/coordination/BroadcastDatasourceLoadingSpec.java b/server/src/main/java/org/apache/druid/server/coordination/BroadcastDatasourceLoadingSpec.java new file mode 100644 index 00000000000..3a11027311e --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordination/BroadcastDatasourceLoadingSpec.java @@ -0,0 +1,170 @@ +/* + * 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.server.coordination; + +import com.google.common.collect.ImmutableSet; +import org.apache.druid.error.InvalidInput; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** + * This class defines the spec for loading of broadcast datasources for a given task. It contains 2 fields: + *

      + *
    1. {@link BroadcastDatasourceLoadingSpec#mode}: This mode defines whether broadcastDatasources need to be + * loaded for the given task, or not. It can take 3 values:
    2. + *
        + *
      • ALL: Load all the broadcast datasources.
      • + *
      • NONE: Load no broadcast datasources.
      • + *
      • ONLY_REQUIRED: Load only the broadcast datasources defined in broadcastDatasourcesToLoad
      • + *
      + *
    3. {@link BroadcastDatasourceLoadingSpec#broadcastDatasourcesToLoad}: Defines the broadcastDatasources to load when the broadcastDatasourceLoadingMode is set to ONLY_REQUIRED.
    4. + *
    + */ +public class BroadcastDatasourceLoadingSpec +{ + + public static final String CTX_BROADCAST_DATASOURCE_LOADING_MODE = "broadcastDatasourceLoadingMode"; + public static final String CTX_BROADCAST_DATASOURCES_TO_LOAD = "broadcastDatasourcesToLoad"; + + public enum Mode + { + ALL, NONE, ONLY_REQUIRED + } + + private final Mode mode; + @Nullable + private final ImmutableSet broadcastDatasourcesToLoad; + + public static final BroadcastDatasourceLoadingSpec ALL = new BroadcastDatasourceLoadingSpec(Mode.ALL, null); + public static final BroadcastDatasourceLoadingSpec NONE = new BroadcastDatasourceLoadingSpec(Mode.NONE, null); + + private BroadcastDatasourceLoadingSpec(Mode mode, @Nullable Set broadcastDatasourcesToLoad) + { + this.mode = mode; + this.broadcastDatasourcesToLoad = broadcastDatasourcesToLoad == null ? null : ImmutableSet.copyOf(broadcastDatasourcesToLoad); + } + + /** + * Creates a BroadcastSegmentLoadingSpec which loads only the broadcast datasources present in the given set. + */ + public static BroadcastDatasourceLoadingSpec loadOnly(Set broadcastDatasourcesToLoad) + { + if (broadcastDatasourcesToLoad == null) { + throw InvalidInput.exception("Expected non-null set of broadcast datasources to load."); + } + return new BroadcastDatasourceLoadingSpec(Mode.ONLY_REQUIRED, broadcastDatasourcesToLoad); + } + + public Mode getMode() + { + return mode; + } + + /** + * @return A non-null immutable set of broadcast datasource names when {@link BroadcastDatasourceLoadingSpec#mode} is ONLY_REQUIRED, null otherwise. + */ + public ImmutableSet getBroadcastDatasourcesToLoad() + { + return broadcastDatasourcesToLoad; + } + + public static BroadcastDatasourceLoadingSpec createFromContext(Map context, BroadcastDatasourceLoadingSpec defaultSpec) + { + if (context == null) { + return defaultSpec; + } + + final Object broadcastDatasourceModeValue = context.get(CTX_BROADCAST_DATASOURCE_LOADING_MODE); + if (broadcastDatasourceModeValue == null) { + return defaultSpec; + } + + final BroadcastDatasourceLoadingSpec.Mode broadcastDatasourceLoadingMode; + try { + broadcastDatasourceLoadingMode = BroadcastDatasourceLoadingSpec.Mode.valueOf(broadcastDatasourceModeValue.toString()); + } + catch (IllegalArgumentException e) { + throw InvalidInput.exception( + "Invalid value of %s[%s]. Allowed values are %s", + CTX_BROADCAST_DATASOURCE_LOADING_MODE, broadcastDatasourceModeValue.toString(), + Arrays.asList(BroadcastDatasourceLoadingSpec.Mode.values()) + ); + } + + if (broadcastDatasourceLoadingMode == Mode.NONE) { + return NONE; + } else if (broadcastDatasourceLoadingMode == Mode.ALL) { + return ALL; + } else if (broadcastDatasourceLoadingMode == Mode.ONLY_REQUIRED) { + final Collection broadcastDatasourcesToLoad; + try { + broadcastDatasourcesToLoad = (Collection) context.get(CTX_BROADCAST_DATASOURCES_TO_LOAD); + } + catch (ClassCastException e) { + throw InvalidInput.exception( + "Invalid value of %s[%s]. Please provide a comma-separated list of broadcast datasource names." + + " For example: [\"datasourceName1\", \"datasourceName2\"]", + CTX_BROADCAST_DATASOURCES_TO_LOAD, context.get(CTX_BROADCAST_DATASOURCES_TO_LOAD) + ); + } + + if (broadcastDatasourcesToLoad == null || broadcastDatasourcesToLoad.isEmpty()) { + throw InvalidInput.exception("Set of broadcast datasources to load cannot be %s for mode[ONLY_REQUIRED].", broadcastDatasourcesToLoad); + } + return BroadcastDatasourceLoadingSpec.loadOnly(new HashSet<>(broadcastDatasourcesToLoad)); + } else { + return defaultSpec; + } + } + + @Override + public String toString() + { + return "BroadcastDatasourceLoadingSpec{" + + "mode=" + mode + + ", broadcastDatasourcesToLoad=" + broadcastDatasourcesToLoad + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + BroadcastDatasourceLoadingSpec that = (BroadcastDatasourceLoadingSpec) o; + return mode == that.mode && Objects.equals(broadcastDatasourcesToLoad, that.broadcastDatasourcesToLoad); + } + + @Override + public int hashCode() + { + return Objects.hash(mode, broadcastDatasourcesToLoad); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentBootstrapper.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentBootstrapper.java index c5b71fbcddc..7eec82e80b1 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentBootstrapper.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentBootstrapper.java @@ -39,12 +39,14 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import org.apache.druid.timeline.DataSegment; import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; @@ -80,6 +82,8 @@ public class SegmentBootstrapper private static final EmittingLogger log = new EmittingLogger(SegmentBootstrapper.class); + private final DataSourceTaskIdHolder datasourceHolder; + @Inject public SegmentBootstrapper( SegmentLoadDropHandler loadDropHandler, @@ -89,7 +93,8 @@ public class SegmentBootstrapper SegmentManager segmentManager, ServerTypeConfig serverTypeConfig, CoordinatorClient coordinatorClient, - ServiceEmitter emitter + ServiceEmitter emitter, + DataSourceTaskIdHolder datasourceHolder ) { this.loadDropHandler = loadDropHandler; @@ -100,6 +105,7 @@ public class SegmentBootstrapper this.serverTypeConfig = serverTypeConfig; this.coordinatorClient = coordinatorClient; this.emitter = emitter; + this.datasourceHolder = datasourceHolder; } @LifecycleStart @@ -261,10 +267,18 @@ public class SegmentBootstrapper /** * @return a list of bootstrap segments. When bootstrap segments cannot be found, an empty list is returned. + * The bootstrap segments returned are filtered by the broadcast datasources indicated by {@link DataSourceTaskIdHolder#getBroadcastDatasourceLoadingSpec()} + * if applicable. */ private List getBootstrapSegments() { - log.info("Fetching bootstrap segments from the coordinator."); + final BroadcastDatasourceLoadingSpec.Mode mode = datasourceHolder.getBroadcastDatasourceLoadingSpec().getMode(); + if (mode == BroadcastDatasourceLoadingSpec.Mode.NONE) { + log.info("Skipping fetch of bootstrap segments."); + return ImmutableList.of(); + } + + log.info("Fetching bootstrap segments from the coordinator with BroadcastDatasourceLoadingSpec mode[%s].", mode); final Stopwatch stopwatch = Stopwatch.createStarted(); List bootstrapSegments = new ArrayList<>(); @@ -272,7 +286,18 @@ public class SegmentBootstrapper try { final BootstrapSegmentsResponse response = FutureUtils.getUnchecked(coordinatorClient.fetchBootstrapSegments(), true); - bootstrapSegments = ImmutableList.copyOf(response.getIterator()); + if (mode == BroadcastDatasourceLoadingSpec.Mode.ONLY_REQUIRED) { + final Set broadcastDatasourcesToLoad = datasourceHolder.getBroadcastDatasourceLoadingSpec().getBroadcastDatasourcesToLoad(); + final List filteredBroadcast = new ArrayList<>(); + response.getIterator().forEachRemaining(segment -> { + if (broadcastDatasourcesToLoad.contains(segment.getDataSource())) { + filteredBroadcast.add(segment); + } + }); + bootstrapSegments = filteredBroadcast; + } else { + bootstrapSegments = ImmutableList.copyOf(response.getIterator()); + } } catch (Exception e) { log.warn("Error fetching bootstrap segments from the coordinator: [%s]. ", e.getMessage()); @@ -284,7 +309,6 @@ public class SegmentBootstrapper emitter.emit(new ServiceMetricEvent.Builder().setMetric("segment/bootstrap/count", bootstrapSegments.size())); log.info("Fetched [%d] bootstrap segments in [%d]ms.", bootstrapSegments.size(), fetchRunMillis); } - return bootstrapSegments; } diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index f3264dad44c..3317375347d 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -58,9 +58,7 @@ import org.apache.druid.query.spec.SpecificSegmentQueryRunner; import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TimeBoundaryInspector; -import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.ResourceIdPopulatingQueryRunner; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.SetAndVerifyContextQueryRunner; @@ -91,7 +89,6 @@ public class ServerManager implements QuerySegmentWalker private final ObjectMapper objectMapper; private final CacheConfig cacheConfig; private final SegmentManager segmentManager; - private final JoinableFactoryWrapper joinableFactoryWrapper; private final ServerConfig serverConfig; @Inject @@ -104,7 +101,6 @@ public class ServerManager implements QuerySegmentWalker Cache cache, CacheConfig cacheConfig, SegmentManager segmentManager, - JoinableFactoryWrapper joinableFactoryWrapper, ServerConfig serverConfig ) { @@ -118,7 +114,6 @@ public class ServerManager implements QuerySegmentWalker this.cacheConfig = cacheConfig; this.segmentManager = segmentManager; - this.joinableFactoryWrapper = joinableFactoryWrapper; this.serverConfig = serverConfig; } @@ -278,8 +273,11 @@ public class ServerManager implements QuerySegmentWalker final AtomicLong cpuTimeAccumulator ) { - - + // Short-circuit when the index comes from a tombstone (it has no data by definition), + // check for null also since no all segments (higher level ones) will have QueryableIndex... + if (segment.isTombstone()) { + return new NoopQueryRunner<>(); + } final SpecificSegmentSpec segmentSpec = new SpecificSegmentSpec(segmentDescriptor); final SegmentId segmentId = segment.getId(); @@ -291,12 +289,6 @@ public class ServerManager implements QuerySegmentWalker return new ReportTimelineMissingSegmentQueryRunner<>(segmentDescriptor); } - StorageAdapter storageAdapter = segment.asStorageAdapter(); - // Short-circuit when the index comes from a tombstone (it has no data by definition), - // check for null also since no all segments (higher level ones) will have QueryableIndex... - if (storageAdapter.isFromTombstone()) { - return new NoopQueryRunner<>(); - } String segmentIdString = segmentId.toString(); MetricsEmittingQueryRunner metricsEmittingQueryRunnerInner = new MetricsEmittingQueryRunner<>( diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/CustomExceptionMapper.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/CustomExceptionMapper.java index 2b1b8351aed..18148b0c348 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/CustomExceptionMapper.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/CustomExceptionMapper.java @@ -23,6 +23,7 @@ package org.apache.druid.server.initialization.jetty; import com.fasterxml.jackson.databind.JsonMappingException; import com.google.common.collect.ImmutableMap; +import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.ext.ExceptionMapper; import javax.ws.rs.ext.Provider; @@ -38,6 +39,7 @@ public class CustomExceptionMapper implements ExceptionMapper nodeRoles; @@ -86,6 +87,7 @@ public class MetricsModule implements Module { JsonConfigProvider.bind(binder, MONITORING_PROPERTY_PREFIX, DruidMonitorSchedulerConfig.class); JsonConfigProvider.bind(binder, MONITORING_PROPERTY_PREFIX, MonitorsConfig.class); + JsonConfigProvider.bind(binder, OshiSysMonitorConfig.PREFIX, OshiSysMonitorConfig.class); DruidBinders.metricMonitorBinder(binder); // get the binder so that it will inject the empty set at a minimum. @@ -200,7 +202,11 @@ public class MetricsModule implements Module @Provides @ManageLifecycle - public OshiSysMonitor getOshiSysMonitor(DataSourceTaskIdHolder dataSourceTaskIdHolder, @Self Set nodeRoles) + public OshiSysMonitor getOshiSysMonitor( + DataSourceTaskIdHolder dataSourceTaskIdHolder, + @Self Set nodeRoles, + OshiSysMonitorConfig oshiSysConfig + ) { if (nodeRoles.contains(NodeRole.PEON)) { return new NoopOshiSysMonitor(); @@ -209,7 +215,7 @@ public class MetricsModule implements Module dataSourceTaskIdHolder.getDataSource(), dataSourceTaskIdHolder.getTaskId() ); - return new OshiSysMonitor(dimensions); + return new OshiSysMonitor(dimensions, oshiSysConfig); } } } diff --git a/server/src/main/java/org/apache/druid/server/scheduling/ThresholdBasedQueryPrioritizationStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/ThresholdBasedQueryPrioritizationStrategy.java index 3f76951352c..9d3909b9358 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/ThresholdBasedQueryPrioritizationStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/ThresholdBasedQueryPrioritizationStrategy.java @@ -30,6 +30,7 @@ import org.apache.druid.server.QueryPrioritizationStrategy; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; +import org.joda.time.base.AbstractInterval; import javax.annotation.Nullable; @@ -49,12 +50,14 @@ public class ThresholdBasedQueryPrioritizationStrategy implements QueryPrioritiz private final Optional periodThreshold; private final Optional durationThreshold; + private final Optional segmentRangeThreshold; @JsonCreator public ThresholdBasedQueryPrioritizationStrategy( @JsonProperty("periodThreshold") @Nullable String periodThresholdString, @JsonProperty("durationThreshold") @Nullable String durationThresholdString, @JsonProperty("segmentCountThreshold") @Nullable Integer segmentCountThreshold, + @JsonProperty("segmentRangeThreshold") @Nullable String segmentRangeThresholdString, @JsonProperty("adjustment") @Nullable Integer adjustment ) { @@ -66,9 +69,12 @@ public class ThresholdBasedQueryPrioritizationStrategy implements QueryPrioritiz this.durationThreshold = durationThresholdString == null ? Optional.empty() : Optional.of(new Period(durationThresholdString).toStandardDuration()); + this.segmentRangeThreshold = segmentRangeThresholdString == null + ? Optional.empty() + : Optional.of(new Period(segmentRangeThresholdString).toStandardDuration()); Preconditions.checkArgument( - segmentCountThreshold != null || periodThreshold.isPresent() || durationThreshold.isPresent(), - "periodThreshold, durationThreshold, or segmentCountThreshold must be set" + segmentCountThreshold != null || periodThreshold.isPresent() || durationThreshold.isPresent() || segmentRangeThreshold.isPresent(), + "periodThreshold, durationThreshold, segmentCountThreshold or segmentRangeThreshold must be set" ); } @@ -84,9 +90,19 @@ public class ThresholdBasedQueryPrioritizationStrategy implements QueryPrioritiz }).orElse(false); final boolean violatesDurationThreshold = durationThreshold.map(duration -> theQuery.getDuration().isLongerThan(duration)).orElse(false); + boolean violatesSegmentRangeThreshold = false; + if (segmentRangeThreshold.isPresent()) { + long segmentRange = segments.stream().filter(segment -> segment.getSegmentDescriptor() != null) + .map(segment -> segment.getSegmentDescriptor().getInterval()) + .distinct() + .mapToLong(AbstractInterval::toDurationMillis) + .sum(); + violatesSegmentRangeThreshold = + segmentRangeThreshold.map(duration -> new Duration(segmentRange).isLongerThan(duration)).orElse(false); + } boolean violatesSegmentThreshold = segments.size() > segmentCountThreshold; - if (violatesPeriodThreshold || violatesDurationThreshold || violatesSegmentThreshold) { + if (violatesPeriodThreshold || violatesDurationThreshold || violatesSegmentThreshold || violatesSegmentRangeThreshold) { final int adjustedPriority = theQuery.context().getPriority() - adjustment; return Optional.of(adjustedPriority); } diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/CsvInputFormatTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/CsvInputFormatTest.java index b2995f1838b..fb77e6bdb15 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/CsvInputFormatTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/CsvInputFormatTest.java @@ -66,7 +66,7 @@ public class CsvInputFormatTest extends BaseExternTableTest public void testConversion() { CsvInputFormat format = new CsvInputFormat( - Collections.singletonList("a"), ";", false, false, 1); + Collections.singletonList("a"), ";", false, false, 1, null); TableMetadata table = TableBuilder.external("foo") .inputSource(toMap(new InlineInputSource("a\n"))) .inputFormat(formatToMap(format)) diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/DelimitedInputFormatTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/DelimitedInputFormatTest.java index 04494ec7c34..158b5855e9e 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/DelimitedInputFormatTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/DelimitedInputFormatTest.java @@ -74,7 +74,7 @@ public class DelimitedInputFormatTest extends BaseExternTableTest public void testConversion() { DelimitedInputFormat format = new DelimitedInputFormat( - Collections.singletonList("a"), ";", "|", false, false, 1); + Collections.singletonList("a"), ";", "|", false, false, 1, null); TableMetadata table = TableBuilder.external("foo") .inputSource(toMap(new InlineInputSource("a\n"))) .inputFormat(formatToMap(format)) diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/ExternalTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/ExternalTableTest.java index 8c8129bf0fc..acbea354703 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/ExternalTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/ExternalTableTest.java @@ -122,7 +122,7 @@ public class ExternalTableTest extends BaseExternTableTest { // Format is given without columns: it is validated CsvInputFormat format = new CsvInputFormat( - Collections.singletonList("a"), ";", false, false, 0); + Collections.singletonList("a"), ";", false, false, 0, null); TableMetadata table = TableBuilder.external("foo") .inputSource(toMap(new InlineInputSource("a\n"))) .inputFormat(formatToMap(format)) @@ -170,7 +170,8 @@ public class ExternalTableTest extends BaseExternTableTest "bob", new DefaultPasswordProvider("secret"), null, - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ); Map sourceMap = toMap(inputSource); sourceMap.remove("uris"); @@ -195,7 +196,8 @@ public class ExternalTableTest extends BaseExternTableTest "bob", new DefaultPasswordProvider("secret"), null, - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ); TableMetadata table = TableBuilder.external("koala") .inputSource(toMap(inputSource)) diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputSourceDefnTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputSourceDefnTest.java index 8a6385db59c..e60139824d7 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputSourceDefnTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputSourceDefnTest.java @@ -57,7 +57,7 @@ public class HttpInputSourceDefnTest extends BaseExternTableTest { mapper.setInjectableValues(new InjectableValues.Std().addValue( HttpInputSourceConfig.class, - new HttpInputSourceConfig(HttpInputSourceConfig.DEFAULT_ALLOWED_PROTOCOLS) + new HttpInputSourceConfig(HttpInputSourceConfig.DEFAULT_ALLOWED_PROTOCOLS, null) )); } @@ -99,7 +99,8 @@ public class HttpInputSourceDefnTest extends BaseExternTableTest null, null, null, - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ); TableMetadata table = TableBuilder.external("foo") .inputSource(toMap(inputSource)) @@ -119,7 +120,8 @@ public class HttpInputSourceDefnTest extends BaseExternTableTest null, null, null, - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ); TableMetadata table = TableBuilder.external("foo") .inputSource(toMap(inputSource)) @@ -150,7 +152,8 @@ public class HttpInputSourceDefnTest extends BaseExternTableTest null, null, null, - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ); TableMetadata table = TableBuilder.external("foo") .inputSource(toMap(inputSource)) @@ -216,7 +219,8 @@ public class HttpInputSourceDefnTest extends BaseExternTableTest "bob", new DefaultPasswordProvider("secret"), null, - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ); TableMetadata table = TableBuilder.external("foo") .inputSource(toMap(inputSource)) @@ -272,11 +276,12 @@ public class HttpInputSourceDefnTest extends BaseExternTableTest // Get the partial table function TableFunction fn = externDefn.tableFn(resolved); - assertEquals(4, fn.parameters().size()); + assertEquals(5, fn.parameters().size()); assertTrue(hasParam(fn, HttpInputSourceDefn.URIS_PARAMETER)); assertTrue(hasParam(fn, HttpInputSourceDefn.USER_PARAMETER)); assertTrue(hasParam(fn, HttpInputSourceDefn.PASSWORD_PARAMETER)); assertTrue(hasParam(fn, HttpInputSourceDefn.PASSWORD_ENV_VAR_PARAMETER)); + assertTrue(hasParam(fn, HttpInputSourceDefn.HEADERS)); // Convert to an external table. ExternalTableSpec externSpec = fn.apply( @@ -320,8 +325,9 @@ public class HttpInputSourceDefnTest extends BaseExternTableTest // Get the partial table function TableFunction fn = externDefn.tableFn(resolved); - assertEquals(1, fn.parameters().size()); + assertEquals(2, fn.parameters().size()); assertTrue(hasParam(fn, HttpInputSourceDefn.URIS_PARAMETER)); + assertTrue(hasParam(fn, HttpInputSourceDefn.HEADERS)); // Convert to an external table. ExternalTableSpec externSpec = fn.apply( @@ -344,7 +350,8 @@ public class HttpInputSourceDefnTest extends BaseExternTableTest "bob", new DefaultPasswordProvider("secret"), null, - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ); TableMetadata table = TableBuilder.external("foo") .inputSource(httpToMap(inputSource)) @@ -382,7 +389,8 @@ public class HttpInputSourceDefnTest extends BaseExternTableTest "bob", new EnvironmentVariablePasswordProvider("SECRET"), null, - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ); TableMetadata table = TableBuilder.external("foo") .inputSource(toMap(inputSource)) @@ -415,7 +423,8 @@ public class HttpInputSourceDefnTest extends BaseExternTableTest "bob", new DefaultPasswordProvider("secret"), null, - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ); TableMetadata table = TableBuilder.external("foo") .inputSource(httpToMap(inputSource)) @@ -484,7 +493,8 @@ public class HttpInputSourceDefnTest extends BaseExternTableTest "bob", new EnvironmentVariablePasswordProvider("SECRET"), null, - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ); TableMetadata table = TableBuilder.external("foo") .inputSource(toMap(inputSource)) @@ -518,7 +528,6 @@ public class HttpInputSourceDefnTest extends BaseExternTableTest assertEquals("secret", ((DefaultPasswordProvider) sourceSpec.getHttpAuthenticationPasswordProvider()).getPassword()); } assertEquals("http://foo.com/my.csv", sourceSpec.getUris().get(0).toString()); - // Just a sanity check: details of CSV conversion are tested elsewhere. CsvInputFormat csvFormat = (CsvInputFormat) externSpec.inputFormat; assertEquals(Arrays.asList("x", "y"), csvFormat.getColumns()); diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/InlineInputSourceDefnTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/InlineInputSourceDefnTest.java index bb3b2354bac..c80b843ce55 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/InlineInputSourceDefnTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/InlineInputSourceDefnTest.java @@ -194,7 +194,7 @@ public class InlineInputSourceDefnTest extends BaseExternTableTest { // Define an inline table CsvInputFormat format = new CsvInputFormat( - Collections.singletonList("a"), ";", false, false, 0); + Collections.singletonList("a"), ";", false, false, 0, null); TableMetadata table = TableBuilder.external("foo") .inputSource(toMap(new InlineInputSource("a,b\nc,d"))) .inputFormat(formatToMap(format)) diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java index 143f8917d78..fc99af76321 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java @@ -19,6 +19,7 @@ package org.apache.druid.metadata; +import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -48,6 +49,7 @@ import org.skife.jdbi.v2.Handle; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -281,6 +283,112 @@ public class IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest extends segmentSchemaTestUtils.verifySegmentSchema(segmentIdSchemaMap); } + @Test + public void testSchemaPermutation() throws JsonProcessingException + { + Set segments = new HashSet<>(); + SegmentSchemaMapping segmentSchemaMapping = new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION); + // Store the first observed column order for each segment for verification purpose + Map> segmentIdSchemaMap = new HashMap<>(); + + RowSignature originalOrder = + RowSignature.builder() + .add("d7", ColumnType.LONG_ARRAY) + .add("b1", ColumnType.FLOAT) + .add("a5", ColumnType.DOUBLE) + .build(); + + // column permutations + List> permutations = Arrays.asList( + Arrays.asList("d7", "a5", "b1"), + Arrays.asList("a5", "b1", "d7"), + Arrays.asList("a5", "d7", "b1"), + Arrays.asList("b1", "d7", "a5"), + Arrays.asList("b1", "a5", "d7"), + Arrays.asList("d7", "a5", "b1") + ); + + boolean first = true; + + Random random = ThreadLocalRandom.current(); + Random permutationRandom = ThreadLocalRandom.current(); + + for (int i = 0; i < 105; i++) { + DataSegment segment = new DataSegment( + "fooDataSource", + Intervals.of("2015-01-01T00Z/2015-01-02T00Z"), + "version", + ImmutableMap.of(), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new LinearShardSpec(i), + 9, + 100 + ); + segments.add(segment); + + int randomNum = random.nextInt(); + + RowSignature rowSignature; + + if (first) { + rowSignature = originalOrder; + } else { + RowSignature.Builder builder = RowSignature.builder(); + List columns = permutations.get(permutationRandom.nextInt(permutations.size())); + + for (String column : columns) { + builder.add(column, originalOrder.getColumnType(column).get()); + } + + rowSignature = builder.build(); + } + + SchemaPayload schemaPayload = new SchemaPayload(rowSignature); + segmentIdSchemaMap.put(segment.getId().toString(), Pair.of(new SchemaPayload(originalOrder), randomNum)); + segmentSchemaMapping.addSchema( + segment.getId(), + new SchemaPayloadPlus(schemaPayload, (long) randomNum), + fingerprintGenerator.generateFingerprint( + schemaPayload, + segment.getDataSource(), + CentralizedDatasourceSchemaConfig.SCHEMA_VERSION + ) + ); + + if (first) { + coordinator.commitSegments(segments, segmentSchemaMapping); + first = false; + } + } + + coordinator.commitSegments(segments, segmentSchemaMapping); + for (DataSegment segment : segments) { + Assert.assertArrayEquals( + mapper.writeValueAsString(segment).getBytes(StandardCharsets.UTF_8), + derbyConnector.lookup( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "id", + "payload", + segment.getId().toString() + ) + ); + } + + List segmentIds = segments.stream() + .map(segment -> segment.getId().toString()) + .sorted(Comparator.naturalOrder()) + .collect(Collectors.toList()); + + Assert.assertEquals(segmentIds, retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get())); + + // Should not update dataSource metadata. + Assert.assertEquals(0, metadataUpdateCounter.get()); + + // verify that only a single schema is created + segmentSchemaTestUtils.verifySegmentSchema(segmentIdSchemaMap); + } + @Test public void testAnnounceHistoricalSegments_schemaExists() throws IOException { diff --git a/server/src/test/java/org/apache/druid/metadata/input/InputSourceModuleTest.java b/server/src/test/java/org/apache/druid/metadata/input/InputSourceModuleTest.java index 074b1dbeb22..496f45596a2 100644 --- a/server/src/test/java/org/apache/druid/metadata/input/InputSourceModuleTest.java +++ b/server/src/test/java/org/apache/druid/metadata/input/InputSourceModuleTest.java @@ -79,7 +79,7 @@ public class InputSourceModuleTest Properties props = new Properties(); Injector injector = makeInjectorWithProperties(props); HttpInputSourceConfig instance = injector.getInstance(HttpInputSourceConfig.class); - Assert.assertEquals(new HttpInputSourceConfig(null), instance); + Assert.assertEquals(new HttpInputSourceConfig(null, null), instance); Assert.assertEquals(HttpInputSourceConfig.DEFAULT_ALLOWED_PROTOCOLS, instance.getAllowedProtocols()); } diff --git a/server/src/test/java/org/apache/druid/rpc/MockServiceClient.java b/server/src/test/java/org/apache/druid/rpc/MockServiceClient.java index da817c3da3b..021db219d96 100644 --- a/server/src/test/java/org/apache/druid/rpc/MockServiceClient.java +++ b/server/src/test/java/org/apache/druid/rpc/MockServiceClient.java @@ -41,6 +41,7 @@ import java.util.Queue; public class MockServiceClient implements ServiceClient { private final Queue expectations = new ArrayDeque<>(16); + private int requestNumber = -1; @Override public ListenableFuture asyncRequest( @@ -50,8 +51,9 @@ public class MockServiceClient implements ServiceClient { final Expectation expectation = expectations.poll(); + requestNumber++; Assert.assertEquals( - "request", + "request[" + requestNumber + "]", expectation == null ? null : expectation.request, requestBuilder ); diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index 90297dd4af9..94bd77e810d 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -71,6 +71,11 @@ import java.util.Set; public class DataSchemaTest extends InitializedNullHandlingTest { + private static ArbitraryGranularitySpec ARBITRARY_GRANULARITY = new ArbitraryGranularitySpec( + Granularities.DAY, + ImmutableList.of(Intervals.of("2014/2015")) + ); + @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -92,17 +97,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - parser, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); Assert.assertEquals( ImmutableSet.of("__time", "time", "col1", "col2", "metric1", "metric2"), @@ -130,18 +134,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest null ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - parser, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); Assert.assertEquals( ImmutableSet.of("__time", "dimC", "col1", "metric1", "metric2"), @@ -167,22 +169,28 @@ public class DataSchemaTest extends InitializedNullHandlingTest ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - parserMap, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - new TransformSpec( - new SelectorDimFilter("dimA", "foo", null), - ImmutableList.of( - new ExpressionTransform("expr", "concat(dimA,dimA)", TestExprMacroTable.INSTANCE) - ) - ), - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parserMap) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withTransform( + new TransformSpec( + new SelectorDimFilter("dimA", "foo", null), + ImmutableList.of( + new ExpressionTransform( + "expr", + "concat(dimA,dimA)", + TestExprMacroTable.INSTANCE + ) + ) + ) + ) + .withObjectMapper(jsonMapper) + .build(); // Test hack that produces a StringInputRowParser. final StringInputRowParser parser = (StringInputRowParser) schema.getParser(); @@ -233,17 +241,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - parser, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); expectedException.expect(DruidException.class); expectedException.expectMessage( @@ -256,25 +263,24 @@ public class DataSchemaTest extends InitializedNullHandlingTest @Test public void testOverlapTimeAndDimPositionZero() { - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - new TimestampSpec("time", "auto", null), - DimensionsSpec.builder() - .setDimensions( - ImmutableList.of( - new LongDimensionSchema("__time"), - new StringDimensionSchema("dimA"), - new StringDimensionSchema("dimB") - ) - ) - .setDimensionExclusions(ImmutableList.of("dimC")) - .build(), - null, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - null, - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withTimestamp(new TimestampSpec("time", "auto", null)) + .withDimensions( + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new LongDimensionSchema("__time"), + new StringDimensionSchema("dimA"), + new StringDimensionSchema("dimB") + ) + ) + .setDimensionExclusions(ImmutableList.of("dimC")) + .build() + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); Assert.assertEquals( ImmutableList.of("__time", "dimA", "dimB"), @@ -290,25 +296,24 @@ public class DataSchemaTest extends InitializedNullHandlingTest expectedException.expect(DruidException.class); expectedException.expectMessage("Encountered dimension[__time] with incorrect type[STRING]. Type must be 'long'."); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - new TimestampSpec("time", "auto", null), - DimensionsSpec.builder() - .setDimensions( - ImmutableList.of( - new StringDimensionSchema("__time"), - new StringDimensionSchema("dimA"), - new StringDimensionSchema("dimB") - ) - ) - .setDimensionExclusions(ImmutableList.of("dimC")) - .build(), - null, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - null, - jsonMapper - ); + DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withTimestamp(new TimestampSpec("time", "auto", null)) + .withDimensions( + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("__time"), + new StringDimensionSchema("dimA"), + new StringDimensionSchema("dimB") + ) + ) + .setDimensionExclusions(ImmutableList.of("dimC")) + .build() + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); } @Test @@ -321,50 +326,49 @@ public class DataSchemaTest extends InitializedNullHandlingTest + DimensionsSpec.WARNING_NON_TIME_SORT_ORDER ); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - new TimestampSpec("time", "auto", null), - DimensionsSpec.builder() - .setDimensions( - ImmutableList.of( - new StringDimensionSchema("dimA"), - new LongDimensionSchema("__time"), - new StringDimensionSchema("dimB") - ) - ) - .setDimensionExclusions(ImmutableList.of("dimC")) - .build(), - null, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - null, - jsonMapper - ); + DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withTimestamp(new TimestampSpec("time", "auto", null)) + .withDimensions( + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("dimA"), + new LongDimensionSchema("__time"), + new StringDimensionSchema("dimB") + ) + ) + .setDimensionExclusions(ImmutableList.of("dimC")) + .build() + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); } @Test public void testOverlapTimeAndDimPositionOne_withExplicitSortOrder() { - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - new TimestampSpec("time", "auto", null), - DimensionsSpec.builder() - .setDimensions( - ImmutableList.of( - new StringDimensionSchema("dimA"), - new LongDimensionSchema("__time"), - new StringDimensionSchema("dimB") - ) - ) - .setDimensionExclusions(ImmutableList.of("dimC")) - .setForceSegmentSortByTime(false) - .build(), - null, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - null, - jsonMapper - ); + DataSchema schema = + DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withTimestamp(new TimestampSpec("time", "auto", null)) + .withDimensions( + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("dimA"), + new LongDimensionSchema("__time"), + new StringDimensionSchema("dimB") + ) + ) + .setDimensionExclusions(ImmutableList.of("dimC")) + .setForceSegmentSortByTime(false) + .build() + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); Assert.assertEquals( ImmutableList.of("dimA", "__time", "dimB"), @@ -402,14 +406,13 @@ public class DataSchemaTest extends InitializedNullHandlingTest ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - parser, - null, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); + expectedException.expect(DruidException.class); expectedException.expectMessage("Encountered dimension[__time] with incorrect type[STRING]. Type must be 'long'."); @@ -442,20 +445,19 @@ public class DataSchemaTest extends InitializedNullHandlingTest + "[metric3] seen in metricsSpec list (2 occurrences)" ); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - parser, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - new DoubleSumAggregatorFactory("metric1", "col3"), - new DoubleSumAggregatorFactory("metric3", "col4"), - new DoubleSumAggregatorFactory("metric3", "col5"), - }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2"), + new DoubleSumAggregatorFactory("metric1", "col3"), + new DoubleSumAggregatorFactory("metric3", "col4"), + new DoubleSumAggregatorFactory("metric3", "col5") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); } @Test @@ -510,24 +512,20 @@ public class DataSchemaTest extends InitializedNullHandlingTest ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DruidExceptionMatcher - .invalidInput() - .expectMessageIs("Invalid value for field [dataSource]: must not be null") - .assertThrowsAndMatches( - () -> new DataSchema( - "", - parser, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - }, - new ArbitraryGranularitySpec( - Granularities.DAY, - ImmutableList.of(Intervals.of("2014/2015")) - ), - null, - jsonMapper - )); + DruidExceptionMatcher.ThrowingSupplier thrower = + () -> DataSchema.builder() + .withDataSource("") + .withParserMap(parser) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); + DruidExceptionMatcher.invalidInput() + .expectMessageIs("Invalid value for field [dataSource]: must not be null") + .assertThrowsAndMatches(thrower); } @@ -547,14 +545,11 @@ public class DataSchemaTest extends InitializedNullHandlingTest dataSource ); DruidExceptionMatcher.invalidInput().expectMessageIs(msg).assertThrowsAndMatches( - () -> new DataSchema( - dataSource, - Collections.emptyMap(), - null, - null, - null, - jsonMapper - ) + () -> DataSchema.builder() + .withDataSource(dataSource) + .withParserMap(Collections.emptyMap()) + .withObjectMapper(jsonMapper) + .build() ); } } @@ -686,17 +681,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DataSchema originalSchema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - parser, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - jsonMapper - ); + DataSchema originalSchema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); String serialized = jsonMapper.writeValueAsString(originalSchema); TestModifiedDataSchema deserialized = jsonMapper.readValue(serialized, TestModifiedDataSchema.class); @@ -734,7 +728,7 @@ public class DataSchemaTest extends InitializedNullHandlingTest new DoubleSumAggregatorFactory("metric1", "col1"), new DoubleSumAggregatorFactory("metric2", "col2"), }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + ARBITRARY_GRANULARITY, null, parser, jsonMapper, @@ -765,10 +759,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest Map parserMap = Mockito.mock(Map.class); Mockito.when(newDimSpec.withDimensionExclusions(ArgumentMatchers.any(Set.class))).thenReturn(newDimSpec); - DataSchema oldSchema = new DataSchema("dataSource", tsSpec, oldDimSpec, - new AggregatorFactory[]{aggFactory}, gSpec, - transSpec, parserMap, jsonMapper - ); + DataSchema oldSchema = DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(tsSpec) + .withDimensions(oldDimSpec) + .withAggregators(aggFactory) + .withGranularity(gSpec) + .withTransform(transSpec) + .withParserMap(parserMap) + .withObjectMapper(jsonMapper) + .build(); DataSchema newSchema = oldSchema.withDimensionsSpec(newDimSpec); Assert.assertSame(oldSchema.getDataSource(), newSchema.getDataSource()); Assert.assertSame(oldSchema.getTimestampSpec(), newSchema.getTimestampSpec()); @@ -795,7 +795,7 @@ public class DataSchemaTest extends InitializedNullHandlingTest .setDimensionExclusions(ImmutableList.of("dimC")) .build(), null, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + ARBITRARY_GRANULARITY, null, multiValuedDimensions ); diff --git a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java index 323ec797890..1ca73d4b934 100644 --- a/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/loading/SegmentLocalCacheManagerTest.java @@ -29,10 +29,10 @@ import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.server.TestSegmentUtils; @@ -1251,11 +1251,11 @@ public class SegmentLocalCacheManagerTest Assert.assertEquals(tombstone.getId(), segment.getId()); Assert.assertEquals(interval, segment.getDataInterval()); - final StorageAdapter storageAdapter = segment.asStorageAdapter(); - Assert.assertNotNull(storageAdapter); - Assert.assertTrue(storageAdapter.isFromTombstone()); + final CursorFactory cursorFactory = segment.asCursorFactory(); + Assert.assertNotNull(cursorFactory); + Assert.assertTrue(segment.isTombstone()); - final QueryableIndex queryableIndex = segment.asQueryableIndex(); + final QueryableIndex queryableIndex = segment.as(QueryableIndex.class); Assert.assertNotNull(queryableIndex); Assert.assertEquals(interval, queryableIndex.getDataInterval()); Assert.assertThrows(UnsupportedOperationException.class, queryableIndex::getMetadata); diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java index 772a79ae0ad..22b0890e855 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java @@ -32,6 +32,7 @@ import org.apache.druid.client.DruidServer; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.InternalQueryConfig; import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; @@ -56,8 +57,10 @@ import org.apache.druid.query.metadata.metadata.SegmentAnalysis; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.PhysicalSegmentInspector; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; +import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.SchemaPayload; import org.apache.druid.segment.SchemaPayloadPlus; import org.apache.druid.segment.SegmentMetadata; @@ -1540,8 +1543,8 @@ public class CoordinatorSegmentMetadataCacheTest extends CoordinatorSegmentMetad config ); - QueryableIndexStorageAdapter index1StorageAdaptor = new QueryableIndexStorageAdapter(index1); - QueryableIndexStorageAdapter index2StorageAdaptor = new QueryableIndexStorageAdapter(index2); + QueryableIndexCursorFactory index1CursorFactory = new QueryableIndexCursorFactory(index1); + QueryableIndexCursorFactory index2CursorFactory = new QueryableIndexCursorFactory(index2); MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); @@ -1558,27 +1561,27 @@ public class CoordinatorSegmentMetadataCacheTest extends CoordinatorSegmentMetad pluses.add(new SegmentSchemaManager.SegmentSchemaMetadataPlus( segment1.getId(), fingerprintGenerator.generateFingerprint( - new SchemaPayload(index1StorageAdaptor.getRowSignature()), + new SchemaPayload(index1CursorFactory.getRowSignature()), segment1.getDataSource(), CentralizedDatasourceSchemaConfig.SCHEMA_VERSION ), new SchemaPayloadPlus( new SchemaPayload( - index1StorageAdaptor.getRowSignature()), - (long) index1StorageAdaptor.getNumRows() + index1CursorFactory.getRowSignature()), + (long) index1.getNumRows() ) )); pluses.add(new SegmentSchemaManager.SegmentSchemaMetadataPlus( segment2.getId(), fingerprintGenerator.generateFingerprint( - new SchemaPayload(index2StorageAdaptor.getRowSignature()), + new SchemaPayload(index2CursorFactory.getRowSignature()), segment1.getDataSource(), CentralizedDatasourceSchemaConfig.SCHEMA_VERSION ), new SchemaPayloadPlus( new SchemaPayload( - index2StorageAdaptor.getRowSignature()), - (long) index2StorageAdaptor.getNumRows() + index2CursorFactory.getRowSignature()), + (long) index2.getNumRows() ) )); @@ -1684,9 +1687,9 @@ public class CoordinatorSegmentMetadataCacheTest extends CoordinatorSegmentMetad try { SchemaPayload schemaPayload = mapper.readValue(r.getBytes(1), SchemaPayload.class); long numRows = r.getLong(2); - QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index2); - Assert.assertEquals(adapter.getRowSignature(), schemaPayload.getRowSignature()); - Assert.assertEquals(adapter.getNumRows(), numRows); + QueryableIndexCursorFactory cursorFa = new QueryableIndexCursorFactory(index2); + Assert.assertEquals(cursorFa.getRowSignature(), schemaPayload.getRowSignature()); + Assert.assertEquals(index2.getNumRows(), numRows); } catch (IOException e) { throw new RuntimeException(e); @@ -1711,12 +1714,14 @@ public class CoordinatorSegmentMetadataCacheTest extends CoordinatorSegmentMetad config.setDisableSegmentMetadataQueries(true); CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(config); - QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index2); + QueryableIndexSegment queryableIndexSegment = new QueryableIndexSegment(index2, SegmentId.dummy("test")); + PhysicalSegmentInspector rowCountInspector = queryableIndexSegment.as(PhysicalSegmentInspector.class); + QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index2); ImmutableMap.Builder segmentStatsMap = new ImmutableMap.Builder<>(); - segmentStatsMap.put(segment3.getId(), new SegmentMetadata((long) adapter.getNumRows(), "fp")); + segmentStatsMap.put(segment3.getId(), new SegmentMetadata((long) rowCountInspector.getNumRows(), "fp")); ImmutableMap.Builder schemaPayloadMap = new ImmutableMap.Builder<>(); - schemaPayloadMap.put("fp", new SchemaPayload(adapter.getRowSignature())); + schemaPayloadMap.put("fp", new SchemaPayload(cursorFactory.getRowSignature())); segmentSchemaCache.updateFinalizedSegmentSchema( new SegmentSchemaCache.FinalizedSegmentSchemaInfo(segmentStatsMap.build(), schemaPayloadMap.build()) ); @@ -2216,11 +2221,116 @@ public class CoordinatorSegmentMetadataCacheTest extends CoordinatorSegmentMetad } @Test - public void testTombstoneSegmentIsNotAdded() throws InterruptedException + public void testTombstoneSegmentIsNotRefreshed() throws IOException { - String datasource = "newSegmentAddTest"; - CountDownLatch addSegmentLatch = new CountDownLatch(1); + String brokerInternalQueryConfigJson = "{\"context\": { \"priority\": 5} }"; + TestHelper.makeJsonMapper(); + InternalQueryConfig internalQueryConfig = MAPPER.readValue( + MAPPER.writeValueAsString( + MAPPER.readValue(brokerInternalQueryConfigJson, InternalQueryConfig.class) + ), + InternalQueryConfig.class + ); + + QueryLifecycleFactory factoryMock = EasyMock.createMock(QueryLifecycleFactory.class); + QueryLifecycle lifecycleMock = EasyMock.createMock(QueryLifecycle.class); + + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + factoryMock, + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + internalQueryConfig, + new NoopServiceEmitter(), + segmentSchemaCache, + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier + ); + + Map queryContext = ImmutableMap.of( + QueryContexts.PRIORITY_KEY, 5, + QueryContexts.BROKER_PARALLEL_MERGE_KEY, false + ); + + DataSegment segment = newSegment("test", 0); + DataSegment tombstone = DataSegment.builder() + .dataSource("test") + .interval(Intervals.of("2012-01-01/2012-01-02")) + .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString()) + .shardSpec(new TombstoneShardSpec()) + .loadSpec(Collections.singletonMap( + "type", + DataSegment.TOMBSTONE_LOADSPEC_TYPE + )) + .size(0) + .build(); + + final DruidServer historicalServer = druidServers.stream() + .filter(s -> s.getType().equals(ServerType.HISTORICAL)) + .findAny() + .orElse(null); + + Assert.assertNotNull(historicalServer); + final DruidServerMetadata historicalServerMetadata = historicalServer.getMetadata(); + + schema.addSegment(historicalServerMetadata, segment); + schema.addSegment(historicalServerMetadata, tombstone); + Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(tombstone.getId())); + + List segmentIterable = ImmutableList.of(segment.getId(), tombstone.getId()); + + SegmentMetadataQuery expectedMetadataQuery = new SegmentMetadataQuery( + new TableDataSource(segment.getDataSource()), + new MultipleSpecificSegmentSpec( + segmentIterable.stream() + .filter(id -> !id.equals(tombstone.getId())) + .map(SegmentId::toDescriptor) + .collect(Collectors.toList()) + ), + new AllColumnIncluderator(), + false, + queryContext, + EnumSet.of(SegmentMetadataQuery.AnalysisType.AGGREGATORS), + false, + null, + null + ); + + EasyMock.expect(factoryMock.factorize()).andReturn(lifecycleMock).once(); + EasyMock.expect(lifecycleMock.runSimple(expectedMetadataQuery, AllowAllAuthenticator.ALLOW_ALL_RESULT, Access.OK)) + .andReturn(QueryResponse.withEmptyContext(Sequences.empty())).once(); + + EasyMock.replay(factoryMock, lifecycleMock); + + schema.refresh(Collections.singleton(segment.getId()), Collections.singleton("test")); + + // verify that metadata query is not issued for tombstone segment + EasyMock.verify(factoryMock, lifecycleMock); + + // Verify that datasource schema building logic doesn't mark the tombstone segment for refresh + Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(tombstone.getId())); + + AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata("test", tombstone.getId()); + Assert.assertNotNull(availableSegmentMetadata); + // fetching metadata for tombstone segment shouldn't mark it for refresh + Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(tombstone.getId())); + + Set metadatas = new HashSet<>(); + schema.iterateSegmentMetadata().forEachRemaining(metadatas::add); + + Assert.assertEquals(1, metadatas.stream().filter(metadata -> metadata.getSegment().isTombstone()).count()); + + // iterating over entire metadata doesn't cause tombstone to be marked for refresh + Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(tombstone.getId())); + } + + @Test + public void testUnusedSegmentIsNotRefreshed() throws InterruptedException, IOException + { + String dataSource = "xyz"; + CountDownLatch latch = new CountDownLatch(1); CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( getQueryLifecycleFactory(walker), serverView, @@ -2232,58 +2342,105 @@ public class CoordinatorSegmentMetadataCacheTest extends CoordinatorSegmentMetad backFillQueue, sqlSegmentsMetadataManager, segmentsMetadataManagerConfigSupplier - ) - { + ) { @Override - public void addSegment(final DruidServerMetadata server, final DataSegment segment) + public void refresh(Set segmentsToRefresh, Set dataSourcesToRebuild) + throws IOException { - super.addSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - addSegmentLatch.countDown(); - } + super.refresh(segmentsToRefresh, dataSourcesToRebuild); + latch.countDown(); } }; + List segments = ImmutableList.of( + newSegment(dataSource, 1), + newSegment(dataSource, 2), + newSegment(dataSource, 3) + ); + + final DruidServer historicalServer = druidServers.stream() + .filter(s -> s.getType().equals(ServerType.HISTORICAL)) + .findAny() + .orElse(null); + + Assert.assertNotNull(historicalServer); + final DruidServerMetadata historicalServerMetadata = historicalServer.getMetadata(); + + ImmutableMap.Builder segmentStatsMap = new ImmutableMap.Builder<>(); + segmentStatsMap.put(segments.get(0).getId(), new SegmentMetadata(20L, "fp")); + segmentStatsMap.put(segments.get(1).getId(), new SegmentMetadata(20L, "fp")); + segmentStatsMap.put(segments.get(2).getId(), new SegmentMetadata(20L, "fp")); + + ImmutableMap.Builder schemaPayloadMap = new ImmutableMap.Builder<>(); + schemaPayloadMap.put("fp", new SchemaPayload(RowSignature.builder().add("c1", ColumnType.DOUBLE).build())); + segmentSchemaCache.updateFinalizedSegmentSchema( + new SegmentSchemaCache.FinalizedSegmentSchemaInfo(segmentStatsMap.build(), schemaPayloadMap.build()) + ); + + schema.addSegment(historicalServerMetadata, segments.get(0)); + schema.addSegment(historicalServerMetadata, segments.get(1)); + schema.addSegment(historicalServerMetadata, segments.get(2)); + + serverView.addSegment(segments.get(0), ServerType.HISTORICAL); + serverView.addSegment(segments.get(1), ServerType.HISTORICAL); + serverView.addSegment(segments.get(2), ServerType.HISTORICAL); + schema.onLeaderStart(); schema.awaitInitialization(); - DataSegment segment = new DataSegment( - datasource, - Intervals.of("2001/2002"), - "1", - Collections.emptyMap(), - Collections.emptyList(), - Collections.emptyList(), - TombstoneShardSpec.INSTANCE, - null, - null, - 0 + Assert.assertTrue(latch.await(2, TimeUnit.SECONDS)); + + // make segment3 unused + segmentStatsMap = new ImmutableMap.Builder<>(); + segmentStatsMap.put(segments.get(0).getId(), new SegmentMetadata(20L, "fp")); + + segmentSchemaCache.updateFinalizedSegmentSchema( + new SegmentSchemaCache.FinalizedSegmentSchemaInfo(segmentStatsMap.build(), schemaPayloadMap.build()) ); - Assert.assertEquals(6, schema.getTotalSegments()); + Map segmentMap = new HashMap<>(); + segmentMap.put(segments.get(0).getId(), segments.get(0)); + segmentMap.put(segments.get(1).getId(), segments.get(1)); - serverView.addSegment(segment, ServerType.HISTORICAL); - Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); - Assert.assertEquals(0, addSegmentLatch.getCount()); + ImmutableDruidDataSource druidDataSource = + new ImmutableDruidDataSource( + "xyz", + Collections.emptyMap(), + segmentMap + ); - Assert.assertEquals(6, schema.getTotalSegments()); - List metadatas = schema - .getSegmentMetadataSnapshot() - .values() - .stream() - .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) - .collect(Collectors.toList()); - Assert.assertEquals(0, metadatas.size()); + Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(ArgumentMatchers.anyString())) + .thenReturn(druidDataSource); - serverView.removeSegment(segment, ServerType.HISTORICAL); - Assert.assertEquals(6, schema.getTotalSegments()); - metadatas = schema - .getSegmentMetadataSnapshot() - .values() - .stream() - .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) - .collect(Collectors.toList()); - Assert.assertEquals(0, metadatas.size()); + Set segmentsToRefresh = segments.stream().map(DataSegment::getId).collect(Collectors.toSet()); + segmentsToRefresh.remove(segments.get(1).getId()); + segmentsToRefresh.remove(segments.get(2).getId()); + + schema.refresh(segmentsToRefresh, Sets.newHashSet(dataSource)); + + Assert.assertTrue(schema.getSegmentsNeedingRefresh().contains(segments.get(1).getId())); + Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(segments.get(2).getId())); + + AvailableSegmentMetadata availableSegmentMetadata = + schema.getAvailableSegmentMetadata(dataSource, segments.get(0).getId()); + + Assert.assertNotNull(availableSegmentMetadata); + // fetching metadata for unused segment shouldn't mark it for refresh + Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(segments.get(0).getId())); + + Set metadatas = new HashSet<>(); + schema.iterateSegmentMetadata().forEachRemaining(metadatas::add); + + Assert.assertEquals( + 1, + metadatas.stream() + .filter( + metadata -> + metadata.getSegment().getId().equals(segments.get(0).getId())).count() + ); + + // iterating over entire metadata doesn't cause unsed segment to be marked for refresh + Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(segments.get(0).getId())); } private void verifyFooDSSchema(CoordinatorSegmentMetadataCache schema, int columns) diff --git a/server/src/test/java/org/apache/druid/segment/metadata/FingerprintGeneratorTest.java b/server/src/test/java/org/apache/druid/segment/metadata/FingerprintGeneratorTest.java index 09358550802..c0100ffe151 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/FingerprintGeneratorTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/FingerprintGeneratorTest.java @@ -22,6 +22,7 @@ package org.apache.druid.segment.metadata; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.common.config.NullHandling; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.any.StringAnyAggregatorFactory; import org.apache.druid.query.aggregation.firstlast.first.LongFirstAggregatorFactory; import org.apache.druid.segment.SchemaPayload; import org.apache.druid.segment.TestHelper; @@ -30,7 +31,9 @@ import org.apache.druid.segment.column.RowSignature; import org.junit.Assert; import org.junit.Test; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; public class FingerprintGeneratorTest @@ -45,13 +48,20 @@ public class FingerprintGeneratorTest @Test public void testGenerateFingerprint_precalculatedHash() { - RowSignature rowSignature = RowSignature.builder().add("c1", ColumnType.FLOAT).build(); + RowSignature rowSignature = + RowSignature.builder() + .add("c1", ColumnType.LONG) + .add("c0", ColumnType.STRING) + .add("c2", ColumnType.FLOAT) + .add("c3", ColumnType.DOUBLE) + .build(); Map aggregatorFactoryMap = new HashMap<>(); - aggregatorFactoryMap.put("longFirst", new LongFirstAggregatorFactory("longFirst", "long-col", null)); + aggregatorFactoryMap.put("longFirst", new LongFirstAggregatorFactory("longFirst", "c1", null)); + aggregatorFactoryMap.put("stringAny", new StringAnyAggregatorFactory("stringAny", "c0", 1024, true)); SchemaPayload schemaPayload = new SchemaPayload(rowSignature, aggregatorFactoryMap); - String expected = "DEE5E8F59833102F0FA5B10F8B8884EA15220D1D2A5F6097A93D8309132E1039"; + String expected = "82E774457D26D0B8D481B6C39872070B25EA3C72C6EFC107B346FA42641740E1"; Assert.assertEquals(expected, fingerprintGenerator.generateFingerprint(schemaPayload, "ds", 0)); } @@ -60,25 +70,38 @@ public class FingerprintGeneratorTest { RowSignature rowSignature = RowSignature.builder() - .add("c1", ColumnType.FLOAT) .add("c2", ColumnType.LONG) + .add("c1", ColumnType.FLOAT) .add("c3", ColumnType.DOUBLE) + .add("c0", ColumnType.STRING) .build(); Map aggregatorFactoryMap = new HashMap<>(); - aggregatorFactoryMap.put("longFirst", new LongFirstAggregatorFactory("longFirst", "long-col", null)); + aggregatorFactoryMap.put("longFirst", new LongFirstAggregatorFactory("longFirst", "c2", null)); + aggregatorFactoryMap.put("stringAny", new StringAnyAggregatorFactory("stringAny", "c0", 1024, true)); SchemaPayload schemaPayload = new SchemaPayload(rowSignature, aggregatorFactoryMap); RowSignature rowSignaturePermutation = RowSignature.builder() .add("c2", ColumnType.LONG) + .add("c0", ColumnType.STRING) .add("c3", ColumnType.DOUBLE) .add("c1", ColumnType.FLOAT) .build(); - SchemaPayload schemaPayloadNew = new SchemaPayload(rowSignaturePermutation, aggregatorFactoryMap); - Assert.assertNotEquals( + Map aggregatorFactoryMapForPermutation = new HashMap<>(); + aggregatorFactoryMapForPermutation.put( + "stringAny", + new StringAnyAggregatorFactory("stringAny", "c0", 1024, true) + ); + aggregatorFactoryMapForPermutation.put( + "longFirst", + new LongFirstAggregatorFactory("longFirst", "c2", null) + ); + + SchemaPayload schemaPayloadNew = new SchemaPayload(rowSignaturePermutation, aggregatorFactoryMapForPermutation); + Assert.assertEquals( fingerprintGenerator.generateFingerprint(schemaPayload, "ds", 0), fingerprintGenerator.generateFingerprint(schemaPayloadNew, "ds", 0) ); @@ -125,4 +148,29 @@ public class FingerprintGeneratorTest fingerprintGenerator.generateFingerprint(schemaPayload, "ds", 1) ); } + + @Test + public void testRowSignatureIsSorted() + { + RowSignature rowSignature = + RowSignature.builder() + .add("c5", ColumnType.STRING) + .add("c1", ColumnType.FLOAT) + .add("b2", ColumnType.LONG) + .add("d3", ColumnType.DOUBLE) + .add("a1", ColumnType.STRING) + .build(); + + RowSignature sortedSignature = fingerprintGenerator.getLexicographicallySortedSignature(rowSignature); + + Assert.assertNotEquals(rowSignature, sortedSignature); + + List columnNames = sortedSignature.getColumnNames(); + List sortedOrder = Arrays.asList("a1", "b2", "c1", "c5", "d3"); + Assert.assertEquals(sortedOrder, columnNames); + + for (String column : sortedOrder) { + Assert.assertEquals(sortedSignature.getColumnType(column), rowSignature.getColumnType(column)); + } + } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java b/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java index 38c3fda1e7e..79f31c7c814 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java @@ -21,12 +21,12 @@ package org.apache.druid.segment.realtime; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; @@ -197,9 +197,9 @@ public class FireHydrantTest extends InitializedNullHandlingTest } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return incrementalIndexSegment.asStorageAdapter(); + return incrementalIndexSegment.asCursorFactory(); } @Override diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java index 22034aa33aa..5f43236e075 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java @@ -30,7 +30,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.core.NoopEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.IndexIO; @@ -151,19 +150,18 @@ public class BatchAppenderatorTester implements AutoCloseable Map.class ); - schema = new DataSchema( - DATASOURCE, - null, - null, - new AggregatorFactory[]{ - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("met", "met") - }, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null, - parserMap, - objectMapper - ); + schema = DataSchema.builder() + .withDataSource(DATASOURCE) + .withAggregators( + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("met", "met") + ) + .withGranularity( + new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null) + ) + .withParserMap(parserMap) + .withObjectMapper(objectMapper) + .build(); tuningConfig = new TestAppenderatorConfig( TuningConfig.DEFAULT_APPENDABLE_INDEX, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java index cc0dc1fad1b..29d758aaed0 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.realtime.appenderator; +import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; @@ -29,6 +30,8 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JSONParseSpec; import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.guice.BuiltInTypesModule; +import org.apache.druid.jackson.AggregatorsModule; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.concurrent.Execs; @@ -36,13 +39,14 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.core.NoopEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; import org.apache.druid.query.ForwardingQueryProcessingPool; import org.apache.druid.query.QueryRunnerTestHelper; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanQueryConfig; import org.apache.druid.query.scan.ScanQueryEngine; @@ -110,6 +114,13 @@ public class StreamAppenderatorTester implements AutoCloseable { objectMapper = new DefaultObjectMapper(); objectMapper.registerSubtypes(LinearShardSpec.class); + objectMapper.registerModules(new AggregatorsModule()); + objectMapper.registerModules(new BuiltInTypesModule().getJacksonModules()); + objectMapper.setInjectableValues( + new InjectableValues.Std() + .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) + .addValue(ObjectMapper.class.getName(), objectMapper) + ); final Map parserMap = objectMapper.convertValue( new MapInputRowParser( @@ -123,17 +134,16 @@ public class StreamAppenderatorTester implements AutoCloseable ), Map.class ); - schema = new DataSchema( - DATASOURCE, - parserMap, - new AggregatorFactory[]{ - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("met", "met") - }, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null, - objectMapper - ); + schema = DataSchema.builder() + .withDataSource(DATASOURCE) + .withParserMap(parserMap) + .withAggregators( + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("met", "met") + ) + .withGranularity(new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null)) + .withObjectMapper(objectMapper) + .build(); tuningConfig = new TestAppenderatorConfig( TuningConfig.DEFAULT_APPENDABLE_INDEX, maxRowsInMemory, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java index 23ac93db009..21f627baa08 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java @@ -98,14 +98,11 @@ public class UnifiedIndexerAppenderatorsManagerTest extends InitializedNullHandl EasyMock.replay(appenderatorConfig); appenderator = manager.createBatchAppenderatorForTask( "taskId", - new DataSchema( - "myDataSource", - new TimestampSpec("__time", "millis", null), - null, - null, - new UniformGranularitySpec(Granularities.HOUR, Granularities.HOUR, false, Collections.emptyList()), - null - ), + DataSchema.builder() + .withDataSource("myDataSource") + .withTimestamp(new TimestampSpec("__time", "millis", null)) + .withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.HOUR, false, Collections.emptyList())) + .build(), appenderatorConfig, new SegmentGenerationMetrics(), new NoopDataSegmentPusher(), diff --git a/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java b/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java index 9d85ec6c8e6..750ea06c653 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java @@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.RowAdapters; import org.apache.druid.segment.RowBasedSegment; @@ -76,14 +75,14 @@ public class SinkTest extends InitializedNullHandlingTest @Test public void testSwap() throws Exception { - final DataSchema schema = new DataSchema( - "test", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null), - null - ); + final DataSchema schema = + DataSchema.builder() + .withDataSource("test") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null)) + .build(); final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final String version = DateTimes.nowUtc().toString(); @@ -256,18 +255,17 @@ public class SinkTest extends InitializedNullHandlingTest @Test public void testGetSinkSignature() throws IndexSizeExceededException { - final DataSchema schema = new DataSchema( - "test", - new TimestampSpec(null, null, null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new LongDimensionSchema("dimLong") - )), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null), - null - ); + final DataSchema schema = + DataSchema.builder() + .withDataSource("test") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("dimLong") + ) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null)) + .build(); final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final String version = DateTimes.nowUtc().toString(); diff --git a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java index 467f375f9f7..fa5585c374e 100644 --- a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java +++ b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java @@ -52,6 +52,7 @@ import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChestTestHelper; import org.apache.druid.query.ResourceLimitExceededException; +import org.apache.druid.query.ResultSerializationMode; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.UnionDataSource; @@ -1721,7 +1722,8 @@ public class ClientQuerySegmentWalkerTest .put(GroupByQueryConfig.CTX_KEY_APPLY_LIMIT_PUSH_DOWN, true) .put(GroupingEngine.CTX_KEY_OUTERMOST, true) .put(GroupingEngine.CTX_KEY_FUDGE_TIMESTAMP, "1979") - .put(QueryContexts.QUERY_RESOURCE_ID, "dummy"); + .put(QueryContexts.QUERY_RESOURCE_ID, "dummy") + .put(ResultSerializationMode.CTX_SERIALIZATION_PARAMETER, "blast"); modifiedQuery = query.withOverriddenContext(contextBuilder.build()); diff --git a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java index 43bce13e2c8..32c26edffee 100644 --- a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java @@ -19,6 +19,7 @@ package org.apache.druid.server; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; @@ -80,12 +81,14 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.server.security.Resource; import org.apache.http.HttpStatus; +import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -98,6 +101,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -424,7 +428,8 @@ public class QueryResourceTest overrideConfig, new AuthConfig(), System.currentTimeMillis(), - System.nanoTime()) + System.nanoTime() + ) { @Override public void emitLogsAndMetrics(@Nullable Throwable e, @Nullable String remoteAddress, long bytesWritten) @@ -453,7 +458,8 @@ public class QueryResourceTest entity.getUnderlyingException(), new DruidExceptionMatcher( DruidException.Persona.OPERATOR, - DruidException.Category.RUNTIME_FAILURE, "legacyQueryException") + DruidException.Category.RUNTIME_FAILURE, "legacyQueryException" + ) .expectMessageIs("something") ); } @@ -1190,7 +1196,7 @@ public class QueryResourceTest final CountDownLatch waitOneScheduled = new CountDownLatch(1); final QueryScheduler scheduler = new QueryScheduler( 40, - new ThresholdBasedQueryPrioritizationStrategy(null, "P90D", null, null), + new ThresholdBasedQueryPrioritizationStrategy(null, "P90D", null, null, null), new HiLoQueryLaningStrategy(1), new ServerConfig() ); @@ -1250,6 +1256,46 @@ public class QueryResourceTest } } + @Test + public void testNativeQueryWriter_goodResponse() throws IOException + { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final QueryResultPusher.Writer writer = new QueryResource.NativeQueryWriter(jsonMapper, baos); + writer.writeResponseStart(); + writer.writeRow(Arrays.asList("foo", "bar")); + writer.writeRow(Collections.singletonList("baz")); + writer.writeResponseEnd(); + writer.close(); + + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of("foo", "bar"), + ImmutableList.of("baz") + ), + jsonMapper.readValue(baos.toByteArray(), Object.class) + ); + } + + @Test + public void testNativeQueryWriter_truncatedResponse() throws IOException + { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final QueryResultPusher.Writer writer = new QueryResource.NativeQueryWriter(jsonMapper, baos); + writer.writeResponseStart(); + writer.writeRow(Arrays.asList("foo", "bar")); + writer.close(); // Simulate an error that occurs midstream; close writer without calling writeResponseEnd. + + final JsonProcessingException e = Assert.assertThrows( + JsonProcessingException.class, + () -> jsonMapper.readValue(baos.toByteArray(), Object.class) + ); + + MatcherAssert.assertThat( + e, + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("expected close marker for Array")) + ); + } + private void createScheduledQueryResource( QueryScheduler scheduler, Collection beforeScheduler, diff --git a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java index d7af61370be..b670cd4dd78 100644 --- a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java +++ b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java @@ -553,7 +553,7 @@ public class QuerySchedulerTest Assert.assertEquals( "Unable to provision, see the following errors:\n" + "\n" - + "1) Problem parsing object at prefix[druid.query.scheduler]: Cannot construct instance of `org.apache.druid.server.scheduling.ThresholdBasedQueryPrioritizationStrategy`, problem: periodThreshold, durationThreshold, or segmentCountThreshold must be set\n" + + "1) Problem parsing object at prefix[druid.query.scheduler]: Cannot construct instance of `org.apache.druid.server.scheduling.ThresholdBasedQueryPrioritizationStrategy`, problem: periodThreshold, durationThreshold, segmentCountThreshold or segmentRangeThreshold must be set\n" + " at [Source: UNKNOWN; line: -1, column: -1] (through reference chain: org.apache.druid.server.QuerySchedulerProvider[\"prioritization\"]).\n" + "\n" + "1 error", diff --git a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java index 07ed8b17e6b..55a882c3218 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerThreadSafetyTest.java @@ -31,10 +31,11 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.CursorFactory; +import org.apache.druid.segment.PhysicalSegmentInspector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; @@ -246,7 +247,7 @@ public class SegmentManagerThreadSafetyTest { return new Segment() { - StorageAdapter storageAdapter = Mockito.mock(StorageAdapter.class); + PhysicalSegmentInspector rowCountInspector = Mockito.mock(PhysicalSegmentInspector.class); @Override public SegmentId getId() @@ -268,15 +269,18 @@ public class SegmentManagerThreadSafetyTest } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - Mockito.when(storageAdapter.getNumRows()).thenReturn(1); - return storageAdapter; + throw new UnsupportedOperationException(); } @Override public T as(Class clazz) { + if (PhysicalSegmentInspector.class.equals(clazz)) { + Mockito.when(rowCountInspector.getNumRows()).thenReturn(1); + return (T) rowCountInspector; + } return null; } diff --git a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java index 866381dd70c..7cc61d94094 100644 --- a/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java +++ b/server/src/test/java/org/apache/druid/server/TestSegmentUtils.java @@ -28,18 +28,15 @@ import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.OrderBy; -import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.CursorBuildSpec; -import org.apache.druid.segment.CursorHolder; +import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.Cursors; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.Metadata; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; -import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.loading.LoadSpec; @@ -242,9 +239,9 @@ public class TestSegmentUtils } @Override - public StorageAdapter asStorageAdapter() + public CursorFactory asCursorFactory() { - return makeFakeStorageAdapter(interval, 0); + return new QueryableIndexCursorFactory(index); } @Override @@ -254,85 +251,6 @@ public class TestSegmentUtils closed = true; } } - - private StorageAdapter makeFakeStorageAdapter(Interval interval, int cardinality) - { - StorageAdapter adapter = new StorageAdapter() - { - @Override - public Interval getInterval() - { - return interval; - } - - @Override - public int getDimensionCardinality(String column) - { - return cardinality; - } - - @Override - public Indexed getAvailableDimensions() - { - return null; - } - - @Override - public Iterable getAvailableMetrics() - { - return null; - } - - @Nullable - @Override - public Comparable getMinValue(String column) - { - return null; - } - - @Nullable - @Override - public Comparable getMaxValue(String column) - { - return null; - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String column) - { - return null; - } - - @Override - public int getNumRows() - { - return 0; - } - - @Override - public Metadata getMetadata() - { - return null; - } - - @Override - public CursorHolder makeCursorHolder(CursorBuildSpec spec) - { - return new CursorHolder() - { - @Nullable - @Override - public Cursor asCursor() - { - return null; - } - }; - } - }; - - return adapter; - } } public static DataSegment makeSegment(String dataSource, String version, Interval interval) diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index c4177f186c7..41d789734ca 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java @@ -162,7 +162,7 @@ public class CompactionStatusTest } @Test - public void testFindPartitionsSpecWhenGivenIsRange() + public void testFindPartitionsSpecWhenGivenIsRangeWithMaxRows() { final PartitionsSpec partitionsSpec = new DimensionRangePartitionsSpec(null, 10000, Collections.singletonList("dim"), false); @@ -174,6 +174,19 @@ public class CompactionStatusTest ); } + @Test + public void testFindPartitionsSpecWhenGivenIsRangeWithTargetRows() + { + final PartitionsSpec partitionsSpec = + new DimensionRangePartitionsSpec(10000, null, Collections.singletonList("dim"), false); + final ClientCompactionTaskQueryTuningConfig tuningConfig + = ClientCompactionTaskQueryTuningConfig.from(createCompactionConfig(partitionsSpec)); + Assert.assertEquals( + new DimensionRangePartitionsSpec(null, 15000, Collections.singletonList("dim"), false), + CompactionStatus.findPartitionsSpecFromConfig(tuningConfig) + ); + } + @Test public void testStatusWhenLastCompactionStateIsNull() { diff --git a/server/src/test/java/org/apache/druid/server/coordination/BroadcastDatasourceLoadingSpecTest.java b/server/src/test/java/org/apache/druid/server/coordination/BroadcastDatasourceLoadingSpecTest.java new file mode 100644 index 00000000000..ddec0901965 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordination/BroadcastDatasourceLoadingSpecTest.java @@ -0,0 +1,166 @@ +/* + * 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.server.coordination; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import junitparams.JUnitParamsRunner; +import junitparams.Parameters; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.StringUtils; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; + +import java.util.Arrays; +import java.util.Set; + +@RunWith(JUnitParamsRunner.class) +public class BroadcastDatasourceLoadingSpecTest +{ + @Test + public void testLoadingAllBroadcastDatasources() + { + final BroadcastDatasourceLoadingSpec spec = BroadcastDatasourceLoadingSpec.ALL; + Assert.assertEquals(BroadcastDatasourceLoadingSpec.Mode.ALL, spec.getMode()); + Assert.assertNull(spec.getBroadcastDatasourcesToLoad()); + } + + @Test + public void testLoadingNoLookups() + { + final BroadcastDatasourceLoadingSpec spec = BroadcastDatasourceLoadingSpec.NONE; + Assert.assertEquals(BroadcastDatasourceLoadingSpec.Mode.NONE, spec.getMode()); + Assert.assertNull(spec.getBroadcastDatasourcesToLoad()); + } + + @Test + public void testLoadingOnlyRequiredLookups() + { + final Set broadcastDatasourcesToLoad = ImmutableSet.of("ds1", "ds2"); + final BroadcastDatasourceLoadingSpec spec = BroadcastDatasourceLoadingSpec.loadOnly(ImmutableSet.of("ds1", "ds2")); + Assert.assertEquals(BroadcastDatasourceLoadingSpec.Mode.ONLY_REQUIRED, spec.getMode()); + Assert.assertEquals(broadcastDatasourcesToLoad, spec.getBroadcastDatasourcesToLoad()); + } + + @Test + public void testLoadingOnlyRequiredLookupsWithNullList() + { + DruidException exception = Assert.assertThrows(DruidException.class, () -> BroadcastDatasourceLoadingSpec.loadOnly(null)); + Assert.assertEquals("Expected non-null set of broadcast datasources to load.", exception.getMessage()); + } + + @Test + public void testCreateBroadcastLoadingSpecFromNullContext() + { + // Default spec is returned in the case of context=null. + Assert.assertEquals( + BroadcastDatasourceLoadingSpec.NONE, + BroadcastDatasourceLoadingSpec.createFromContext( + null, + BroadcastDatasourceLoadingSpec.NONE + ) + ); + + Assert.assertEquals( + BroadcastDatasourceLoadingSpec.ALL, + BroadcastDatasourceLoadingSpec.createFromContext( + null, + BroadcastDatasourceLoadingSpec.ALL + ) + ); + } + + @Test + public void testCreateBroadcastLoadingSpecFromContext() + { + // Only required lookups are returned in the case of context having the lookup keys. + Assert.assertEquals( + BroadcastDatasourceLoadingSpec.loadOnly(ImmutableSet.of("ds1", "ds2")), + BroadcastDatasourceLoadingSpec.createFromContext( + ImmutableMap.of( + BroadcastDatasourceLoadingSpec.CTX_BROADCAST_DATASOURCES_TO_LOAD, Arrays.asList("ds1", "ds2"), + BroadcastDatasourceLoadingSpec.CTX_BROADCAST_DATASOURCE_LOADING_MODE, BroadcastDatasourceLoadingSpec.Mode.ONLY_REQUIRED + ), + BroadcastDatasourceLoadingSpec.ALL + ) + ); + + // No lookups are returned in the case of context having mode=NONE, irrespective of the default spec. + Assert.assertEquals( + BroadcastDatasourceLoadingSpec.NONE, + BroadcastDatasourceLoadingSpec.createFromContext( + ImmutableMap.of( + BroadcastDatasourceLoadingSpec.CTX_BROADCAST_DATASOURCE_LOADING_MODE, BroadcastDatasourceLoadingSpec.Mode.NONE), + BroadcastDatasourceLoadingSpec.ALL + ) + ); + + // All lookups are returned in the case of context having mode=ALL, irrespective of the default spec. + Assert.assertEquals( + BroadcastDatasourceLoadingSpec.ALL, + BroadcastDatasourceLoadingSpec.createFromContext( + ImmutableMap.of(BroadcastDatasourceLoadingSpec.CTX_BROADCAST_DATASOURCE_LOADING_MODE, BroadcastDatasourceLoadingSpec.Mode.ALL), + BroadcastDatasourceLoadingSpec.NONE + ) + ); + } + + @Test + @Parameters( + { + "NONE1", + "A", + "Random mode", + "all", + "only required", + "none" + } + ) + public void testSpecFromInvalidModeInContext(final String mode) + { + final DruidException exception = Assert.assertThrows(DruidException.class, () -> BroadcastDatasourceLoadingSpec.createFromContext( + ImmutableMap.of(BroadcastDatasourceLoadingSpec.CTX_BROADCAST_DATASOURCE_LOADING_MODE, mode), BroadcastDatasourceLoadingSpec.ALL)); + Assert.assertEquals(StringUtils.format("Invalid value of %s[%s]. Allowed values are [ALL, NONE, ONLY_REQUIRED]", + BroadcastDatasourceLoadingSpec.CTX_BROADCAST_DATASOURCE_LOADING_MODE, mode), exception.getMessage()); + } + + + @Test + @Parameters( + { + "foo bar", + "foo]" + } + ) + public void testSpecFromInvalidBroadcastDatasourcesInContext(final Object lookupsToLoad) + { + final DruidException exception = Assert.assertThrows(DruidException.class, () -> + BroadcastDatasourceLoadingSpec.createFromContext( + ImmutableMap.of( + BroadcastDatasourceLoadingSpec.CTX_BROADCAST_DATASOURCES_TO_LOAD, lookupsToLoad, + BroadcastDatasourceLoadingSpec.CTX_BROADCAST_DATASOURCE_LOADING_MODE, BroadcastDatasourceLoadingSpec.Mode.ONLY_REQUIRED), + BroadcastDatasourceLoadingSpec.ALL) + ); + Assert.assertEquals(StringUtils.format("Invalid value of %s[%s]. Please provide a comma-separated list of " + + "broadcast datasource names. For example: [\"datasourceName1\", \"datasourceName2\"]", + BroadcastDatasourceLoadingSpec.CTX_BROADCAST_DATASOURCES_TO_LOAD, lookupsToLoad), exception.getMessage()); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java index 7629a6b875c..187725317a2 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java @@ -36,6 +36,7 @@ import org.apache.druid.segment.loading.StorageLocation; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.TestSegmentUtils; +import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Before; @@ -137,7 +138,8 @@ public class SegmentBootstrapperCacheTest segmentManager, new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, - emitter + emitter, + new DataSourceTaskIdHolder() ); bootstrapper.start(); @@ -164,7 +166,8 @@ public class SegmentBootstrapperCacheTest segmentManager, new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, - emitter + emitter, + new DataSourceTaskIdHolder() ); bootstrapper.start(); @@ -204,7 +207,8 @@ public class SegmentBootstrapperCacheTest segmentManager, new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, - emitter + emitter, + new DataSourceTaskIdHolder() ); bootstrapper.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperTest.java index c41763f1824..fe1424e2700 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperTest.java @@ -20,13 +20,23 @@ package org.apache.druid.server.coordination; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Scopes; +import com.google.inject.name.Names; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.ServerTypeConfig; +import org.apache.druid.jackson.JacksonModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Before; @@ -125,7 +135,8 @@ public class SegmentBootstrapperTest segmentManager, new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, - serviceEmitter + serviceEmitter, + new DataSourceTaskIdHolder() ); Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); @@ -184,7 +195,8 @@ public class SegmentBootstrapperTest segmentManager, new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, - serviceEmitter + serviceEmitter, + new DataSourceTaskIdHolder() ); Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); @@ -240,7 +252,8 @@ public class SegmentBootstrapperTest segmentManager, new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, - serviceEmitter + serviceEmitter, + new DataSourceTaskIdHolder() ); Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); @@ -267,6 +280,129 @@ public class SegmentBootstrapperTest bootstrapper.stop(); } + @Test + public void testLoadNoBootstrapSegments() throws Exception + { + final Set segments = new HashSet<>(); + for (int i = 0; i < COUNT; ++i) { + segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-01"))); + segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-02"))); + segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-01"))); + segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-02"))); + } + + Injector injector = Guice.createInjector( + new JacksonModule(), + new LifecycleModule(), + binder -> { + binder.bindScope(LazySingleton.class, Scopes.SINGLETON); + final BroadcastDatasourceLoadingSpec broadcastMode = BroadcastDatasourceLoadingSpec.NONE; + binder.bind(Key.get(BroadcastDatasourceLoadingSpec.class, Names.named(DataSourceTaskIdHolder.BROADCAST_DATASOURCES_TO_LOAD_FOR_TASK))) + .toInstance(broadcastMode); + } + ); + + final TestCoordinatorClient coordinatorClient = new TestCoordinatorClient(segments); + final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(); + final SegmentManager segmentManager = new SegmentManager(cacheManager); + final SegmentLoadDropHandler handler = new SegmentLoadDropHandler( + segmentLoaderConfig, + segmentAnnouncer, + segmentManager + ); + final SegmentBootstrapper bootstrapper = new SegmentBootstrapper( + handler, + segmentLoaderConfig, + segmentAnnouncer, + serverAnnouncer, + segmentManager, + new ServerTypeConfig(ServerType.HISTORICAL), + coordinatorClient, + serviceEmitter, + injector.getInstance(DataSourceTaskIdHolder.class) + ); + + Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); + + bootstrapper.start(); + + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); + Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); + + final ImmutableList expectedBootstrapSegments = ImmutableList.of(); + + Assert.assertEquals(expectedBootstrapSegments, segmentAnnouncer.getObservedSegments()); + + Assert.assertEquals(expectedBootstrapSegments, cacheManager.getObservedBootstrapSegments()); + Assert.assertEquals(expectedBootstrapSegments, cacheManager.getObservedBootstrapSegmentsLoadedIntoPageCache()); + + bootstrapper.stop(); + } + + @Test + public void testLoadOnlyRequiredBootstrapSegments() throws Exception + { + final Set segments = new HashSet<>(); + final DataSegment ds1Segment1 = makeSegment("test1", "1", Intervals.of("P1D/2011-04-01")); + final DataSegment ds1Segment2 = makeSegment("test1", "1", Intervals.of("P1D/2012-04-01")); + final DataSegment ds2Segment1 = makeSegment("test2", "1", Intervals.of("P1d/2011-04-01")); + final DataSegment ds2Segment2 = makeSegment("test2", "1", Intervals.of("P1d/2012-04-01")); + segments.add(ds1Segment1); + segments.add(ds1Segment2); + segments.add(ds2Segment1); + segments.add(ds2Segment2); + + Injector injector = Guice.createInjector( + new JacksonModule(), + new LifecycleModule(), + binder -> { + binder.bindScope(LazySingleton.class, Scopes.SINGLETON); + final BroadcastDatasourceLoadingSpec broadcastMode = BroadcastDatasourceLoadingSpec.loadOnly(ImmutableSet.of("test1")); + binder.bind(Key.get(BroadcastDatasourceLoadingSpec.class, Names.named(DataSourceTaskIdHolder.BROADCAST_DATASOURCES_TO_LOAD_FOR_TASK))) + .toInstance(broadcastMode); + } + ); + + final TestCoordinatorClient coordinatorClient = new TestCoordinatorClient(segments); + final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(); + final SegmentManager segmentManager = new SegmentManager(cacheManager); + final SegmentLoadDropHandler handler = new SegmentLoadDropHandler( + segmentLoaderConfig, + segmentAnnouncer, + segmentManager + ); + final SegmentBootstrapper bootstrapper = new SegmentBootstrapper( + handler, + segmentLoaderConfig, + segmentAnnouncer, + serverAnnouncer, + segmentManager, + new ServerTypeConfig(ServerType.HISTORICAL), + coordinatorClient, + serviceEmitter, + injector.getInstance(DataSourceTaskIdHolder.class) + ); + + Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); + + bootstrapper.start(); + + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); + Assert.assertFalse(segmentManager.getDataSourceCounts().isEmpty()); + Assert.assertEquals(ImmutableSet.of("test1"), segmentManager.getDataSourceNames()); + + final ImmutableList expectedBootstrapSegments = ImmutableList.of(ds1Segment2, ds1Segment1); + + Assert.assertEquals(expectedBootstrapSegments, segmentAnnouncer.getObservedSegments()); + + Assert.assertEquals(expectedBootstrapSegments, cacheManager.getObservedBootstrapSegments()); + Assert.assertEquals(expectedBootstrapSegments, cacheManager.getObservedBootstrapSegmentsLoadedIntoPageCache()); + serviceEmitter.verifyValue("segment/bootstrap/count", expectedBootstrapSegments.size()); + serviceEmitter.verifyEmitted("segment/bootstrap/time", 1); + + bootstrapper.stop(); + } + @Test public void testLoadBootstrapSegmentsWhenExceptionThrown() throws Exception { @@ -285,7 +421,8 @@ public class SegmentBootstrapperTest segmentManager, new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, - serviceEmitter + serviceEmitter, + new DataSourceTaskIdHolder() ); Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); diff --git a/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java index ec4a008c1ed..69a7c08f2be 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ServerManagerTest.java @@ -77,7 +77,6 @@ import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; -import org.apache.druid.segment.join.JoinableFactoryWrapperTest; import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; @@ -208,7 +207,6 @@ public class ServerManagerTest new LocalCacheProvider().get(), new CacheConfig(), segmentManager, - JoinableFactoryWrapperTest.NOOP_JOINABLE_FACTORY_WRAPPER, new ServerConfig() ); diff --git a/server/src/test/java/org/apache/druid/server/metrics/MetricsModuleTest.java b/server/src/test/java/org/apache/druid/server/metrics/MetricsModuleTest.java index 74899716862..f4563eb8e5b 100644 --- a/server/src/test/java/org/apache/druid/server/metrics/MetricsModuleTest.java +++ b/server/src/test/java/org/apache/druid/server/metrics/MetricsModuleTest.java @@ -47,6 +47,7 @@ import org.apache.druid.java.util.metrics.MonitorScheduler; import org.apache.druid.java.util.metrics.NoopOshiSysMonitor; import org.apache.druid.java.util.metrics.NoopSysMonitor; import org.apache.druid.java.util.metrics.OshiSysMonitor; +import org.apache.druid.java.util.metrics.OshiSysMonitorConfig; import org.apache.druid.java.util.metrics.SysMonitor; import org.apache.druid.server.DruidNode; import org.hamcrest.CoreMatchers; @@ -212,6 +213,24 @@ public class MetricsModuleTest Assert.assertTrue(sysMonitor instanceof NoopOshiSysMonitor); Mockito.verify(emitter, Mockito.never()).emit(ArgumentMatchers.any(ServiceEventBuilder.class)); } + + @Test + public void testGetOshiSysMonitorViaInjectorBroker() + { + Properties properties = new Properties(); + properties.setProperty("druid.monitoring.sys.categories", "[\"mem\"]"); + final Injector injector = createInjector(properties, ImmutableSet.of(NodeRole.BROKER)); + final OshiSysMonitor sysMonitor = injector.getInstance(OshiSysMonitor.class); + final ServiceEmitter emitter = Mockito.mock(ServiceEmitter.class); + sysMonitor.doMonitor(emitter); + + Assert.assertTrue(sysMonitor instanceof OshiSysMonitor); + Mockito.verify(emitter, Mockito.atLeastOnce()).emit(ArgumentMatchers.any(ServiceEventBuilder.class)); + + Assert.assertTrue(injector.getInstance(OshiSysMonitorConfig.class).shouldEmitMetricCategory("mem")); + Assert.assertFalse(injector.getInstance(OshiSysMonitorConfig.class).shouldEmitMetricCategory("swap")); + } + @Test public void testGetOshiSysMonitorWhenNull() { diff --git a/server/src/test/java/org/apache/druid/server/scheduling/ThresholdBasedQueryPrioritizationStrategyTest.java b/server/src/test/java/org/apache/druid/server/scheduling/ThresholdBasedQueryPrioritizationStrategyTest.java index 705337feae2..1cc46783d03 100644 --- a/server/src/test/java/org/apache/druid/server/scheduling/ThresholdBasedQueryPrioritizationStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/scheduling/ThresholdBasedQueryPrioritizationStrategyTest.java @@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.Druids; import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.server.QueryPrioritizationStrategy; @@ -59,7 +60,7 @@ public class ThresholdBasedQueryPrioritizationStrategyTest public void testPrioritizationPeriodThresholdInsidePeriod() { QueryPrioritizationStrategy strategy = new ThresholdBasedQueryPrioritizationStrategy( - "P90D", null, null, adjustment); + "P90D", null, null, null, adjustment); DateTime startDate = DateTimes.nowUtc().minusDays(1); DateTime endDate = DateTimes.nowUtc(); TimeseriesQuery query = queryBuilder.intervals(ImmutableList.of(new Interval(startDate, endDate))) @@ -79,6 +80,7 @@ public class ThresholdBasedQueryPrioritizationStrategyTest "P90D", null, null, + null, adjustment ); DateTime startDate = DateTimes.nowUtc().minusDays(100); @@ -101,6 +103,7 @@ public class ThresholdBasedQueryPrioritizationStrategyTest null, "P7D", null, + null, adjustment ); DateTime startDate = DateTimes.nowUtc().minusDays(1); @@ -122,6 +125,7 @@ public class ThresholdBasedQueryPrioritizationStrategyTest null, "P7D", null, + null, adjustment ); DateTime startDate = DateTimes.nowUtc().minusDays(20); @@ -144,6 +148,7 @@ public class ThresholdBasedQueryPrioritizationStrategyTest null, null, 2, + null, adjustment ); DateTime startDate = DateTimes.nowUtc().minusDays(1); @@ -168,6 +173,7 @@ public class ThresholdBasedQueryPrioritizationStrategyTest null, null, 2, + null, adjustment ); DateTime startDate = DateTimes.nowUtc().minusDays(20); @@ -189,4 +195,53 @@ public class ThresholdBasedQueryPrioritizationStrategyTest ).get() ); } + + @Test + public void testPrioritizationSegmentRangeWithinThreshold() + { + QueryPrioritizationStrategy strategy = new ThresholdBasedQueryPrioritizationStrategy( + null, + null, + null, + "P7D", + adjustment + ); + DateTime startDate = DateTimes.nowUtc().minusDays(1); + DateTime endDate = DateTimes.nowUtc(); + TimeseriesQuery query = queryBuilder.intervals(ImmutableList.of(new Interval(startDate, endDate))) + .granularity(Granularities.MINUTE) + .context(ImmutableMap.of()) + .build(); + SegmentServerSelector segmentServerSelector = EasyMock.createMock(SegmentServerSelector.class); + EasyMock.expect(segmentServerSelector.getSegmentDescriptor()).andReturn(new SegmentDescriptor(new Interval(startDate, endDate), "", 0)).times(2); + EasyMock.replay(segmentServerSelector); + Assert.assertFalse( + strategy.computePriority(QueryPlus.wrap(query), ImmutableSet.of(segmentServerSelector)).isPresent() + ); + } + + @Test + public void testPrioritizationSegmentRangeOverThreshold() + { + QueryPrioritizationStrategy strategy = new ThresholdBasedQueryPrioritizationStrategy( + null, + null, + null, + "P7D", + adjustment + ); + DateTime startDate = DateTimes.nowUtc().minusDays(20); + DateTime endDate = DateTimes.nowUtc(); + TimeseriesQuery query = queryBuilder.intervals(ImmutableList.of(new Interval(startDate, endDate))) + .granularity(Granularities.HOUR) + .context(ImmutableMap.of()) + .build(); + SegmentServerSelector segmentServerSelector = EasyMock.createMock(SegmentServerSelector.class); + EasyMock.expect(segmentServerSelector.getSegmentDescriptor()).andReturn(new SegmentDescriptor(new Interval(startDate, endDate), "", 0)).times(2); + EasyMock.replay(segmentServerSelector); + Assert.assertEquals( + -adjustment, + (int) strategy.computePriority(QueryPlus.wrap(query), ImmutableSet.of(segmentServerSelector)).get() + ); + } } diff --git a/services/pom.xml b/services/pom.xml index 514de6de7e4..83c109b5aba 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT @@ -56,6 +56,12 @@ org.apache.druid druid-indexing-service ${project.parent.version} + + + org.eclipse.aether + aether-api + +
    org.apache.druid @@ -171,10 +177,6 @@ io.netty netty-common - - org.eclipse.aether - aether-api - javax.servlet javax.servlet-api @@ -187,17 +189,49 @@ com.google.guava guava - - org.eclipse.aether - aether-util - com.google.inject.extensions guice-servlet - io.tesla.aether - tesla-aether + org.apache.maven.resolver + maven-resolver-connector-basic + 1.3.1 + + + org.apache.maven.resolver + maven-resolver-transport-http + 1.3.1 + + + org.apache.maven.resolver + maven-resolver-util + 1.3.1 + + + org.apache.maven.resolver + maven-resolver-impl + 1.3.1 + + + org.apache.maven.resolver + maven-resolver-spi + 1.3.1 + + + org.apache.maven.resolver + maven-resolver-api + 1.3.1 + + + org.apache.maven + maven-artifact + 3.6.0 + + + org.apache.maven + maven-resolver-provider + 3.6.0 javax.xml.bind diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index eb572850cda..15374625d30 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -47,7 +47,6 @@ import org.apache.druid.curator.ZkEnablementConfig; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.Binders; import org.apache.druid.guice.CacheModule; -import org.apache.druid.guice.DruidProcessingModule; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceTaskLogsModule; import org.apache.druid.guice.IndexingServiceTuningConfigModule; @@ -58,6 +57,7 @@ import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.ManageLifecycleServer; +import org.apache.druid.guice.PeonProcessingModule; import org.apache.druid.guice.PolyBind; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; @@ -123,6 +123,7 @@ import org.apache.druid.segment.realtime.appenderator.PeonAppenderatorsManager; import org.apache.druid.server.DruidNode; import org.apache.druid.server.ResponseContextConfig; import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.coordination.BroadcastDatasourceLoadingSpec; import org.apache.druid.server.coordination.SegmentBootstrapper; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordination.ZkCoordinator; @@ -176,12 +177,26 @@ public class CliPeon extends GuiceRunnable private boolean isZkEnabled = true; /** + *

    This option is deprecated, see {@link #loadBroadcastDatasourcesMode} option.

    + * * If set to "true", the peon will bind classes necessary for loading broadcast segments. This is used for * queryable tasks, such as streaming ingestion tasks. + * */ - @Option(name = "--loadBroadcastSegments", title = "loadBroadcastSegments", description = "Enable loading of broadcast segments") + @Deprecated + @Option(name = "--loadBroadcastSegments", title = "loadBroadcastSegments", + description = "Enable loading of broadcast segments. This option is deprecated and will be removed in a" + + " future release. Use --loadBroadcastDatasourceMode instead.") public String loadBroadcastSegments = "false"; + /** + * Broadcast datasource loading mode. The peon will bind classes necessary required for loading broadcast segments if + * the mode is {@link BroadcastDatasourceLoadingSpec.Mode#ALL} or {@link BroadcastDatasourceLoadingSpec.Mode#ONLY_REQUIRED}. + */ + @Option(name = "--loadBroadcastDatasourceMode", title = "loadBroadcastDatasourceMode", + description = "Specify the broadcast datasource loading mode for the peon. Supported values are ALL, NONE, ONLY_REQUIRED.") + public String loadBroadcastDatasourcesMode = BroadcastDatasourceLoadingSpec.Mode.ALL.toString(); + @Option(name = "--taskId", title = "taskId", description = "TaskId for fetching task.json remotely") public String taskId = ""; @@ -205,7 +220,7 @@ public class CliPeon extends GuiceRunnable protected List getModules() { return ImmutableList.of( - new DruidProcessingModule(), + new PeonProcessingModule(), new QueryableModule(), new QueryRunnerFactoryModule(), new SegmentWranglerModule(), @@ -274,7 +289,11 @@ public class CliPeon extends GuiceRunnable binder.bind(ServerTypeConfig.class).toInstance(new ServerTypeConfig(ServerType.fromString(serverType))); LifecycleModule.register(binder, Server.class); - if ("true".equals(loadBroadcastSegments)) { + final BroadcastDatasourceLoadingSpec.Mode mode = + BroadcastDatasourceLoadingSpec.Mode.valueOf(loadBroadcastDatasourcesMode); + if ("true".equals(loadBroadcastSegments) + || mode == BroadcastDatasourceLoadingSpec.Mode.ALL + || mode == BroadcastDatasourceLoadingSpec.Mode.ONLY_REQUIRED) { binder.install(new BroadcastSegmentLoadingModule()); } } @@ -340,6 +359,14 @@ public class CliPeon extends GuiceRunnable { return task.getLookupLoadingSpec(); } + + @Provides + @LazySingleton + @Named(DataSourceTaskIdHolder.BROADCAST_DATASOURCES_TO_LOAD_FOR_TASK) + public BroadcastDatasourceLoadingSpec getBroadcastDatasourcesToLoad(final Task task) + { + return task.getBroadcastDatasourceLoadingSpec(); + } }, new QueryablePeonModule(), new IndexingServiceInputSourceModule(), diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index 3f8cfcb94a4..5c31f02b603 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -75,8 +75,8 @@ import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnHolder; @@ -295,14 +295,14 @@ public class DumpSegment extends GuiceRunnable throws IOException { final ObjectMapper objectMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index); + final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index); final DimFilter filter = filterJson != null ? objectMapper.readValue(filterJson, DimFilter.class) : null; final CursorBuildSpec buildSpec = CursorBuildSpec.builder() .setFilter(Filters.toFilter(filter)) .build(); - try (final CursorHolder cursorHolder = adapter.makeCursorHolder(buildSpec)) { + try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) { final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { return; diff --git a/services/src/main/java/org/apache/druid/cli/PullDependencies.java b/services/src/main/java/org/apache/druid/cli/PullDependencies.java index 6ea8626ba7e..f0588d2b372 100644 --- a/services/src/main/java/org/apache/druid/cli/PullDependencies.java +++ b/services/src/main/java/org/apache/druid/cli/PullDependencies.java @@ -22,43 +22,45 @@ package org.apache.druid.cli; import com.github.rvesse.airline.annotations.Command; import com.github.rvesse.airline.annotations.Option; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSetMultimap; import com.google.common.collect.SetMultimap; import com.google.inject.Inject; -import io.netty.util.SuppressForbidden; -import io.tesla.aether.Repository; -import io.tesla.aether.TeslaAether; -import io.tesla.aether.guice.RepositorySystemSessionProvider; -import io.tesla.aether.internal.DefaultTeslaAether; import org.apache.druid.guice.ExtensionsConfig; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.maven.artifact.resolver.ArtifactNotFoundException; +import org.apache.maven.repository.internal.MavenRepositorySystemUtils; +import org.eclipse.aether.DefaultRepositorySystemSession; +import org.eclipse.aether.RepositorySystem; import org.eclipse.aether.RepositorySystemSession; import org.eclipse.aether.artifact.Artifact; import org.eclipse.aether.artifact.DefaultArtifact; import org.eclipse.aether.collection.CollectRequest; +import org.eclipse.aether.connector.basic.BasicRepositoryConnectorFactory; import org.eclipse.aether.graph.Dependency; import org.eclipse.aether.graph.DependencyNode; -import org.eclipse.aether.repository.Authentication; +import org.eclipse.aether.impl.DefaultServiceLocator; +import org.eclipse.aether.repository.LocalRepository; import org.eclipse.aether.repository.Proxy; import org.eclipse.aether.repository.RemoteRepository; +import org.eclipse.aether.resolution.ArtifactResult; import org.eclipse.aether.resolution.DependencyRequest; +import org.eclipse.aether.resolution.DependencyResolutionException; +import org.eclipse.aether.resolution.DependencyResult; +import org.eclipse.aether.spi.connector.RepositoryConnectorFactory; +import org.eclipse.aether.spi.connector.transport.TransporterFactory; +import org.eclipse.aether.transport.http.HttpTransporterFactory; import org.eclipse.aether.util.artifact.JavaScopes; import org.eclipse.aether.util.filter.DependencyFilterUtils; import org.eclipse.aether.util.repository.AuthenticationBuilder; +import org.eclipse.aether.util.repository.DefaultProxySelector; import java.io.File; import java.io.IOException; -import java.io.OutputStream; -import java.io.PrintStream; -import java.io.UnsupportedEncodingException; -import java.net.URI; -import java.net.URISyntaxException; import java.util.ArrayList; import java.util.List; import java.util.Set; @@ -86,72 +88,6 @@ public class PullDependencies implements Runnable .put("com.fasterxml.jackson.core", "jackson-core") .put("com.fasterxml.jackson.core", "jackson-annotations") .build(); - /* - It is possible that extensions will pull down a lot of jars that are either - duplicates OR conflict with druid jars. In that case, there are two problems that arise - - 1. Large quantity of jars are passed around to things like hadoop when they are not needed (and should not be included) - 2. Classpath priority becomes "mostly correct" and attempted to enforced correctly, but not fully tested - - These jar groups should be included by druid and *not* pulled down in extensions - Note to future developers: This list is hand-crafted and will probably be out of date in the future - A good way to know where to look for errant dependencies is to compare the lib/ directory in the distribution - tarball with the jars included in the extension directories. - - This list is best-effort, and might still pull down more than desired. - - A simple example is that if an extension's dependency uses some-library-123.jar, - druid uses some-library-456.jar, and hadoop uses some-library-666.jar, then we probably want to use some-library-456.jar, - so don't pull down some-library-123.jar, and ask hadoop to load some-library-456.jar. - - In the case where some-library is NOT on this list, both some-library-456.jar and some-library-123.jar will be - on the class path and propagated around the system. Most places TRY to make sure some-library-456.jar has - precedence, but it is easy for this assumption to be violated and for the precedence of some-library-456.jar, - some-library-123.jar and some-library-456.jar to not be properly defined. - - As of this writing there are no special unit tests for classloader issues and library version conflicts. - - Different tasks which are classloader sensitive attempt to maintain a sane order for loading libraries in the - classloader, but it is always possible that something didn't load in the right order. Also we don't want to be - throwing around a ton of jars we don't need to. - - Here is a list of dependencies extensions should probably exclude. - - Conflicts can be discovered using the following command on the distribution tarball: - `find lib -iname "*.jar" | cut -d / -f 2 | sed -e 's/-[0-9]\.[0-9]/@/' | cut -f 1 -d @ | sort | uniq | xargs -I {} find extensions -name "*{}*.jar" | sort` - - "org.apache.druid", - "com.metamx.druid", - "asm", - "org.ow2.asm", - "org.jboss.netty", - "com.google.guava", - "com.google.code.findbugs", - "com.google.protobuf", - "com.esotericsoftware.minlog", - "log4j", - "org.slf4j", - "commons-logging", - "org.eclipse.jetty", - "org.mortbay.jetty", - "com.sun.jersey", - "com.sun.jersey.contribs", - "common-beanutils", - "commons-codec", - "commons-lang", - "commons-cli", - "commons-io", - "javax.activation", - "org.apache.httpcomponents", - "org.apache.zookeeper", - "org.codehaus.jackson", - "com.fasterxml.jackson", - "com.fasterxml.jackson.core", - "com.fasterxml.jackson.dataformat", - "com.fasterxml.jackson.datatype", - "org.roaringbitmap", - "net.java.dev.jets3t" - */ private static final Dependencies SECURITY_VULNERABILITY_EXCLUSIONS = Dependencies.builder() @@ -160,8 +96,6 @@ public class PullDependencies implements Runnable private final Dependencies hadoopExclusions; - private TeslaAether aether; - @Inject public ExtensionsConfig extensionsConfig; @@ -196,60 +130,53 @@ public class PullDependencies implements Runnable title = "A local repository that Maven will use to put downloaded files. Then pull-deps will lay these files out into the extensions directory as needed." ) public String localRepository = StringUtils.format("%s/%s", System.getProperty("user.home"), ".m2/repository"); - - @Option( - name = {"-r", "--remoteRepository"}, - title = "Add a remote repository. Unless --no-default-remote-repositories is provided, these will be used after https://repo1.maven.org/maven2/" - ) - List remoteRepositories = new ArrayList<>(); - @Option( name = "--no-default-remote-repositories", description = "Don't use the default remote repositories, only use the repositories provided directly via --remoteRepository" ) public boolean noDefaultRemoteRepositories = false; - @Option( name = {"-d", "--defaultVersion"}, title = "Version to use for extension artifacts without version information." ) public String defaultVersion = PullDependencies.class.getPackage().getImplementationVersion(); - @Option( name = {"--use-proxy"}, title = "Use http/https proxy to pull dependencies." ) public boolean useProxy = false; - @Option( name = {"--proxy-type"}, title = "The proxy type, should be either http or https" ) public String proxyType = "https"; - @Option( name = {"--proxy-host"}, title = "The proxy host" ) public String proxyHost = ""; - @Option( name = {"--proxy-port"}, title = "The proxy port" ) public int proxyPort = -1; - @Option( name = {"--proxy-username"}, title = "The proxy username" ) public String proxyUsername = ""; - @Option( name = {"--proxy-password"}, title = "The proxy password" ) public String proxyPassword = ""; + @Option( + name = {"-r", "--remoteRepository"}, + title = "Add a remote repository. Unless --no-default-remote-repositories is provided, these will be used after https://repo1.maven.org/maven2/" + ) + List remoteRepositories = new ArrayList<>(); + private RepositorySystem repositorySystem; + private RepositorySystemSession repositorySystemSession; @SuppressWarnings("unused") // used by com.github.rvesse.airline public PullDependencies() @@ -261,18 +188,74 @@ public class PullDependencies implements Runnable } // Used for testing only - PullDependencies(TeslaAether aether, ExtensionsConfig extensionsConfig, Dependencies hadoopExclusions) + PullDependencies( + RepositorySystem repositorySystem, + RepositorySystemSession repositorySystemSession, + ExtensionsConfig extensionsConfig, + Dependencies hadoopExclusions + ) { - this.aether = aether; + this.repositorySystem = repositorySystem; + this.repositorySystemSession = repositorySystemSession; this.extensionsConfig = extensionsConfig; this.hadoopExclusions = hadoopExclusions; } + private RepositorySystem getRepositorySystem() + { + DefaultServiceLocator locator = MavenRepositorySystemUtils.newServiceLocator(); + locator.addService(RepositoryConnectorFactory.class, BasicRepositoryConnectorFactory.class); + locator.addService(TransporterFactory.class, HttpTransporterFactory.class); + return locator.getService(RepositorySystem.class); + } + + protected RepositorySystemSession getRepositorySystemSession() + { + DefaultRepositorySystemSession session = MavenRepositorySystemUtils.newSession(); + LocalRepository localRepo = new LocalRepository(localRepository); + session.setLocalRepositoryManager(repositorySystem.newLocalRepositoryManager(session, localRepo)); + + // Set up the proxy configuration if required + if (useProxy) { + Proxy proxy = new Proxy( + proxyType, + proxyHost, + proxyPort, + isBlank(proxyUsername) ? null : new AuthenticationBuilder() + .addUsername(proxyUsername) + .addPassword(proxyPassword) + .build() + ); + + final DefaultProxySelector proxySelector = new DefaultProxySelector(); + proxySelector.add(proxy, null); + + session.setProxySelector(proxySelector); + } + + return session; + } + + protected List getRemoteRepositories() + { + List repositories = new ArrayList<>(); + + if (!noDefaultRemoteRepositories) { + repositories.add(new RemoteRepository.Builder("central", "default", DEFAULT_REMOTE_REPOSITORIES.get(0)).build()); + } + + for (String repoUrl : remoteRepositories) { + repositories.add(new RemoteRepository.Builder(null, "default", repoUrl).build()); + } + + return repositories; + } + @Override public void run() { - if (aether == null) { - aether = getAetherClient(); + if (repositorySystem == null) { + repositorySystem = getRepositorySystem(); } final File extensionsDir = new File(extensionsConfig.getDirectory()); @@ -334,7 +317,7 @@ public class PullDependencies implements Runnable } } - private Artifact getArtifact(String coordinate) + protected Artifact getArtifact(String coordinate) { DefaultArtifact versionedArtifact; try { @@ -367,6 +350,12 @@ public class PullDependencies implements Runnable { final CollectRequest collectRequest = new CollectRequest(); collectRequest.setRoot(new Dependency(versionedArtifact, JavaScopes.RUNTIME)); + + List repositories = getRemoteRepositories(); + for (RemoteRepository repo : repositories) { + collectRequest.addRepository(repo); + } + final DependencyRequest dependencyRequest = new DependencyRequest( collectRequest, DependencyFilterUtils.andFilter( @@ -375,13 +364,7 @@ public class PullDependencies implements Runnable String scope = node.getDependency().getScope(); if (scope != null) { scope = StringUtils.toLowerCase(scope); - if ("provided".equals(scope)) { - return false; - } - if ("test".equals(scope)) { - return false; - } - if ("system".equals(scope)) { + if ("provided".equals(scope) || "test".equals(scope) || "system".equals(scope)) { return false; } } @@ -402,7 +385,17 @@ public class PullDependencies implements Runnable try { log.info("Start downloading extension [%s]", versionedArtifact); - final List artifacts = aether.resolveArtifacts(dependencyRequest); + if (repositorySystemSession == null) { + repositorySystemSession = getRepositorySystemSession(); + } + + final DependencyResult result = repositorySystem.resolveDependencies( + repositorySystemSession, + dependencyRequest + ); + final List artifacts = result.getArtifactResults().stream() + .map(ArtifactResult::getArtifact) + .collect(Collectors.toList()); for (Artifact artifact : artifacts) { if (exclusions.contain(artifact)) { @@ -413,140 +406,20 @@ public class PullDependencies implements Runnable } } } - catch (Exception e) { - log.error(e, "Unable to resolve artifacts for [%s].", dependencyRequest); + catch (DependencyResolutionException e) { + if (e.getCause() instanceof ArtifactNotFoundException) { + log.error("Artifact not found in any configured repositories: [%s]", versionedArtifact); + } else { + log.error(e, "Unable to resolve artifacts for [%s].", dependencyRequest); + } + } + catch (IOException e) { + log.error(e, "I/O error while processing artifact [%s].", versionedArtifact); throw new RuntimeException(e); } log.info("Finish downloading extension [%s]", versionedArtifact); } - @SuppressForbidden(reason = "System#out") - private DefaultTeslaAether getAetherClient() - { - /* - DefaultTeslaAether logs a bunch of stuff to System.out, which is annoying. We choose to disable that - unless debug logging is turned on. "Disabling" it, however, is kinda bass-ackwards. We copy out a reference - to the current System.out, and set System.out to a noop output stream. Then after DefaultTeslaAether has pulled - The reference we swap things back. - - This has implications for other things that are running in parallel to this. Namely, if anything else also grabs - a reference to System.out or tries to log to it while we have things adjusted like this, then they will also log - to nothingness. Fortunately, the code that calls this is single-threaded and shouldn't hopefully be running - alongside anything else that's grabbing System.out. But who knows. - */ - - final List remoteUriList = new ArrayList<>(); - if (!noDefaultRemoteRepositories) { - remoteUriList.addAll(DEFAULT_REMOTE_REPOSITORIES); - } - remoteUriList.addAll(remoteRepositories); - - List remoteRepositories = new ArrayList<>(); - for (String uri : remoteUriList) { - try { - URI u = new URI(uri); - Repository r = new Repository(uri); - - if (u.getUserInfo() != null) { - String[] auth = u.getUserInfo().split(":", 2); - if (auth.length == 2) { - r.setUsername(auth[0]); - r.setPassword(auth[1]); - } else { - log.warn( - "Invalid credentials in repository URI, expecting [:], got [%s] for [%s]", - u.getUserInfo(), - uri - ); - } - } - remoteRepositories.add(r); - } - catch (URISyntaxException e) { - throw new RuntimeException(e); - } - } - - if (log.isTraceEnabled() || log.isDebugEnabled()) { - return createTeslaAether(remoteRepositories); - } - - PrintStream oldOut = System.out; - try { - System.setOut( - new PrintStream( - new OutputStream() - { - @Override - public void write(int b) - { - - } - - @Override - public void write(byte[] b) - { - - } - - @Override - public void write(byte[] b, int off, int len) - { - - } - }, - false, - StringUtils.UTF8_STRING - ) - ); - return createTeslaAether(remoteRepositories); - } - catch (UnsupportedEncodingException e) { - // should never happen - throw new IllegalStateException(e); - } - finally { - System.setOut(oldOut); - } - } - - private DefaultTeslaAether createTeslaAether(List remoteRepositories) - { - if (!useProxy) { - return new DefaultTeslaAether( - localRepository, - remoteRepositories.toArray(new Repository[0]) - ); - } - - if (!StringUtils.toLowerCase(proxyType).equals(Proxy.TYPE_HTTP) && - !StringUtils.toLowerCase(proxyType).equals(Proxy.TYPE_HTTPS)) { - throw new IllegalArgumentException("invalid proxy type: " + proxyType); - } - - RepositorySystemSession repositorySystemSession = - new RepositorySystemSessionProvider(new File(localRepository)).get(); - List rl = remoteRepositories.stream().map(r -> { - RemoteRepository.Builder builder = new RemoteRepository.Builder(r.getId(), "default", r.getUrl()); - if (r.getUsername() != null && r.getPassword() != null) { - Authentication auth = new AuthenticationBuilder().addUsername(r.getUsername()) - .addPassword(r.getPassword()) - .build(); - builder.setAuthentication(auth); - } - - final Authentication proxyAuth; - if (Strings.isNullOrEmpty(proxyUsername)) { - proxyAuth = null; - } else { - proxyAuth = new AuthenticationBuilder().addUsername(proxyUsername).addPassword(proxyPassword).build(); - } - builder.setProxy(new Proxy(proxyType, proxyHost, proxyPort, proxyAuth)); - return builder.build(); - }).collect(Collectors.toList()); - return new DefaultTeslaAether(rl, repositorySystemSession); - } - /** * Create the extension directory for a specific maven coordinate. * The name of this directory should be the artifactId in the coordinate @@ -567,6 +440,11 @@ public class PullDependencies implements Runnable } } + private boolean isBlank(final String toCheck) + { + return toCheck == null || toCheck.isEmpty(); + } + @VisibleForTesting static class Dependencies { @@ -579,17 +457,17 @@ public class PullDependencies implements Runnable groupIdToArtifactIds = builder.groupIdToArtifactIdsBuilder.build(); } + static Builder builder() + { + return new Builder(); + } + boolean contain(Artifact artifact) { Set artifactIds = groupIdToArtifactIds.get(artifact.getGroupId()); return artifactIds.contains(ANY_ARTIFACT_ID) || artifactIds.contains(artifact.getArtifactId()); } - static Builder builder() - { - return new Builder(); - } - static final class Builder { private final ImmutableSetMultimap.Builder groupIdToArtifactIdsBuilder = diff --git a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java index f17d5129143..fc7e9d33323 100644 --- a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java +++ b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java @@ -133,7 +133,7 @@ public class DumpSegmentTest extends InitializedNullHandlingTest Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null)); List segments = createSegments(tempFolder, closer); - QueryableIndex queryableIndex = segments.get(0).asQueryableIndex(); + QueryableIndex queryableIndex = segments.get(0).as(QueryableIndex.class); File outputFile = tempFolder.newFile(); @@ -147,11 +147,11 @@ public class DumpSegmentTest extends InitializedNullHandlingTest ); final byte[] fileBytes = Files.readAllBytes(outputFile.toPath()); final String output = StringUtils.fromUtf8(fileBytes); - final String expected = "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"x\":200,\"y\":2.2}}\n" - + "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"x\":400,\"y\":1.1,\"z\":\"a\"}}\n" - + "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"x\":200,\"z\":\"b\"}}\n" - + "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"x\":100,\"y\":1.1,\"z\":\"a\"}}\n" - + "{\"__time\":1609459200000,\"count\":1,\"nest\":{\"y\":3.3,\"z\":\"b\"}}\n"; + final String expected = "{\"__time\":1609459200000,\"nest\":{\"x\":200,\"y\":2.2},\"count\":1}\n" + + "{\"__time\":1609459200000,\"nest\":{\"x\":400,\"y\":1.1,\"z\":\"a\"},\"count\":1}\n" + + "{\"__time\":1609459200000,\"nest\":{\"x\":200,\"z\":\"b\"},\"count\":1}\n" + + "{\"__time\":1609459200000,\"nest\":{\"x\":100,\"y\":1.1,\"z\":\"a\"},\"count\":1}\n" + + "{\"__time\":1609459200000,\"nest\":{\"y\":3.3,\"z\":\"b\"},\"count\":1}\n"; Assert.assertEquals(expected, output); } @@ -207,7 +207,7 @@ public class DumpSegmentTest extends InitializedNullHandlingTest Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null)); List segments = createSegments(tempFolder, closer); - QueryableIndex queryableIndex = segments.get(0).asQueryableIndex(); + QueryableIndex queryableIndex = segments.get(0).as(QueryableIndex.class); File outputFile = tempFolder.newFile(); DumpSegment.runDumpNestedColumn( @@ -247,7 +247,7 @@ public class DumpSegmentTest extends InitializedNullHandlingTest Mockito.when(injector.getInstance(DefaultColumnFormatConfig.class)).thenReturn(new DefaultColumnFormatConfig(null, null)); List segments = createSegments(tempFolder, closer); - QueryableIndex queryableIndex = segments.get(0).asQueryableIndex(); + QueryableIndex queryableIndex = segments.get(0).as(QueryableIndex.class); File outputFile = tempFolder.newFile(); DumpSegment.runDumpNestedColumnPath( diff --git a/services/src/test/java/org/apache/druid/cli/PullDependenciesTest.java b/services/src/test/java/org/apache/druid/cli/PullDependenciesTest.java index 2d4db23b0e3..851d0f17720 100644 --- a/services/src/test/java/org/apache/druid/cli/PullDependenciesTest.java +++ b/services/src/test/java/org/apache/druid/cli/PullDependenciesTest.java @@ -21,16 +21,32 @@ package org.apache.druid.cli; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import io.tesla.aether.internal.DefaultTeslaAether; import org.apache.druid.guice.ExtensionsConfig; import org.apache.druid.java.util.common.StringUtils; +import org.apache.maven.repository.internal.MavenRepositorySystemUtils; +import org.eclipse.aether.DefaultRepositorySystemSession; +import org.eclipse.aether.RepositorySystem; +import org.eclipse.aether.RepositorySystemSession; import org.eclipse.aether.artifact.Artifact; import org.eclipse.aether.artifact.DefaultArtifact; +import org.eclipse.aether.connector.basic.BasicRepositoryConnectorFactory; import org.eclipse.aether.graph.DefaultDependencyNode; import org.eclipse.aether.graph.Dependency; -import org.eclipse.aether.graph.DependencyFilter; import org.eclipse.aether.graph.DependencyNode; +import org.eclipse.aether.impl.DefaultServiceLocator; +import org.eclipse.aether.repository.Authentication; +import org.eclipse.aether.repository.LocalRepository; +import org.eclipse.aether.repository.Proxy; +import org.eclipse.aether.repository.RemoteRepository; +import org.eclipse.aether.resolution.ArtifactRequest; +import org.eclipse.aether.resolution.ArtifactResult; import org.eclipse.aether.resolution.DependencyRequest; +import org.eclipse.aether.resolution.DependencyResult; +import org.eclipse.aether.spi.connector.RepositoryConnectorFactory; +import org.eclipse.aether.spi.connector.transport.TransporterFactory; +import org.eclipse.aether.transport.http.HttpTransporterFactory; +import org.eclipse.aether.util.artifact.JavaScopes; +import org.eclipse.aether.util.repository.AuthenticationBuilder; import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Before; @@ -50,6 +66,11 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; + public class PullDependenciesTest { private static final String EXTENSION_A_COORDINATE = "groupX:extension_A:123"; @@ -71,23 +92,18 @@ public class PullDependenciesTest .put(DEPENDENCY_GROUPID, HADOOP_CLIENT_VULNERABLE_ARTIFACTID1) .put(DEPENDENCY_GROUPID, HADOOP_CLIENT_VULNERABLE_ARTIFACTID2) .build(); - + private static File localRepo; // a mock local repository that stores jars + private static Map> extensionToDependency; @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - - private File localRepo; // a mock local repository that stores jars - private final Artifact extension_A = new DefaultArtifact(EXTENSION_A_COORDINATE); private final Artifact extension_B = new DefaultArtifact(EXTENSION_B_COORDINATE); private final Artifact hadoop_client_2_3_0 = new DefaultArtifact(HADOOP_CLIENT_2_3_0_COORDINATE); private final Artifact hadoop_client_2_4_0 = new DefaultArtifact(HADOOP_CLIENT_2_4_0_COORDINATE); - private PullDependencies pullDependencies; private File rootExtensionsDir; private File rootHadoopDependenciesDir; - private Map> extensionToDependency; - @Before public void setUp() throws Exception { @@ -105,18 +121,22 @@ public class PullDependenciesTest rootExtensionsDir = temporaryFolder.newFolder("extensions"); rootHadoopDependenciesDir = temporaryFolder.newFolder("druid_hadoop_dependencies"); + RepositorySystem realRepositorySystem = RealRepositorySystemUtil.newRepositorySystem(); + RepositorySystem spyMockRepositorySystem = spy(realRepositorySystem); + RepositorySystemSession repositorySystemSession = RealRepositorySystemUtil.newRepositorySystemSession( + spyMockRepositorySystem, + localRepo.getPath() + ); + + doAnswer(invocation -> { + DependencyRequest request = invocation.getArgument(1); + return mockDependencyResult(request.getCollectRequest().getRoot().getArtifact()); + }).when(spyMockRepositorySystem).resolveDependencies(eq(repositorySystemSession), any(DependencyRequest.class)); + + pullDependencies = new PullDependencies( - new DefaultTeslaAether() - { - @Override - public List resolveArtifacts(DependencyRequest request) - { - return getArtifactsForExtension( - request.getCollectRequest().getRoot().getArtifact(), - request.getFilter() - ); - } - }, + spyMockRepositorySystem, + repositorySystemSession, new ExtensionsConfig() { @Override @@ -140,14 +160,15 @@ public class PullDependenciesTest HADOOP_CLIENT_2_4_0_COORDINATE ); - // Because --clean is specified, pull-deps will first remove existing root extensions and hadoop dependencies pullDependencies.clean = true; } - private List getArtifactsForExtension(Artifact artifact, DependencyFilter filter) + private DependencyResult mockDependencyResult(Artifact artifact) { - final List names = extensionToDependency.get(artifact); - final List artifacts = new ArrayList<>(); + final List names = extensionToDependency.getOrDefault(artifact, Collections.emptyList()); + final List artifacts = new ArrayList<>(); + List children = new ArrayList<>(); + for (String name : names) { final File jarFile = new File(localRepo, name + ".jar"); try { @@ -156,18 +177,23 @@ public class PullDependenciesTest catch (IOException e) { throw new RuntimeException(e); } - - DependencyNode node = new DefaultDependencyNode( - new Dependency( - new DefaultArtifact(DEPENDENCY_GROUPID, name, null, "jar", "1.0", null, jarFile), - "compile" - ) + Artifact depArtifact = new DefaultArtifact("groupid", name, null, "jar", "1.0", + null, jarFile ); - if (filter.accept(node, Collections.emptyList())) { - artifacts.add(node.getArtifact()); - } + DependencyNode depNode = new DefaultDependencyNode(new Dependency(depArtifact, JavaScopes.COMPILE)); + children.add(depNode); + ArtifactResult artifactResult = new ArtifactResult(new ArtifactRequest(depNode)); + artifactResult.setArtifact(depArtifact); + artifacts.add(artifactResult); } - return artifacts; + + DependencyNode rootNode = new DefaultDependencyNode(new Dependency(artifact, JavaScopes.COMPILE)); + rootNode.setChildren(children); + + DependencyResult result = new DependencyResult(new DependencyRequest()); + result.setRoot(rootNode); + result.setArtifactResults(artifacts); + return result; } private List getExpectedJarFiles(Artifact artifact) @@ -299,4 +325,157 @@ public class PullDependenciesTest Assert.assertThat(dependencies, CoreMatchers.not(CoreMatchers.hasItem(HADOOP_CLIENT_VULNERABLE_JAR1))); Assert.assertThat(dependencies, CoreMatchers.not(CoreMatchers.hasItem(HADOOP_CLIENT_VULNERABLE_JAR2))); } + + @Test + public void testPullDependenciesCleanFlag() throws IOException + { + File dummyFile1 = new File(rootExtensionsDir, "dummy.txt"); + File dummyFile2 = new File(rootHadoopDependenciesDir, "dummy.txt"); + Assert.assertTrue(dummyFile1.createNewFile()); + Assert.assertTrue(dummyFile2.createNewFile()); + + pullDependencies.clean = true; + pullDependencies.run(); + + Assert.assertFalse(dummyFile1.exists()); + Assert.assertFalse(dummyFile2.exists()); + } + + @Test + public void testPullDependenciesNoDefaultRemoteRepositories() + { + pullDependencies.noDefaultRemoteRepositories = true; + pullDependencies.remoteRepositories = ImmutableList.of("https://custom.repo"); + + pullDependencies.run(); + + List repositories = pullDependencies.getRemoteRepositories(); + Assert.assertEquals(1, repositories.size()); + Assert.assertEquals("https://custom.repo", repositories.get(0).getUrl()); + } + + @Test + public void testPullDependenciesDirectoryCreationFailure() throws IOException + { + if (rootExtensionsDir.exists()) { + rootExtensionsDir.delete(); + } + Assert.assertTrue(rootExtensionsDir.createNewFile()); + + Assert.assertThrows(IllegalArgumentException.class, () -> pullDependencies.run()); + } + + @Test + public void testGetArtifactWithValidCoordinate() + { + String coordinate = "groupX:artifactX:1.0.0"; + DefaultArtifact artifact = (DefaultArtifact) pullDependencies.getArtifact(coordinate); + Assert.assertEquals("groupX", artifact.getGroupId()); + Assert.assertEquals("artifactX", artifact.getArtifactId()); + Assert.assertEquals("1.0.0", artifact.getVersion()); + } + + @Test + public void testGetArtifactwithCoordinateWithoutDefaultVersion() + { + String coordinate = "groupY:artifactY"; + Assert.assertThrows( + "Bad artifact coordinates groupY:artifactY, expected format is :[:[:]]:", + IllegalArgumentException.class, + () -> pullDependencies.getArtifact(coordinate) + ); + + } + + @Test + public void testGetArtifactWithCoordinateWithoutVersion() + { + pullDependencies.defaultVersion = "2.0.0"; + String coordinate = "groupY:artifactY"; + DefaultArtifact artifact = (DefaultArtifact) pullDependencies.getArtifact(coordinate); + Assert.assertEquals("groupY", artifact.getGroupId()); + Assert.assertEquals("artifactY", artifact.getArtifactId()); + Assert.assertEquals("2.0.0", artifact.getVersion()); + } + + @Test + public void testGetRemoteRepositoriesWithDefaultRepositories() + { + pullDependencies.noDefaultRemoteRepositories = false; // Use default remote repositories + pullDependencies.remoteRepositories = ImmutableList.of("https://custom.repo"); + + List repositories = pullDependencies.getRemoteRepositories(); + Assert.assertEquals(2, repositories.size()); + Assert.assertEquals("https://repo1.maven.org/maven2/", repositories.get(0).getUrl()); + Assert.assertEquals("https://custom.repo", repositories.get(1).getUrl()); + } + + @Test + public void testGetRepositorySystemSessionWithProxyConfiguration() + { + pullDependencies.useProxy = true; + pullDependencies.proxyType = "http"; + pullDependencies.proxyHost = "localhost"; + pullDependencies.proxyPort = 8080; + pullDependencies.proxyUsername = "user"; + pullDependencies.proxyPassword = "password"; + + DefaultRepositorySystemSession session = (DefaultRepositorySystemSession) pullDependencies.getRepositorySystemSession(); + + LocalRepository localRepo = session.getLocalRepositoryManager().getRepository(); + Assert.assertEquals(pullDependencies.localRepository, localRepo.getBasedir().getAbsolutePath()); + + Proxy proxy = session.getProxySelector().getProxy( + new RemoteRepository.Builder("test", "default", "http://example.com").build() + ); + RemoteRepository testRepository = new RemoteRepository.Builder("test", "default", "http://example.com") + .setProxy(proxy) + .build(); + + Assert.assertNotNull(proxy); + Assert.assertEquals("localhost", proxy.getHost()); + Assert.assertEquals(8080, proxy.getPort()); + Assert.assertEquals("http", proxy.getType()); + + Authentication auth = new AuthenticationBuilder().addUsername("user").addPassword("password").build(); + Assert.assertEquals(auth, proxy.getAuthentication()); + } + + @Test + public void testGetRepositorySystemSessionWithoutProxyConfiguration() + { + pullDependencies.useProxy = false; + DefaultRepositorySystemSession session = (DefaultRepositorySystemSession) pullDependencies.getRepositorySystemSession(); + LocalRepository localRepo = session.getLocalRepositoryManager().getRepository(); + Assert.assertEquals(pullDependencies.localRepository, localRepo.getBasedir().getAbsolutePath()); + Proxy proxy = session.getProxySelector().getProxy( + new RemoteRepository.Builder("test", "default", "http://example.com").build() + ); + Assert.assertNull(proxy); + } + + private static class RealRepositorySystemUtil + { + public static RepositorySystem newRepositorySystem() + { + DefaultServiceLocator locator = MavenRepositorySystemUtils.newServiceLocator(); + locator.addService(RepositoryConnectorFactory.class, BasicRepositoryConnectorFactory.class); + locator.addService(TransporterFactory.class, HttpTransporterFactory.class); + return locator.getService(RepositorySystem.class); + } + + public static DefaultRepositorySystemSession newRepositorySystemSession( + RepositorySystem system, + String localRepoPath + ) + { + DefaultRepositorySystemSession session = MavenRepositorySystemUtils.newSession(); + + LocalRepository localRepo = new LocalRepository(localRepoPath); + session.setLocalRepositoryManager(system.newLocalRepositoryManager(session, localRepo)); + + return session; + } + } + } diff --git a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java index b617b7e6b87..c0634ed403c 100644 --- a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java @@ -30,7 +30,6 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -133,14 +132,11 @@ public class DruidJsonValidatorTest null, new TaskResource("rofl", 2), new IndexTask.IndexIngestionSpec( - new DataSchema( - "foo", - null, - new AggregatorFactory[0], - new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null), - null, - jsonMapper - ), + DataSchema.builder() + .withDataSource("foo") + .withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null)) + .withObjectMapper(jsonMapper) + .build(), new IndexTask.IndexIOConfig( new LocalInputSource(new File("lol"), "rofl"), new JsonInputFormat(null, null, null, null, null), diff --git a/sql/pom.xml b/sql/pom.xml index b3ccc6ad258..6c564e5cdaa 100644 --- a/sql/pom.xml +++ b/sql/pom.xml @@ -29,7 +29,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CastOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CastOperatorConversion.java index ac34abe48d1..d82d3ea9546 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CastOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CastOperatorConversion.java @@ -93,11 +93,17 @@ public class CastOperatorConversion implements SqlOperatorConversion : ExpressionType.fromColumnType(toDruidType); if (toExpressionType == null) { - // We have no runtime type for these SQL types. + // We have no runtime type for to SQL type. return null; } if (fromExpressionType == null) { - return DruidExpression.ofLiteral(toDruidType, DruidExpression.nullLiteral()); + // Calcites.getColumnTypeForRelDataType returns null in cases of NULL, but also any type which cannot be + // mapped to a native druid type. in the case of the former, make a null literal of the toType + if (fromType.equals(SqlTypeName.NULL)) { + return DruidExpression.ofLiteral(toDruidType, DruidExpression.nullLiteral()); + } + // otherwise, we have no runtime type for from SQL type. + return null; } final DruidExpression typeCastExpression; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java index a6006046553..9c6cfb0448b 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/NestedDataOperatorConversions.java @@ -780,6 +780,52 @@ public class NestedDataOperatorConversions } } + public static class JsonMergeOperatorConversion implements SqlOperatorConversion + { + private static final String FUNCTION_NAME = "json_merge"; + private static final SqlFunction SQL_FUNCTION = OperatorConversions + .operatorBuilder(FUNCTION_NAME) + .operandTypeChecker(OperandTypes.variadic(SqlOperandCountRanges.from(1))) + .operandTypeInference((callBinding, returnType, operandTypes) -> { + RelDataTypeFactory typeFactory = callBinding.getTypeFactory(); + for (int i = 0; i < operandTypes.length; i++) { + operandTypes[i] = typeFactory.createTypeWithNullability( + typeFactory.createSqlType(SqlTypeName.ANY), + true + ); + } + }) + .returnTypeInference(NESTED_RETURN_TYPE_INFERENCE) + .functionCategory(SqlFunctionCategory.SYSTEM) + .build(); + + @Override + public SqlOperator calciteOperator() + { + return SQL_FUNCTION; + } + + @Nullable + @Override + public DruidExpression toDruidExpression( + PlannerContext plannerContext, + RowSignature rowSignature, + RexNode rexNode + ) + { + return OperatorConversions.convertCall( + plannerContext, + rowSignature, + rexNode, + druidExpressions -> DruidExpression.ofExpression( + ColumnType.NESTED_DATA, + DruidExpression.functionCall("json_merge"), + druidExpressions + ) + ); + } + } + public static class ToJsonStringOperatorConversion implements SqlOperatorConversion { private static final String FUNCTION_NAME = "to_json_string"; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java index 64929c6445e..6310b23543d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java @@ -218,6 +218,9 @@ public class Calcites if (elementType != null) { return ColumnType.ofArray(elementType); } + if (type.getComponentType().getSqlTypeName() == SqlTypeName.NULL) { + return ColumnType.LONG_ARRAY; + } return null; } else { return null; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java index 27efe16270e..0fb1c9fb9ff 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java @@ -353,6 +353,7 @@ public class DruidOperatorTable implements SqlOperatorTable .add(new NestedDataOperatorConversions.JsonValueReturningArrayDoubleOperatorConversion()) .add(new NestedDataOperatorConversions.JsonValueReturningArrayVarcharOperatorConversion()) .add(new NestedDataOperatorConversions.JsonObjectOperatorConversion()) + .add(new NestedDataOperatorConversions.JsonMergeOperatorConversion()) .add(new NestedDataOperatorConversions.ToJsonStringOperatorConversion()) .add(new NestedDataOperatorConversions.ParseJsonOperatorConversion()) .add(new NestedDataOperatorConversions.TryParseJsonOperatorConversion()) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java index 82dd6afe8c9..dfc374f51bb 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java @@ -406,7 +406,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand if (plannerContext.getPlannerConfig().isUseNativeQueryExplain()) { DruidRel druidRel = (DruidRel) rel; try { - explanation = explainSqlPlanAsNativeQueries(relRoot, druidRel); + explanation = explainSqlPlanAsNativeQueries(plannerContext, relRoot, druidRel); } catch (Exception ex) { log.warn(ex, "Unable to translate to a native Druid query. Resorting to legacy Druid explain plan."); @@ -453,7 +453,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand * @return A string representing an array of native queries that correspond to the given SQL query, in JSON format * @throws JsonProcessingException */ - private String explainSqlPlanAsNativeQueries(final RelRoot relRoot, DruidRel rel) throws JsonProcessingException + private String explainSqlPlanAsNativeQueries(PlannerContext plannerContext, final RelRoot relRoot, DruidRel rel) throws JsonProcessingException { ObjectMapper jsonMapper = handlerContext.jsonMapper(); List druidQueryList; @@ -470,6 +470,9 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand for (DruidQuery druidQuery : druidQueryList) { Query nativeQuery = druidQuery.getQuery(); + + plannerContext.dispatchHook(DruidHook.NATIVE_PLAN, nativeQuery); + ObjectNode objectNode = jsonMapper.createObjectNode(); objectNode.set("query", jsonMapper.convertValue(nativeQuery, ObjectNode.class)); objectNode.set("signature", jsonMapper.convertValue(druidQuery.getOutputRowSignature(), ArrayNode.class)); @@ -582,6 +585,11 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand DruidQuery finalBaseQuery = baseQuery; final Supplier> resultsSupplier = () -> plannerContext.getQueryMaker().runQuery(finalBaseQuery); + if (explain != null) { + plannerContext.dispatchHook(DruidHook.NATIVE_PLAN, finalBaseQuery.getQuery()); + return planExplanation(possiblyLimitedRoot, newRoot, true); + } + return new PlannerResult(resultsSupplier, finalBaseQuery.getOutputRowType()); } else { final DruidRel druidRel = (DruidRel) planner.transform( diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryValidations.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryValidations.java index 1f38debfb9a..c516496af31 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryValidations.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryValidations.java @@ -75,7 +75,7 @@ public class QueryValidations if (shuttle.found != null) { throw new ValidationException( StringUtils.format( - "%s join is not supported by engine [%s] with %s: [%s]. Try %s: %s.", + "%s JOIN is not supported by engine[%s] with %s[%s]. Try %s[%s].", shuttle.found.getJoinType(), plannerContext.getEngine().name(), PlannerContext.CTX_SQL_JOIN_ALGORITHM, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/DruidQueryGenerator.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/DruidQueryGenerator.java index 001d3463919..61f6d13755b 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/DruidQueryGenerator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/querygen/DruidQueryGenerator.java @@ -36,6 +36,7 @@ import org.apache.druid.sql.calcite.rel.DruidQuery; import org.apache.druid.sql.calcite.rel.PartialDruidQuery; import org.apache.druid.sql.calcite.rel.PartialDruidQuery.Stage; import org.apache.druid.sql.calcite.rel.logical.DruidAggregate; +import org.apache.druid.sql.calcite.rel.logical.DruidJoin; import org.apache.druid.sql.calcite.rel.logical.DruidLogicalNode; import org.apache.druid.sql.calcite.rel.logical.DruidSort; @@ -58,20 +59,83 @@ public class DruidQueryGenerator this.vertexFactory = new PDQVertexFactory(plannerContext, rexBuilder); } + /** + * Tracks the upstream nodes during traversal. + * + * Its main purpose is to provide access to parent nodes; + * so that context sensitive logics can be formalized with it. + */ + static class DruidNodeStack + { + static class Entry + { + public final DruidLogicalNode node; + public final int operandIndex; + + public Entry(DruidLogicalNode node, int operandIndex) + { + this.node = node; + this.operandIndex = operandIndex; + } + } + + Stack stack = new Stack<>(); + + public void push(DruidLogicalNode item) + { + push(item, 0); + } + + public void push(DruidLogicalNode item, int operandIndex) + { + stack.push(new Entry(item, operandIndex)); + } + + public void pop() + { + stack.pop(); + } + + public int size() + { + return stack.size(); + } + + public DruidLogicalNode peekNode() + { + return stack.peek().node; + } + + public DruidLogicalNode parentNode() + { + return getNode(1).node; + } + + public Entry getNode(int i) + { + return stack.get(stack.size() - 1 - i); + } + + public int peekOperandIndex() + { + return stack.peek().operandIndex; + } + } + public DruidQuery buildQuery() { - Stack stack = new Stack<>(); + DruidNodeStack stack = new DruidNodeStack(); stack.push(relRoot); Vertex vertex = buildVertexFor(stack); return vertex.buildQuery(true); } - private Vertex buildVertexFor(Stack stack) + private Vertex buildVertexFor(DruidNodeStack stack) { List newInputs = new ArrayList<>(); - for (RelNode input : stack.peek().getInputs()) { - stack.push((DruidLogicalNode) input); + for (RelNode input : stack.peekNode().getInputs()) { + stack.push((DruidLogicalNode) input, newInputs.size()); newInputs.add(buildVertexFor(stack)); stack.pop(); } @@ -79,11 +143,11 @@ public class DruidQueryGenerator return vertex; } - private Vertex processNodeWithInputs(Stack stack, List newInputs) + private Vertex processNodeWithInputs(DruidNodeStack stack, List newInputs) { - DruidLogicalNode node = stack.peek(); + DruidLogicalNode node = stack.peekNode(); if (node instanceof SourceDescProducer) { - return vertexFactory.createVertex(PartialDruidQuery.create(node), newInputs); + return vertexFactory.createVertex(stack, PartialDruidQuery.create(node), newInputs); } if (newInputs.size() == 1) { Vertex inputVertex = newInputs.get(0); @@ -92,6 +156,7 @@ public class DruidQueryGenerator return newVertex.get(); } inputVertex = vertexFactory.createVertex( + stack, PartialDruidQuery.createOuterQuery(((PDQVertex) inputVertex).partialDruidQuery, vertexFactory.plannerContext), ImmutableList.of(inputVertex) ); @@ -116,7 +181,7 @@ public class DruidQueryGenerator /** * Extends the current vertex to include the specified parent. */ - Optional extendWith(Stack stack); + Optional extendWith(DruidNodeStack stack); /** * Decides wether this {@link Vertex} can be unwrapped into an {@link SourceDesc}. @@ -133,6 +198,42 @@ public class DruidQueryGenerator SourceDesc unwrapSourceDesc(); } + enum JoinSupportTweaks + { + NONE, + LEFT, + RIGHT; + + static JoinSupportTweaks analyze(DruidNodeStack stack) + { + if (stack.size() < 2) { + return NONE; + } + DruidLogicalNode possibleJoin = stack.parentNode(); + if (!(possibleJoin instanceof DruidJoin)) { + return NONE; + } + if (stack.peekOperandIndex() == 0) { + return LEFT; + } else { + return RIGHT; + } + } + + boolean finalizeSubQuery() + { + return this == NONE; + } + + boolean forceSubQuery(SourceDesc sourceDesc) + { + if (sourceDesc.dataSource.isGlobal()) { + return false; + } + return this == RIGHT; + } + } + /** * {@link PartialDruidQuery} based {@link Vertex} factory. */ @@ -147,20 +248,23 @@ public class DruidQueryGenerator this.rexBuilder = rexBuilder; } - Vertex createVertex(PartialDruidQuery partialDruidQuery, List inputs) + Vertex createVertex(DruidNodeStack stack, PartialDruidQuery partialDruidQuery, List inputs) { - return new PDQVertex(partialDruidQuery, inputs); + JoinSupportTweaks jst = JoinSupportTweaks.analyze(stack); + return new PDQVertex(partialDruidQuery, inputs, jst); } public class PDQVertex implements Vertex { final PartialDruidQuery partialDruidQuery; final List inputs; + final JoinSupportTweaks jst; - public PDQVertex(PartialDruidQuery partialDruidQuery, List inputs) + public PDQVertex(PartialDruidQuery partialDruidQuery, List inputs, JoinSupportTweaks jst) { this.partialDruidQuery = partialDruidQuery; this.inputs = inputs; + this.jst = jst; } @Override @@ -172,7 +276,7 @@ public class DruidQueryGenerator source.rowSignature, plannerContext, rexBuilder, - !topLevel + !(topLevel) && jst.finalizeSubQuery() ); } @@ -207,21 +311,22 @@ public class DruidQueryGenerator * Extends the the current partial query with the new parent if possible. */ @Override - public Optional extendWith(Stack stack) + public Optional extendWith(DruidNodeStack stack) { Optional newPartialQuery = extendPartialDruidQuery(stack); if (!newPartialQuery.isPresent()) { return Optional.empty(); + } - return Optional.of(createVertex(newPartialQuery.get(), inputs)); + return Optional.of(createVertex(stack, newPartialQuery.get(), inputs)); } /** * Merges the given {@link RelNode} into the current {@link PartialDruidQuery}. */ - private Optional extendPartialDruidQuery(Stack stack) + private Optional extendPartialDruidQuery(DruidNodeStack stack) { - DruidLogicalNode parentNode = stack.peek(); + DruidLogicalNode parentNode = stack.peekNode(); if (accepts(stack, Stage.WHERE_FILTER, Filter.class)) { PartialDruidQuery newPartialQuery = partialDruidQuery.withWhereFilter((Filter) parentNode); return Optional.of(newPartialQuery); @@ -261,12 +366,12 @@ public class DruidQueryGenerator return Optional.empty(); } - private boolean accepts(Stack stack, Stage stage, Class clazz) + private boolean accepts(DruidNodeStack stack, Stage stage, Class clazz) { - DruidLogicalNode currentNode = stack.peek(); + DruidLogicalNode currentNode = stack.peekNode(); if (Project.class == clazz && stack.size() >= 2) { // peek at parent and postpone project for next query stage - DruidLogicalNode parentNode = stack.get(stack.size() - 2); + DruidLogicalNode parentNode = stack.parentNode(); if (stage.ordinal() > Stage.AGGREGATE.ordinal() && parentNode instanceof DruidAggregate && !partialDruidQuery.canAccept(Stage.AGGREGATE)) { @@ -295,6 +400,9 @@ public class DruidQueryGenerator @Override public boolean canUnwrapSourceDesc() { + if (jst.forceSubQuery(getSource())) { + return false; + } if (partialDruidQuery.stage() == Stage.SCAN) { return true; } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java index ee58446b5ab..e5c403f1e1d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java @@ -463,7 +463,7 @@ public class DruidCorrelateUnnestRel extends DruidRel ) { // Compute signature of the correlation operation. It's like a join: the left and right sides are concatenated. - // On the native query side, this is what is ultimately emitted by the UnnestStorageAdapter. + // On the native query side, this is what is ultimately emitted by the UnnestSegment. // // Ignore prefix (lhs) from computeJoinRowSignature; we don't need this since we will declare the name of the // single output column directly. (And we know it's the last column in the signature.) 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 51cf9e923ed..3ce33e72245 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 @@ -85,7 +85,6 @@ import org.apache.druid.query.topn.InvertedTopNMetricSpec; import org.apache.druid.query.topn.NumericTopNMetricSpec; import org.apache.druid.query.topn.TopNMetricSpec; import org.apache.druid.query.topn.TopNQuery; -import org.apache.druid.segment.RowBasedStorageAdapter; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; @@ -916,12 +915,12 @@ public class DruidQuery ) { if (Granularities.ALL.equals(queryGranularity)) { - // Always OK: no storage adapter has problem with ALL. + // Always OK: no cursor factory has problem with ALL. return true; } if (dataSource.getAnalysis().isConcreteAndTableBased()) { - // Always OK: queries on concrete tables (regular Druid datasources) use segment-based storage adapters + // Always OK: queries on concrete tables (regular Druid datasources) use segment-based cursors // (IncrementalIndex or QueryableIndex). These clip query interval to data interval, making wide query // intervals safer. They do not have special checks for granularity and interval safety. return true; @@ -931,9 +930,9 @@ public class DruidQuery // count on interval-clipping to save us. for (final Interval filtrationInterval : filtration.getIntervals()) { - // Query may be using RowBasedStorageAdapter. We don't know for sure, so check - // RowBasedStorageAdapter#isQueryGranularityAllowed to be safe. - if (!RowBasedStorageAdapter.isQueryGranularityAllowed(filtrationInterval, queryGranularity)) { + // Query may be using RowBasedCursorFactory. We don't know for sure, so check if the interval is too big + + if (Intervals.ETERNITY.equals(filtrationInterval)) { return false; } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java index 2035bca5387..7b1e1ec7091 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java @@ -51,6 +51,7 @@ import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rel.CannotBuildQueryException; import org.apache.druid.sql.calcite.rel.DruidQuery; +import org.apache.druid.sql.hook.DruidHook; import org.joda.time.Interval; import java.util.ArrayList; @@ -171,6 +172,7 @@ public class NativeQueryMaker implements QueryMaker ) { Hook.QUERY_PLAN.run(query); + plannerContext.dispatchHook(DruidHook.NATIVE_PLAN, query); if (query.getId() == null) { final String queryId = UUID.randomUUID().toString(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java index 2477ac38dec..d02d302437b 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java @@ -109,7 +109,6 @@ public class NativeSqlEngine implements SqlEngine case ALLOW_TOP_LEVEL_UNION_ALL: case TIME_BOUNDARY_QUERY: case GROUPBY_IMPLICITLY_SORTS: - case WINDOW_LEAF_OPERATOR: return true; case CAN_INSERT: case CAN_REPLACE: @@ -117,6 +116,7 @@ public class NativeSqlEngine implements SqlEngine case WRITE_EXTERNAL_DATA: case SCAN_ORDER_BY_NON_TIME: case SCAN_NEEDS_SIGNATURE: + case WINDOW_LEAF_OPERATOR: return false; default: throw SqlEngines.generateUnrecognizedFeatureException(NativeSqlEngine.class.getSimpleName(), feature); diff --git a/sql/src/main/java/org/apache/druid/sql/hook/DruidHook.java b/sql/src/main/java/org/apache/druid/sql/hook/DruidHook.java index 1ad41502e21..e2cbe08329d 100644 --- a/sql/src/main/java/org/apache/druid/sql/hook/DruidHook.java +++ b/sql/src/main/java/org/apache/druid/sql/hook/DruidHook.java @@ -21,6 +21,7 @@ package org.apache.druid.sql.hook; import com.google.errorprone.annotations.Immutable; import org.apache.calcite.rel.RelNode; +import org.apache.druid.query.Query; import java.util.Objects; /** @@ -36,6 +37,8 @@ public interface DruidHook HookKey DRUID_PLAN = new HookKey<>("druidPlan", RelNode.class); HookKey SQL = new HookKey<>("sql", String.class); HookKey MSQ_PLAN = new HookKey<>("msqPlan", String.class); + @SuppressWarnings("rawtypes") + HookKey NATIVE_PLAN = new HookKey<>("nativePlan", Query.class); @Immutable class HookKey diff --git a/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaTestDriver.java b/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaTestDriver.java index 30eca2782e7..debd835fc78 100644 --- a/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaTestDriver.java +++ b/sql/src/test/java/org/apache/druid/quidem/DruidAvaticaTestDriver.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Provides; +import com.google.inject.name.Named; import com.google.inject.name.Names; import org.apache.calcite.avatica.server.AbstractAvaticaHandler; import org.apache.druid.guice.DruidInjectorBuilder; @@ -120,9 +121,13 @@ public class DruidAvaticaTestDriver implements Driver @Provides @LazySingleton - public DruidConnectionExtras getConnectionExtras(ObjectMapper objectMapper, DruidHookDispatcher druidHookDispatcher) + public DruidConnectionExtras getConnectionExtras( + ObjectMapper objectMapper, + DruidHookDispatcher druidHookDispatcher, + @Named("isExplainSupported") Boolean isExplainSupported + ) { - return new DruidConnectionExtras.DruidConnectionExtrasImpl(objectMapper, druidHookDispatcher); + return new DruidConnectionExtras.DruidConnectionExtrasImpl(objectMapper, druidHookDispatcher, isExplainSupported); } @Provides @@ -281,6 +286,12 @@ public class DruidAvaticaTestDriver implements Driver { return delegate.getPlannerComponentSupplier(); } + + @Override + public Boolean isExplainSupported() + { + return delegate.isExplainSupported(); + } } protected File createTempFolder(String prefix) diff --git a/sql/src/test/java/org/apache/druid/quidem/DruidConnectionExtras.java b/sql/src/test/java/org/apache/druid/quidem/DruidConnectionExtras.java index 176411a6bb7..287e125aa4d 100644 --- a/sql/src/test/java/org/apache/druid/quidem/DruidConnectionExtras.java +++ b/sql/src/test/java/org/apache/druid/quidem/DruidConnectionExtras.java @@ -30,15 +30,19 @@ public interface DruidConnectionExtras DruidHookDispatcher getDruidHookDispatcher(); + boolean isExplainSupported(); + class DruidConnectionExtrasImpl implements DruidConnectionExtras { private final ObjectMapper objectMapper; private final DruidHookDispatcher druidHookDispatcher; + private final boolean isExplainSupported; - public DruidConnectionExtrasImpl(ObjectMapper objectMapper, DruidHookDispatcher druidHookDispatcher) + public DruidConnectionExtrasImpl(ObjectMapper objectMapper, DruidHookDispatcher druidHookDispatcher, boolean isExplainSupported) { this.objectMapper = objectMapper; this.druidHookDispatcher = druidHookDispatcher; + this.isExplainSupported = isExplainSupported; } @Override @@ -52,6 +56,12 @@ public interface DruidConnectionExtras { return druidHookDispatcher; } + + @Override + public boolean isExplainSupported() + { + return isExplainSupported; + } } static DruidConnectionExtras unwrapOrThrow(Connection connection) @@ -61,4 +71,5 @@ public interface DruidConnectionExtras } throw new UnsupportedOperationException("Expected DruidConnectionExtras to be implemented by connection!"); } + } diff --git a/sql/src/test/java/org/apache/druid/quidem/DruidQuidemCommandHandler.java b/sql/src/test/java/org/apache/druid/quidem/DruidQuidemCommandHandler.java index 8c6155cd6a4..215182b8e28 100644 --- a/sql/src/test/java/org/apache/druid/quidem/DruidQuidemCommandHandler.java +++ b/sql/src/test/java/org/apache/druid/quidem/DruidQuidemCommandHandler.java @@ -34,7 +34,6 @@ import org.apache.calcite.util.Util; import org.apache.druid.query.Query; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.rel.DruidRel; -import org.apache.druid.sql.calcite.util.QueryLogHook; import org.apache.druid.sql.hook.DruidHook; import org.apache.druid.sql.hook.DruidHook.HookKey; import org.apache.druid.sql.hook.DruidHookDispatcher; @@ -45,11 +44,9 @@ import java.sql.ResultSet; import java.sql.Statement; import java.util.ArrayList; import java.util.List; -import java.util.stream.Collectors; public class DruidQuidemCommandHandler implements CommandHandler { - @Override public Command parseCommand(List lines, List content, String line) { @@ -83,33 +80,63 @@ public class DruidQuidemCommandHandler implements CommandHandler } @Override - public final String describe(Context x) + public final String describe(Context context) { - return commandName() + " [sql: " + x.previousSqlCommand().sql + "]"; + return commandName() + " [sql: " + context.previousSqlCommand().sql + "]"; } @Override - public final void execute(Context x, boolean execute) + public final void execute(Context context, boolean execute) { if (execute) { try { - executeExplain(x); + executeExplain(context); } catch (Exception e) { throw new Error(e); } } else { - x.echo(content); + context.echo(content); } - x.echo(lines); + context.echo(lines); } - protected final void executeQuery(Context x) + protected final List executeExplainCollectHookValues(Context context, HookKey hook) throws IOException + { + DruidHookDispatcher dhp = unwrapDruidHookDispatcher(context); + List logged = new ArrayList<>(); + try (Closeable unhook = dhp.withHook(hook, (key, value) -> { + logged.add(value); + })) { + executeExplainQuery(context); + } + return logged; + } + + protected final void executeQuery(Context context) + { + final SqlCommand sqlCommand = context.previousSqlCommand(); + executeQuery(context, sqlCommand.sql); + } + + protected final void executeExplainQuery(Context context) + { + boolean isExplainSupported = DruidConnectionExtras.unwrapOrThrow(context.connection()).isExplainSupported(); + + final SqlCommand sqlCommand = context.previousSqlCommand(); + + if (isExplainSupported) { + executeQuery(context, "explain plan for " + sqlCommand.sql); + } else { + executeQuery(context, sqlCommand.sql); + } + } + + protected final void executeQuery(Context context, String sql) { - final SqlCommand sqlCommand = x.previousSqlCommand(); try ( - final Statement statement = x.connection().createStatement(); - final ResultSet resultSet = statement.executeQuery(sqlCommand.sql)) { + final Statement statement = context.connection().createStatement(); + final ResultSet resultSet = statement.executeQuery(sql)) { // throw away all results while (resultSet.next()) { Util.discard(false); @@ -120,12 +147,12 @@ public class DruidQuidemCommandHandler implements CommandHandler } } - protected final DruidHookDispatcher unwrapDruidHookDispatcher(Context x) + protected final DruidHookDispatcher unwrapDruidHookDispatcher(Context context) { - return DruidConnectionExtras.unwrapOrThrow(x.connection()).getDruidHookDispatcher(); + return DruidConnectionExtras.unwrapOrThrow(context.connection()).getDruidHookDispatcher(); } - protected abstract void executeExplain(Context x) throws Exception; + protected abstract void executeExplain(Context context) throws Exception; } /** Command that prints the plan for the current query. */ @@ -137,27 +164,18 @@ public class DruidQuidemCommandHandler implements CommandHandler } @Override - protected void executeExplain(Context x) throws Exception + @SuppressWarnings("rawtypes") + protected void executeExplain(Context context) throws Exception { - DruidConnectionExtras connectionExtras = (DruidConnectionExtras) x.connection(); + DruidConnectionExtras connectionExtras = DruidConnectionExtras.unwrapOrThrow(context.connection()); ObjectMapper objectMapper = connectionExtras.getObjectMapper(); - QueryLogHook qlh = new QueryLogHook(objectMapper); - qlh.logQueriesForGlobal( - () -> { - executeQuery(x); - } - ); - List> queries = qlh.getRecordedQueries(); + List logged = executeExplainCollectHookValues(context, DruidHook.NATIVE_PLAN); - queries = queries - .stream() - .map(q -> BaseCalciteQueryTest.recursivelyClearContext(q, objectMapper)) - .collect(Collectors.toList()); - - for (Query query : queries) { + for (Query query : logged) { + query = BaseCalciteQueryTest.recursivelyClearContext(query, objectMapper); String str = objectMapper.writerWithDefaultPrettyPrinter().writeValueAsString(query); - x.echo(ImmutableList.of(str)); + context.echo(ImmutableList.of(str)); } } } @@ -176,22 +194,16 @@ public class DruidQuidemCommandHandler implements CommandHandler } @Override - protected final void executeExplain(Context x) throws IOException + protected final void executeExplain(Context context) throws IOException { - DruidHookDispatcher dhp = unwrapDruidHookDispatcher(x); - List logged = new ArrayList<>(); - try (Closeable unhook = dhp.withHook(hook, (key, relNode) -> { - logged.add(relNode); - })) { - executeQuery(x); - } + List logged = executeExplainCollectHookValues(context, hook); for (RelNode node : logged) { if (node instanceof DruidRel) { - node = ((DruidRel) node).unwrapLogicalPlan(); + node = ((DruidRel) node).unwrapLogicalPlan(); } String str = RelOptUtil.dumpPlan("", node, SqlExplainFormat.TEXT, SqlExplainLevel.EXPPLAN_ATTRIBUTES); - x.echo(ImmutableList.of(str)); + context.echo(ImmutableList.of(str)); } } } @@ -210,17 +222,10 @@ public class DruidQuidemCommandHandler implements CommandHandler } @Override - protected final void executeExplain(Context x) throws IOException + protected final void executeExplain(Context context) throws IOException { - DruidHookDispatcher dhp = unwrapDruidHookDispatcher(x); - List logged = new ArrayList<>(); - try (Closeable unhook = dhp.withHook(hook, (key, relNode) -> { - logged.add(relNode); - })) { - executeQuery(x); - } - - x.echo(logged); + List logged = executeExplainCollectHookValues(context, hook); + context.echo(logged); } } 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 fcaaf044824..4b26af38adb 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 @@ -35,12 +35,14 @@ import org.apache.druid.error.DruidException.Persona; import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.hll.VersionOneHyperLogLogCollector; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.math.expr.Evals; +import org.apache.druid.math.expr.ExprEval; import org.apache.druid.query.DataSource; import org.apache.druid.query.Druids; import org.apache.druid.query.JoinDataSource; @@ -982,23 +984,55 @@ public class BaseCalciteQueryTest extends CalciteTestBase mismatchMessage(row, column), (Float) expectedCell, (Float) resultCell, - ASSERTION_EPSILON); + ASSERTION_EPSILON + ); } else if (expectedCell instanceof Double) { assertEquals( mismatchMessage(row, column), (Double) expectedCell, (Double) resultCell, - ASSERTION_EPSILON); + ASSERTION_EPSILON + ); + } else if (expectedCell instanceof Object[] || expectedCell instanceof List) { + final Object[] expectedCellCasted = homogenizeArray(expectedCell); + final Object[] resultCellCasted = homogenizeArray(resultCell); + if (expectedCellCasted.length != resultCellCasted.length) { + throw new RE( + "Mismatched array lengths: expected[%s] with length[%d], actual[%s] with length[%d]", + Arrays.toString(expectedCellCasted), + expectedCellCasted.length, + Arrays.toString(resultCellCasted), + resultCellCasted.length + ); + } + for (int i = 0; i < expectedCellCasted.length; ++i) { + validate(row, column, type, expectedCellCasted[i], resultCellCasted[i]); + } } else { EQUALS.validate(row, column, type, expectedCell, resultCell); } } }, + + RELAX_NULLS_EPS { + @Override + void validate(int row, int column, ValueType type, Object expectedCell, Object resultCell) + { + if (expectedCell == null) { + if (resultCell == null) { + return; + } + expectedCell = NullHandling.defaultValueForType(type); + } + EQUALS_EPS.validate(row, column, type, expectedCell, resultCell); + } + }, + /** * Comparision which accepts 1000 units of least precision. */ EQUALS_RELATIVE_1000_ULPS { - static final int ASSERTION_ERROR_ULPS = 1000; + private static final int ASSERTION_ERROR_ULPS = 1000; @Override void validate(int row, int column, ValueType type, Object expectedCell, Object resultCell) @@ -1019,10 +1053,43 @@ public class BaseCalciteQueryTest extends CalciteTestBase (Double) resultCell, eps ); + } else if (expectedCell instanceof Object[] || expectedCell instanceof List) { + final Object[] expectedCellCasted = homogenizeArray(expectedCell); + final Object[] resultCellCasted = homogenizeArray(resultCell); + + if (expectedCellCasted.length != resultCellCasted.length) { + throw new RE( + "Mismatched array lengths: expected[%s] with length[%d], actual[%s] with length[%d]", + Arrays.toString(expectedCellCasted), + expectedCellCasted.length, + Arrays.toString(resultCellCasted), + resultCellCasted.length + ); + } + for (int i = 0; i < expectedCellCasted.length; ++i) { + validate(row, column, type, expectedCellCasted[i], resultCellCasted[i]); + } } else { EQUALS.validate(row, column, type, expectedCell, resultCell); } } + }, + + /** + * Relax nulls which accepts 1000 units of least precision. + */ + RELAX_NULLS_RELATIVE_1000_ULPS { + @Override + void validate(int row, int column, ValueType type, Object expectedCell, Object resultCell) + { + if (expectedCell == null) { + if (resultCell == null) { + return; + } + expectedCell = NullHandling.defaultValueForType(type); + } + EQUALS_RELATIVE_1000_ULPS.validate(row, column, type, expectedCell, resultCell); + } }; abstract void validate(int row, int column, ValueType type, Object expectedCell, Object resultCell); @@ -1032,6 +1099,15 @@ public class BaseCalciteQueryTest extends CalciteTestBase return StringUtils.format("column content mismatch at %d,%d", row, column); } + private static Object[] homogenizeArray(Object array) + { + if (array instanceof Object[]) { + return (Object[]) array; + } else if (array instanceof List) { + return ExprEval.coerceListToArray((List) array, true).rhs; + } + throw new ISE("Found array[%s] of type[%s] which is not handled", array.toString(), array.getClass().getName()); + } } /** @@ -1217,11 +1293,6 @@ public class BaseCalciteQueryTest extends CalciteTestBase skipVectorize = true; } - protected void sqlNativeIncompatible() - { - assumeTrue(testBuilder().config.isRunningMSQ(), "test case is not SQL native compatible"); - } - protected void msqIncompatible() { assumeFalse(testBuilder().config.isRunningMSQ(), "test case is not MSQ compatible"); 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 2cd798e193a..d3453404cd9 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 @@ -116,8 +116,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest } } - // test some query stuffs, sort of limited since no native array column types so either need to use constructor or - // array aggregator @Test public void testSelectConstantArrayExpressionFromTable() { @@ -7478,4 +7476,26 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ) ); } + + @Test + public void testNullArray() + { + testQuery( + "SELECT arrayLongNulls = ARRAY[null, null] FROM druid.arrays LIMIT 1", + ImmutableList.of( + newScanQueryBuilder() + .dataSource(CalciteTests.ARRAYS_DATASOURCE) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns(expressionVirtualColumn("v0", "(\"arrayLongNulls\" == array(null,null))", ColumnType.LONG)) + .columns("v0") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .limit(1) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{false} + ) + ); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java index 813722e6990..60daa769b61 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java @@ -479,7 +479,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml { ExternalDataSource externalDataSource = new ExternalDataSource( new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"), - new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0), + new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null), RowSignature.builder() .add("a", ColumnType.STRING) .add("b", ColumnType.STRING) @@ -541,7 +541,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml { ExternalDataSource externalDataSource = new ExternalDataSource( new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"), - new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0), + new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null), RowSignature.builder() .add("a", ColumnType.STRING) .add("b", ColumnType.STRING) @@ -603,7 +603,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml { ExternalDataSource externalDataSource = new ExternalDataSource( new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"), - new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0), + new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null), RowSignature.builder() .add("a", ColumnType.STRING) .add("b", ColumnType.STRING) @@ -665,7 +665,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml { ExternalDataSource externalDataSource = new ExternalDataSource( new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"), - new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0), + new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null), RowSignature.builder() .add("a", ColumnType.STRING) .add("b", ColumnType.STRING) @@ -834,7 +834,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml { ExternalDataSource externalDataSource = new ExternalDataSource( new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"), - new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0), + new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null), RowSignature.builder() .add("a", ColumnType.STRING) .add("b", ColumnType.STRING) @@ -953,7 +953,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml { ExternalDataSource externalDataSource = new ExternalDataSource( new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"), - new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0), + new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null), RowSignature.builder() .add("a", ColumnType.STRING) .add("b", ColumnType.STRING) @@ -1012,7 +1012,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml { ExternalDataSource externalDataSource = new ExternalDataSource( new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"), - new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0), + new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null), RowSignature.builder() .add("a", ColumnType.STRING) .add("b", ColumnType.STRING) @@ -1077,7 +1077,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml { ExternalDataSource externalDataSource = new ExternalDataSource( new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"), - new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0), + new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0, null), RowSignature.builder() .add("a", ColumnType.STRING) .add("b", ColumnType.STRING) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteIngestionDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteIngestionDmlTest.java index e26803fa498..8edbce19996 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteIngestionDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteIngestionDmlTest.java @@ -116,7 +116,7 @@ public class CalciteIngestionDmlTest extends BaseCalciteQueryTest protected final ExternalDataSource externalDataSource = new ExternalDataSource( new InlineInputSource("a,b,1\nc,d,2\n"), - new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0), + new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null), RowSignature.builder() .add("x", ColumnType.STRING) .add("y", ColumnType.STRING) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index ed4aa6d91eb..61d955c3be7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -369,7 +369,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest ), Calcites.escapeStringLiteral( queryJsonMapper.writeValueAsString( - new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0) + new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null) ) ) ); @@ -414,7 +414,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest ), Calcites.escapeStringLiteral( queryJsonMapper.writeValueAsString( - new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0) + new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null) ) ) ); @@ -458,7 +458,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest new InlineInputSource("a,b,1\nc,d,2\n") ), queryJsonMapper.writeValueAsString( - new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0) + new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null) ) ); } @@ -494,7 +494,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest new TestFileInputSource(ImmutableList.of(new File("/tmp/foo.csv").getAbsoluteFile())); final ExternalDataSource externalDataSource = new ExternalDataSource( inputSource, - new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0), + new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null), RowSignature.builder() .add("x", ColumnType.STRING) .add("y", ColumnType.STRING) @@ -510,7 +510,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest + "inputFormat => '%s'))", queryJsonMapper.writeValueAsString(inputSource), queryJsonMapper.writeValueAsString( - new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0) + new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null) ) ); } @@ -552,7 +552,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest queryJsonMapper.writeValueAsString( new TestFileInputSource(ImmutableList.of(new File("/tmp/foo.csv").getAbsoluteFile()))), queryJsonMapper.writeValueAsString( - new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0) + new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null) ) ); } @@ -860,7 +860,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest // Test correctness of the query when only the CLUSTERED BY clause is present - final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"join\",\"left\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://boo.gz\"]},\"inputFormat\":{\"type\":\"json\"},\"signature\":[{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}]},\"right\":{\"type\":\"query\",\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://foo.tsv\"]},\"inputFormat\":{\"type\":\"tsv\",\"delimiter\":\"\\t\",\"findColumnsFromHeader\":true},\"signature\":[{\"name\":\"Country\",\"type\":\"STRING\"},{\"name\":\"Capital\",\"type\":\"STRING\"},{\"name\":\"ISO3\",\"type\":\"STRING\"},{\"name\":\"ISO2\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false}},\"rightPrefix\":\"j0.\",\"condition\":\"(\\\"countryIsoCode\\\" == \\\"j0.ISO2\\\")\",\"joinType\":\"LEFT\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"timestamp_parse(\\\"timestamp\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"isRobot\",\"order\":\"ascending\"},{\"columnName\":\"j0.Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"isRobot\",\"j0.Capital\",\"regionName\",\"v0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"j0.Capital\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"isRobot\",\"outputColumn\":\"isRobotAlias\"},{\"queryColumn\":\"j0.Capital\",\"outputColumn\":\"countryCapital\"},{\"queryColumn\":\"regionName\",\"outputColumn\":\"regionName\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"join\",\"left\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://boo.gz\"],\"requestHeaders\":{}},\"inputFormat\":{\"type\":\"json\"},\"signature\":[{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}]},\"right\":{\"type\":\"query\",\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://foo.tsv\"],\"requestHeaders\":{}},\"inputFormat\":{\"type\":\"tsv\",\"delimiter\":\"\\t\",\"findColumnsFromHeader\":true},\"signature\":[{\"name\":\"Country\",\"type\":\"STRING\"},{\"name\":\"Capital\",\"type\":\"STRING\"},{\"name\":\"ISO3\",\"type\":\"STRING\"},{\"name\":\"ISO2\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false}},\"rightPrefix\":\"j0.\",\"condition\":\"(\\\"countryIsoCode\\\" == \\\"j0.ISO2\\\")\",\"joinType\":\"LEFT\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"timestamp_parse(\\\"timestamp\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"isRobot\",\"order\":\"ascending\"},{\"columnName\":\"j0.Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"isRobot\",\"j0.Capital\",\"regionName\",\"v0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"j0.Capital\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"isRobot\",\"outputColumn\":\"isRobotAlias\"},{\"queryColumn\":\"j0.Capital\",\"outputColumn\":\"countryCapital\"},{\"queryColumn\":\"regionName\",\"outputColumn\":\"regionName\"}]}]"; testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, @@ -1488,7 +1488,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest { ExternalDataSource restrictedSignature = new ExternalDataSource( new InlineInputSource("100\nc200\n"), - new CsvInputFormat(ImmutableList.of("__time"), null, false, false, 0), + new CsvInputFormat(ImmutableList.of("__time"), null, false, false, 0, null), RowSignature.builder() .add("__time", ColumnType.STRING) .build() 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 04113067cb1..04cea5e2c13 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 @@ -232,8 +232,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN_EXTRA_COLUMNS, separateDefaultModeTest = true) @Test - @NotYetSupported(Modes.STACK_OVERFLOW) public void testJoinOuterGroupByAndSubqueryHasLimit() { // Cannot vectorize JOIN operator. @@ -321,7 +321,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") - @NotYetSupported(Modes.JOIN_TABLE_TABLE) public void testJoinOuterGroupByAndSubqueryNoLimit(Map queryContext) { // Fully removing the join allows this query to vectorize. @@ -405,7 +404,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest } @Test - @NotYetSupported(Modes.JOIN_TABLE_TABLE) public void testJoinWithLimitBeforeJoining() { // Cannot vectorize JOIN operator. @@ -1532,7 +1530,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.FINALIZING_FIELD_ACCESS) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinQueryOfLookup(Map queryContext) @@ -1712,7 +1709,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_TABLE_TABLE) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN_CAST_MATERIALIZED_EARLIER) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse(Map queryContext) @@ -1770,7 +1767,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_TABLE_TABLE) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinLookupTableTable(Map queryContext) @@ -1853,7 +1849,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_TABLE_TABLE) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoinLookupTableTableChained(Map queryContext) @@ -2082,7 +2077,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_TABLE_TABLE) + @DecoupledTestConfig(quidemReason = QuidemTestCaseReason.EQUIV_PLAN_CAST_MATERIALIZED_EARLIER) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testJoinTableLookupTableMismatchedTypesWithoutComma(Map queryContext) @@ -3508,7 +3503,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest @Test public void testLeftJoinRightTableCanBeEmpty() { - // HashJoinSegmentStorageAdapter is not vectorizable + // HashJoinSegmentCursorFactory is not vectorizable cannotVectorize(); final DataSource rightTable; @@ -3729,7 +3724,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_TABLE_TABLE) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testLeftJoinWithNotNullFilter(Map queryContext) @@ -3777,7 +3771,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_TABLE_TABLE) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testInnerJoin(Map queryContext) @@ -3832,7 +3825,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest ); } - @NotYetSupported(Modes.JOIN_TABLE_TABLE) @MethodSource("provideQueryContexts") @ParameterizedTest(name = "{0}") public void testJoinWithExplicitIsNotDistinctFromCondition(Map queryContext) @@ -5845,7 +5837,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest @SqlTestFrameworkConfig.MinTopNThreshold(1) @Test - @NotYetSupported(Modes.JOIN_TABLE_TABLE) public void testJoinWithAliasAndOrderByNoGroupBy() { Map context = new HashMap<>(QUERY_CONTEXT_DEFAULT); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index 88ad05a4a27..b03b6698b1d 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -69,6 +69,7 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.segment.nested.NestedPathField; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.virtual.NestedFieldVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -4920,6 +4921,55 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testJsonMerging() + { + testQuery( + "SELECT " + + "JSON_MERGE('{\"a\":\"x\"}',JSON_OBJECT(KEY 'x' VALUE JSON_VALUE(nest, '$.x')))\n" + + "FROM druid.nested", + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns( + new ExpressionVirtualColumn( + "v0", + "json_merge('{\\u0022a\\u0022:\\u0022x\\u0022}',json_object('x',\"v1\"))", + ColumnType.NESTED_DATA, + queryFramework().macroTable() + ), + new NestedFieldVirtualColumn( + "nest", + "v1", + ColumnType.STRING, + ImmutableList.of( + new NestedPathField("x") + ), + false, + null, + false + ) + ) + .columns("v0") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .build() + ), + ImmutableList.of( + new Object[]{"{\"a\":\"x\",\"x\":\"100\"}"}, + new Object[]{"{\"a\":\"x\",\"x\":null}"}, + new Object[]{"{\"a\":\"x\",\"x\":\"200\"}"}, + new Object[]{"{\"a\":\"x\",\"x\":null}"}, + new Object[]{"{\"a\":\"x\",\"x\":null}"}, + new Object[]{"{\"a\":\"x\",\"x\":\"100\"}"}, + new Object[]{"{\"a\":\"x\",\"x\":null}"} + ), + RowSignature.builder() + .add("EXPR$0", ColumnType.NESTED_DATA) + .build() + ); + } + @Test public void testCompositionTyping() { 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 58d29a738c0..732681de238 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 @@ -15550,6 +15550,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ) ); } + @Test public void testGroupByDateTrunc() { @@ -16085,11 +16086,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .run(); } - @NotYetSupported(Modes.CANNOT_RETRIEVE_ROWS) + @NotYetSupported(Modes.UNSUPPORTED_DATASOURCE) @Test public void testWindowingOverJoin() { - msqIncompatible(); testBuilder() .sql("with " + "main as " 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 165b4aa3f63..cd6aa514675 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 @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.druid.error.DruidException; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RE; @@ -70,9 +71,17 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest private static final Map DEFAULT_QUERY_CONTEXT = ImmutableMap.of( PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true + QueryContexts.ENABLE_DEBUG, true, + QueryContexts.CTX_SQL_STRINGIFY_ARRAYS, false ); + private static final Map DEFAULT_QUERY_CONTEXT_WITH_SUBQUERY_BYTES = + ImmutableMap.builder() + .putAll(DEFAULT_QUERY_CONTEXT) + .put(QueryContexts.MAX_SUBQUERY_BYTES_KEY, "100000") + .put(QueryContexts.MAX_SUBQUERY_ROWS_KEY, "0") + .build(); + public static Object[] parametersForWindowQueryTest() throws Exception { final URL windowFolderUrl = ClassLoader.getSystemResource("calcite/tests/window"); @@ -160,7 +169,7 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest } } } - assertResultsValid(ResultMatchMode.RELAX_NULLS, input.expectedResults, results); + assertResultsValid(ResultMatchMode.RELAX_NULLS_EPS, input.expectedResults, results); } private void validateOperators(List expectedOperators, List currentOperators) @@ -222,6 +231,30 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest } } + @MethodSource("parametersForWindowQueryTest") + @ParameterizedTest(name = "{0}") + @SuppressWarnings("unchecked") + public void windowQueryTestsWithSubqueryBytes(String filename) throws Exception + { + TestCase testCase = new TestCase(filename); + + assumeTrue(testCase.getType() != TestType.failingTest); + + if (testCase.getType() == TestType.operatorValidation) { + testBuilder() + .skipVectorize(true) + .sql(testCase.getSql()) + .queryContext( + ImmutableMap.builder() + .putAll(DEFAULT_QUERY_CONTEXT_WITH_SUBQUERY_BYTES) + .putAll(testCase.getQueryContext()) + .build() + ) + .addCustomVerification(QueryVerification.ofResults(testCase)) + .run(); + } + } + @Test public void testWithArrayConcat() { @@ -236,19 +269,41 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest .expectedResults( ResultMatchMode.RELAX_NULLS, ImmutableList.of( - new Object[]{"Austria", null, "#de.wikipedia", "[\"abc\",\"#de.wikipedia\"]"}, - new Object[]{"Republic of Korea", null, "#en.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#en.wikipedia\",\"abc\",\"#ja.wikipedia\",\"abc\",\"#ko.wikipedia\"]"}, - new Object[]{"Republic of Korea", null, "#ja.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#en.wikipedia\",\"abc\",\"#ja.wikipedia\",\"abc\",\"#ko.wikipedia\"]"}, - new Object[]{"Republic of Korea", null, "#ko.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#en.wikipedia\",\"abc\",\"#ja.wikipedia\",\"abc\",\"#ko.wikipedia\"]"}, - new Object[]{"Republic of Korea", "Seoul", "#ko.wikipedia", "[\"abc\",\"#ko.wikipedia\"]"}, - new Object[]{"Austria", "Vienna", "#de.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#es.wikipedia\",\"abc\",\"#tr.wikipedia\"]"}, - new Object[]{"Austria", "Vienna", "#es.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#es.wikipedia\",\"abc\",\"#tr.wikipedia\"]"}, - new Object[]{"Austria", "Vienna", "#tr.wikipedia", "[\"abc\",\"#de.wikipedia\",\"abc\",\"#es.wikipedia\",\"abc\",\"#tr.wikipedia\"]"} + new Object[]{"Austria", null, "#de.wikipedia", ImmutableList.of("abc", "#de.wikipedia")}, + new Object[]{"Republic of Korea", null, "#en.wikipedia", ImmutableList.of("abc", "#de.wikipedia", "abc", "#en.wikipedia", "abc", "#ja.wikipedia", "abc", "#ko.wikipedia")}, + new Object[]{"Republic of Korea", null, "#ja.wikipedia", ImmutableList.of("abc", "#de.wikipedia", "abc", "#en.wikipedia", "abc", "#ja.wikipedia", "abc", "#ko.wikipedia")}, + new Object[]{"Republic of Korea", null, "#ko.wikipedia", ImmutableList.of("abc", "#de.wikipedia", "abc", "#en.wikipedia", "abc", "#ja.wikipedia", "abc", "#ko.wikipedia")}, + new Object[]{"Republic of Korea", "Seoul", "#ko.wikipedia", ImmutableList.of("abc", "#ko.wikipedia")}, + new Object[]{"Austria", "Vienna", "#de.wikipedia", ImmutableList.of("abc", "#de.wikipedia", "abc", "#es.wikipedia", "abc", "#tr.wikipedia")}, + new Object[]{"Austria", "Vienna", "#es.wikipedia", ImmutableList.of("abc", "#de.wikipedia", "abc", "#es.wikipedia", "abc", "#tr.wikipedia")}, + new Object[]{"Austria", "Vienna", "#tr.wikipedia", ImmutableList.of("abc", "#de.wikipedia", "abc", "#es.wikipedia", "abc", "#tr.wikipedia")} ) ) .run(); } + @Test + public void testFailure_partitionByMVD() + { + final DruidException e = Assert.assertThrows( + DruidException.class, + () -> testBuilder() + .sql("select cityName, countryName, array_to_mv(array[1,length(cityName)]),\n" + + "row_number() over (partition by array_to_mv(array[1,length(cityName)]) order by countryName, cityName)\n" + + "from wikipedia\n" + + "where countryName in ('Austria', 'Republic of Korea') and cityName is not null\n" + + "order by 1, 2, 3") + .queryContext(DEFAULT_QUERY_CONTEXT) + .run() + ); + + assertEquals( + "Encountered a multi value column. Window processing does not support MVDs. " + + "Consider using UNNEST or MV_TO_ARRAY.", + e.getMessage() + ); + } + private WindowOperatorQuery getWindowOperatorQuery(List> queries) { assertEquals(1, queries.size()); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java index f23197d437a..14ae52def5f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledTestConfig.java @@ -21,8 +21,6 @@ package org.apache.druid.sql.calcite; import org.apache.calcite.rel.rules.CoreRules; import org.apache.druid.query.QueryContexts; -import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; - import java.lang.annotation.ElementType; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; @@ -87,9 +85,18 @@ public @interface DecoupledTestConfig */ DEFINETLY_WORSE_PLAN, /** - * A new {@link FinalizingFieldAccessPostAggregator} appeared in the plan. + * Some extra unused columns are being projected. + * + * Example: ScanQuery over a join projects columns=[dim2, j0.m1, m1, m2] instead of just columns=[dim2, m2] */ - FINALIZING_FIELD_ACCESS; + EQUIV_PLAN_EXTRA_COLUMNS, + /** + * Materialization of a CAST was pushed down to a join branch + * + * instead of joining on condition (CAST("j0.k", 'DOUBLE') == "_j0.m1") + * a vc was computed for CAST("j0.k", 'DOUBLE') + */ + EQUIV_PLAN_CAST_MATERIALIZED_EARLIER; public boolean isPresent() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java index 0cc30c49c34..01bab3f99de 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java @@ -7731,13 +7731,10 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest windowQueryTest(); } - // This test gives the following error on sql-native engine: - // Column[w0] of type[class org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn] cannot be sorted. @DrillTest("druid_queries/empty_and_non_empty_over/wikipedia_query_1") @Test public void test_empty_and_non_empty_over_wikipedia_query_1() { - sqlNativeIncompatible(); windowQueryTest(); } @@ -7803,4 +7800,18 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest { windowQueryTest(); } + + @DrillTest("druid_queries/array_concat_agg/only_sorting_column_1") + @Test + public void test_array_concat_agg_with_only_sorting_column_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/array_concat_agg/empty_over_1") + @Test + public void test_array_concat_agg_with_empty_over_1() + { + windowQueryTest(); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java index 29200730fee..c5caa3258ce 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java @@ -20,8 +20,14 @@ package org.apache.druid.sql.calcite; import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import com.google.inject.Binder; import org.apache.calcite.avatica.SqlType; import org.apache.druid.catalog.model.Columns; import org.apache.druid.data.input.impl.CsvInputFormat; @@ -30,20 +36,28 @@ import org.apache.druid.data.input.impl.HttpInputSourceConfig; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.systemfield.SystemFields; +import org.apache.druid.guice.DruidInjectorBuilder; +import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.metadata.DefaultPasswordProvider; +import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.sql.calcite.external.ExternalDataSource; +import org.apache.druid.sql.calcite.external.ExternalOperatorConversion; import org.apache.druid.sql.calcite.external.Externals; +import org.apache.druid.sql.calcite.external.HttpOperatorConversion; +import org.apache.druid.sql.calcite.external.InlineOperatorConversion; +import org.apache.druid.sql.calcite.external.LocalOperatorConversion; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.guice.SqlBindings; import org.apache.druid.sql.http.SqlParameter; import org.hamcrest.CoreMatchers; import org.junit.internal.matchers.ThrowableMessageMatcher; @@ -52,8 +66,10 @@ import org.junit.jupiter.api.Test; import java.io.File; import java.net.URI; import java.net.URISyntaxException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.List; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -68,6 +84,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; * query ensure that the resulting MSQ task is identical regardless of the path * taken. */ +@SqlTestFrameworkConfig.ComponentSupplier(IngestTableFunctionTest.ExportComponentSupplier.class) public class IngestTableFunctionTest extends CalciteIngestionDmlTest { protected static URI toURI(String uri) @@ -86,9 +103,24 @@ public class IngestTableFunctionTest extends CalciteIngestionDmlTest "bob", new DefaultPasswordProvider("secret"), SystemFields.none(), - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ), - new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0), + new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null), + RowSignature.builder() + .add("x", ColumnType.STRING) + .add("y", ColumnType.STRING) + .add("z", ColumnType.LONG) + .build() + ); + protected final ExternalDataSource localDataSource = new ExternalDataSource( + new LocalInputSource( + null, + null, + Arrays.asList(new File("/tmp/foo.csv"), new File("/tmp/bar.csv")), + SystemFields.none() + ), + new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0, null), RowSignature.builder() .add("x", ColumnType.STRING) .add("y", ColumnType.STRING) @@ -259,9 +291,10 @@ public class IngestTableFunctionTest extends CalciteIngestionDmlTest "bob", new DefaultPasswordProvider("secret"), SystemFields.none(), - new HttpInputSourceConfig(null) + ImmutableMap.of("Accept", "application/ndjson", "a", "b"), + new HttpInputSourceConfig(null, Sets.newHashSet("Accept", "a")) ), - new CsvInputFormat(ImmutableList.of("timestamp", "isRobot"), null, false, false, 0), + new CsvInputFormat(ImmutableList.of("timestamp", "isRobot"), null, false, false, 0, null), RowSignature.builder() .add("timestamp", ColumnType.STRING) .add("isRobot", ColumnType.STRING) @@ -280,7 +313,8 @@ public class IngestTableFunctionTest extends CalciteIngestionDmlTest " userName => 'bob',\n" + " password => 'secret',\n" + " uris => ARRAY['http://example.com/foo.csv', 'http://example.com/bar.csv'],\n" + - " format => 'csv'\n" + + " format => 'csv',\n" + + " headers=> '{\"Accept\":\"application/ndjson\", \"a\": \"b\" }'\n" + " )\n" + ") EXTEND (\"timestamp\" VARCHAR, isRobot VARCHAR)\n" + "PARTITIONED BY HOUR") @@ -313,7 +347,7 @@ public class IngestTableFunctionTest extends CalciteIngestionDmlTest " format => 'csv'))\n" + " EXTEND (x VARCHAR, y VARCHAR, z BIGINT)\n" + "PARTITIONED BY ALL TIME"; - final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"http://foo.com/bar.csv\"],\"httpAuthenticationUsername\":\"bob\",\"httpAuthenticationPassword\":{\"type\":\"default\",\"password\":\"secret\"}},\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"http://foo.com/bar.csv\"],\"httpAuthenticationUsername\":\"bob\",\"httpAuthenticationPassword\":{\"type\":\"default\",\"password\":\"secret\"},\"requestHeaders\":{}},\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\",\"partitionedBy\":{\"type\":\"all\"}}"; @@ -390,7 +424,8 @@ public class IngestTableFunctionTest extends CalciteIngestionDmlTest "bob", new DefaultPasswordProvider("secret"), SystemFields.none(), - new HttpInputSourceConfig(null) + null, + new HttpInputSourceConfig(null, null) ), new JsonInputFormat(null, null, null, null, null), RowSignature.builder() @@ -544,21 +579,6 @@ public class IngestTableFunctionTest extends CalciteIngestionDmlTest .verify(); } - protected final ExternalDataSource localDataSource = new ExternalDataSource( - new LocalInputSource( - null, - null, - Arrays.asList(new File("/tmp/foo.csv"), new File("/tmp/bar.csv")), - SystemFields.none() - ), - new CsvInputFormat(ImmutableList.of("x", "y", "z"), null, false, false, 0), - RowSignature.builder() - .add("x", ColumnType.STRING) - .add("y", ColumnType.STRING) - .add("z", ColumnType.LONG) - .build() - ); - /** * Basic use of LOCALFILES */ @@ -697,4 +717,66 @@ public class IngestTableFunctionTest extends CalciteIngestionDmlTest .expectLogicalPlanFrom("localExtern") .verify(); } + + protected static class ExportComponentSupplier extends IngestionDmlComponentSupplier + { + public ExportComponentSupplier(TempDirProducer tempFolderProducer) + { + super(tempFolderProducer); + } + + @Override + public void configureGuice(DruidInjectorBuilder builder) + { + builder.addModule(new DruidModule() + { + + // Clone of MSQExternalDataSourceModule since it is not + // visible here. + @Override + public List getJacksonModules() + { + return Collections.singletonList( + new SimpleModule(getClass().getSimpleName()) + .registerSubtypes(ExternalDataSource.class) + ); + } + + @Override + public void configure(Binder binder) + { + // Adding the config to allow following 2 headers. + binder.bind(HttpInputSourceConfig.class) + .toInstance(new HttpInputSourceConfig(null, ImmutableSet.of("Accept", "a"))); + + } + }); + + builder.addModule(new DruidModule() + { + + @Override + public List getJacksonModules() + { + // We want this module to bring input sources along for the ride. + List modules = new ArrayList<>(new InputSourceModule().getJacksonModules()); + modules.add(new SimpleModule("test-module").registerSubtypes(TestFileInputSource.class)); + return modules; + } + + @Override + public void configure(Binder binder) + { + // Set up the EXTERN macro. + SqlBindings.addOperatorConversion(binder, ExternalOperatorConversion.class); + + // Enable the extended table functions for testing even though these + // are not enabled in production in Druid 26. + SqlBindings.addOperatorConversion(binder, HttpOperatorConversion.class); + SqlBindings.addOperatorConversion(binder, InlineOperatorConversion.class); + SqlBindings.addOperatorConversion(binder, LocalOperatorConversion.class); + } + }); + } + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java index e5442a2bda2..50dd2e43773 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java @@ -21,7 +21,6 @@ package org.apache.druid.sql.calcite; import com.google.common.base.Throwables; import org.apache.druid.error.DruidException; -import org.apache.druid.java.util.common.ISE; import org.junit.AssumptionViolatedException; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.api.extension.InvocationInterceptor; @@ -89,15 +88,13 @@ public @interface NotYetSupported RESULT_MISMATCH(AssertionError.class, "(assertResulEquals|AssertionError: column content mismatch)"), LONG_CASTING(AssertionError.class, "expected: java.lang.Long"), UNSUPPORTED_NULL_ORDERING(DruidException.class, "(A|DE)SCENDING ordering with NULLS (LAST|FIRST)"), - CANNOT_RETRIEVE_ROWS(UnsupportedOperationException.class, "Cannot retrieve number of rows from join segment"), + UNSUPPORTED_DATASOURCE(DruidException.class, "WindowOperatorQuery must run on top of a query or inline data source"), UNION_WITH_COMPLEX_OPERAND(DruidException.class, "Only Table and Values are supported as inputs for Union"), UNION_MORE_STRICT_ROWTYPE_CHECK(DruidException.class, "Row signature mismatch in Union inputs"), JOIN_CONDITION_NOT_PUSHED_CONDITION(DruidException.class, "SQL requires a join with '.*' condition"), JOIN_CONDITION_UNSUPORTED_OPERAND(DruidException.class, "SQL .* unsupported operand type"), - JOIN_TABLE_TABLE(ISE.class, "Cannot handle subquery structure for dataSource: JoinDataSource"), CORRELATE_CONVERSION(DruidException.class, "Missing conversion( is|s are) LogicalCorrelate"), SORT_REMOVE_TROUBLE(DruidException.class, "Calcite assertion violated.*Sort\\."), - STACK_OVERFLOW(StackOverflowError.class, ""), CANNOT_JOIN_LOOKUP_NON_KEY(RuntimeException.class, "Cannot join lookup with condition referring to non-key"), SORT_REMOVE_CONSTANT_KEYS_CONFLICT(DruidException.class, "not enough rules"); // @formatter:on diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java index 10504ca49d5..b613c602f63 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java @@ -37,6 +37,7 @@ import org.apache.druid.client.InternalQueryConfig; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.guava.Sequences; @@ -53,7 +54,7 @@ import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexStorageAdapter; +import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -220,10 +221,10 @@ public class BrokerSegmentMetadataCacheTest extends BrokerSegmentMetadataCacheTe @Test public void testCoordinatorReturnsAllDSSchema() throws InterruptedException { - final RowSignature dataSource1RowSignature = new QueryableIndexStorageAdapter(index1).getRowSignature(); - final RowSignature dataSource2RowSignature = new QueryableIndexStorageAdapter(index2).getRowSignature(); - final RowSignature someDataSourceRowSignature = new QueryableIndexStorageAdapter(indexAuto1).getRowSignature(); - final RowSignature foo3RowSignature = new QueryableIndexStorageAdapter(indexAuto2).getRowSignature(); + final RowSignature dataSource1RowSignature = new QueryableIndexCursorFactory(index1).getRowSignature(); + final RowSignature dataSource2RowSignature = new QueryableIndexCursorFactory(index2).getRowSignature(); + final RowSignature someDataSourceRowSignature = new QueryableIndexCursorFactory(indexAuto1).getRowSignature(); + final RowSignature foo3RowSignature = new QueryableIndexCursorFactory(indexAuto2).getRowSignature(); NoopCoordinatorClient coordinatorClient = new NoopCoordinatorClient() { @Override @@ -272,9 +273,9 @@ public class BrokerSegmentMetadataCacheTest extends BrokerSegmentMetadataCacheTe @Test public void testCoordinatorReturnsFewDSSchema() throws InterruptedException { - final RowSignature dataSource1RowSignature = new QueryableIndexStorageAdapter(index1).getRowSignature(); - final RowSignature dataSource2RowSignature = new QueryableIndexStorageAdapter(index2).getRowSignature(); - final RowSignature someDataSourceRowSignature = new QueryableIndexStorageAdapter(indexAuto1).getRowSignature(); + final RowSignature dataSource1RowSignature = new QueryableIndexCursorFactory(index1).getRowSignature(); + final RowSignature dataSource2RowSignature = new QueryableIndexCursorFactory(index2).getRowSignature(); + final RowSignature someDataSourceRowSignature = new QueryableIndexCursorFactory(indexAuto1).getRowSignature(); NoopCoordinatorClient coordinatorClient = new NoopCoordinatorClient() { @Override @@ -1139,71 +1140,109 @@ public class BrokerSegmentMetadataCacheTest extends BrokerSegmentMetadataCacheTe } @Test - public void testTombstoneSegmentIsNotAdded() throws InterruptedException + public void testTombstoneSegmentIsNotRefreshed() throws IOException { - String datasource = "newSegmentAddTest"; - CountDownLatch addSegmentLatch = new CountDownLatch(1); + String brokerInternalQueryConfigJson = "{\"context\": { \"priority\": 5} }"; + + TestHelper.makeJsonMapper(); + InternalQueryConfig internalQueryConfig = MAPPER.readValue( + MAPPER.writeValueAsString( + MAPPER.readValue(brokerInternalQueryConfigJson, InternalQueryConfig.class) + ), + InternalQueryConfig.class + ); + + QueryLifecycleFactory factoryMock = EasyMock.createMock(QueryLifecycleFactory.class); + QueryLifecycle lifecycleMock = EasyMock.createMock(QueryLifecycle.class); + BrokerSegmentMetadataCache schema = new BrokerSegmentMetadataCache( - CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), + factoryMock, serverView, - BrokerSegmentMetadataCacheConfig.create(), + SEGMENT_CACHE_CONFIG_DEFAULT, new NoopEscalator(), - new InternalQueryConfig(), + internalQueryConfig, new NoopServiceEmitter(), new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), new NoopCoordinatorClient(), CentralizedDatasourceSchemaConfig.create() - ) - { - @Override - public void addSegment(final DruidServerMetadata server, final DataSegment segment) - { - super.addSegment(server, segment); - if (datasource.equals(segment.getDataSource())) { - addSegmentLatch.countDown(); - } - } - }; - - schema.start(); - schema.awaitInitialization(); - - DataSegment segment = new DataSegment( - datasource, - Intervals.of("2001/2002"), - "1", - Collections.emptyMap(), - Collections.emptyList(), - Collections.emptyList(), - TombstoneShardSpec.INSTANCE, - null, - null, - 0 ); - Assert.assertEquals(6, schema.getTotalSegments()); + Map queryContext = ImmutableMap.of( + QueryContexts.PRIORITY_KEY, 5, + QueryContexts.BROKER_PARALLEL_MERGE_KEY, false + ); - serverView.addSegment(segment, ServerType.HISTORICAL); - Assert.assertTrue(addSegmentLatch.await(1, TimeUnit.SECONDS)); - Assert.assertEquals(0, addSegmentLatch.getCount()); + DataSegment segment = newSegment("test", 0); + DataSegment tombstone = DataSegment.builder() + .dataSource("test") + .interval(Intervals.of("2012-01-01/2012-01-02")) + .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString()) + .shardSpec(new TombstoneShardSpec()) + .loadSpec(Collections.singletonMap( + "type", + DataSegment.TOMBSTONE_LOADSPEC_TYPE + )) + .size(0) + .build(); - Assert.assertEquals(6, schema.getTotalSegments()); - List metadatas = schema - .getSegmentMetadataSnapshot() - .values() - .stream() - .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) - .collect(Collectors.toList()); - Assert.assertEquals(0, metadatas.size()); + final ImmutableDruidServer historicalServer = druidServers.stream() + .filter(s -> s.getType().equals(ServerType.HISTORICAL)) + .findAny() + .orElse(null); - serverView.removeSegment(segment, ServerType.HISTORICAL); - Assert.assertEquals(6, schema.getTotalSegments()); - metadatas = schema - .getSegmentMetadataSnapshot() - .values() - .stream() - .filter(metadata -> datasource.equals(metadata.getSegment().getDataSource())) - .collect(Collectors.toList()); - Assert.assertEquals(0, metadatas.size()); + Assert.assertNotNull(historicalServer); + final DruidServerMetadata historicalServerMetadata = historicalServer.getMetadata(); + + schema.addSegment(historicalServerMetadata, segment); + schema.addSegment(historicalServerMetadata, tombstone); + Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(tombstone.getId())); + + List segmentIterable = ImmutableList.of(segment.getId(), tombstone.getId()); + + SegmentMetadataQuery expectedMetadataQuery = new SegmentMetadataQuery( + new TableDataSource(segment.getDataSource()), + new MultipleSpecificSegmentSpec( + segmentIterable.stream() + .filter(id -> !id.equals(tombstone.getId())) + .map(SegmentId::toDescriptor) + .collect(Collectors.toList()) + ), + new AllColumnIncluderator(), + false, + queryContext, + EnumSet.noneOf(SegmentMetadataQuery.AnalysisType.class), + false, + null, + null + ); + + EasyMock.expect(factoryMock.factorize()).andReturn(lifecycleMock).once(); + EasyMock.expect(lifecycleMock.runSimple(expectedMetadataQuery, AllowAllAuthenticator.ALLOW_ALL_RESULT, Access.OK)) + .andReturn(QueryResponse.withEmptyContext(Sequences.empty())); + + EasyMock.replay(factoryMock, lifecycleMock); + + Set segmentsToRefresh = new HashSet<>(); + segmentsToRefresh.add(segment.getId()); + schema.refresh(segmentsToRefresh, Collections.singleton("test")); + + // verify that metadata is not issued for tombstone segment + EasyMock.verify(factoryMock, lifecycleMock); + + // Verify that datasource schema building logic doesn't mark the tombstone segment for refresh + Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(tombstone.getId())); + + AvailableSegmentMetadata availableSegmentMetadata = schema.getAvailableSegmentMetadata("test", tombstone.getId()); + Assert.assertNotNull(availableSegmentMetadata); + // fetching metadata for tombstone segment shouldn't mark it for refresh + Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(tombstone.getId())); + + Set metadatas = new HashSet<>(); + schema.iterateSegmentMetadata().forEachRemaining(metadatas::add); + + Assert.assertEquals(1, metadatas.stream().filter(metadata -> metadata.getSegment().isTombstone()).count()); + + // iterating over entire metadata doesn't cause tombstone to be marked for refresh + Assert.assertFalse(schema.getSegmentsNeedingRefresh().contains(tombstone.getId())); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java index 0f355b2f1b9..d59078deff9 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java @@ -197,6 +197,13 @@ public class SqlTestFramework { configureGuice(injectorBuilder); } + + /** + * Communicates if explain are supported. + * + * MSQ right now needs a full query run. + */ + Boolean isExplainSupported(); } public interface PlannerComponentSupplier @@ -335,6 +342,12 @@ public class SqlTestFramework { tempDirProducer.close(); } + + @Override + public Boolean isExplainSupported() + { + return true; + } } public static class StandardPlannerComponentSupplier implements PlannerComponentSupplier @@ -668,6 +681,13 @@ public class SqlTestFramework { return getTestConfig().getDruidTestURI(); } + + @Provides + @Named("isExplainSupported") + public Boolean isExplainSupported() + { + return builder.componentSupplier.isExplainSupported(); + } } public static final DruidViewMacroFactory DRUID_VIEW_MACRO_FACTORY = new TestDruidViewMacroFactory(); diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq deleted file mode 100644 index da6ada101d4..00000000000 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq +++ /dev/null @@ -1,113 +0,0 @@ -# testInnerJoinQueryOfLookup@all_disabled case-crc:d41a4a0d -# quidem testcase reason: FINALIZING_FIELD_ACCESS -!set debug true -!set defaultTimeout 300000 -!set enableJoinFilterRewrite false -!set enableJoinFilterRewriteValueColumnFilters false -!set enableRewriteJoinToFilter false -!set maxScatterGatherBytes 9223372036854775807 -!set plannerStrategy DECOUPLED -!set sqlCurrentTimestamp 2000-01-01T00:00:00Z -!set sqlQueryId dummy -!set outputformat mysql -!use druidtest:/// -SELECT dim1, dim2, t1.v, t1.v -FROM foo -INNER JOIN - (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 - ON foo.dim2 = t1.k; -+------+------+------+------+ -| dim1 | dim2 | v | v | -+------+------+------+------+ -| | a | xabc | xabc | -| 1 | a | xabc | xabc | -+------+------+------+------+ -(2 rows) - -!ok -LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) - LogicalJoin(condition=[=($1, $2)], joinType=[inner]) - LogicalProject(dim1=[$1], dim2=[$2]) - LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) - LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) - LogicalTableScan(table=[[lookup, lookyloo]]) - -!logicalPlan -DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) - DruidJoin(condition=[=($1, $2)], joinType=[inner]) - DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) - DruidTableScan(table=[[druid, foo]], druid=[logical]) - DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) - DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) - DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) - -!druidPlan -{ - "queryType" : "scan", - "dataSource" : { - "type" : "join", - "left" : { - "type" : "table", - "name" : "foo" - }, - "right" : { - "type" : "query", - "query" : { - "queryType" : "groupBy", - "dataSource" : { - "type" : "lookup", - "lookup" : "lookyloo" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "granularity" : { - "type" : "all" - }, - "dimensions" : [ { - "type" : "extraction", - "dimension" : "k", - "outputName" : "d0", - "outputType" : "STRING", - "extractionFn" : { - "type" : "substring", - "index" : 0, - "length" : 1 - } - } ], - "aggregations" : [ { - "type" : "stringAny", - "name" : "a0:a", - "fieldName" : "v", - "maxStringBytes" : 10, - "aggregateMultipleValues" : true - } ], - "postAggregations" : [ { - "type" : "finalizingFieldAccess", - "name" : "a0", - "fieldName" : "a0:a" - } ], - "limitSpec" : { - "type" : "NoopLimitSpec" - } - } - }, - "rightPrefix" : "j0.", - "condition" : "(\"dim2\" == \"j0.d0\")", - "joinType" : "INNER" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "resultFormat" : "compactedList", - "columns" : [ "dim1", "dim2", "j0.a0" ], - "columnTypes" : [ "STRING", "STRING", "STRING" ], - "granularity" : { - "type" : "all" - }, - "legacy" : false -} -!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq deleted file mode 100644 index 30482302b2e..00000000000 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq +++ /dev/null @@ -1,113 +0,0 @@ -# testInnerJoinQueryOfLookup@all_enabled case-crc:93392df4 -# quidem testcase reason: FINALIZING_FIELD_ACCESS -!set debug true -!set defaultTimeout 300000 -!set enableJoinFilterRewrite true -!set enableJoinFilterRewriteValueColumnFilters true -!set enableRewriteJoinToFilter true -!set maxScatterGatherBytes 9223372036854775807 -!set plannerStrategy DECOUPLED -!set sqlCurrentTimestamp 2000-01-01T00:00:00Z -!set sqlQueryId dummy -!set outputformat mysql -!use druidtest:/// -SELECT dim1, dim2, t1.v, t1.v -FROM foo -INNER JOIN - (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 - ON foo.dim2 = t1.k; -+------+------+------+------+ -| dim1 | dim2 | v | v | -+------+------+------+------+ -| | a | xabc | xabc | -| 1 | a | xabc | xabc | -+------+------+------+------+ -(2 rows) - -!ok -LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) - LogicalJoin(condition=[=($1, $2)], joinType=[inner]) - LogicalProject(dim1=[$1], dim2=[$2]) - LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) - LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) - LogicalTableScan(table=[[lookup, lookyloo]]) - -!logicalPlan -DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) - DruidJoin(condition=[=($1, $2)], joinType=[inner]) - DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) - DruidTableScan(table=[[druid, foo]], druid=[logical]) - DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) - DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) - DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) - -!druidPlan -{ - "queryType" : "scan", - "dataSource" : { - "type" : "join", - "left" : { - "type" : "table", - "name" : "foo" - }, - "right" : { - "type" : "query", - "query" : { - "queryType" : "groupBy", - "dataSource" : { - "type" : "lookup", - "lookup" : "lookyloo" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "granularity" : { - "type" : "all" - }, - "dimensions" : [ { - "type" : "extraction", - "dimension" : "k", - "outputName" : "d0", - "outputType" : "STRING", - "extractionFn" : { - "type" : "substring", - "index" : 0, - "length" : 1 - } - } ], - "aggregations" : [ { - "type" : "stringAny", - "name" : "a0:a", - "fieldName" : "v", - "maxStringBytes" : 10, - "aggregateMultipleValues" : true - } ], - "postAggregations" : [ { - "type" : "finalizingFieldAccess", - "name" : "a0", - "fieldName" : "a0:a" - } ], - "limitSpec" : { - "type" : "NoopLimitSpec" - } - } - }, - "rightPrefix" : "j0.", - "condition" : "(\"dim2\" == \"j0.d0\")", - "joinType" : "INNER" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "resultFormat" : "compactedList", - "columns" : [ "dim1", "dim2", "j0.a0" ], - "columnTypes" : [ "STRING", "STRING", "STRING" ], - "granularity" : { - "type" : "all" - }, - "legacy" : false -} -!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq deleted file mode 100644 index ec13b17cbc0..00000000000 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq +++ /dev/null @@ -1,110 +0,0 @@ -# testInnerJoinQueryOfLookup@default case-crc:ee151062 -# quidem testcase reason: FINALIZING_FIELD_ACCESS -!set debug true -!set defaultTimeout 300000 -!set maxScatterGatherBytes 9223372036854775807 -!set plannerStrategy DECOUPLED -!set sqlCurrentTimestamp 2000-01-01T00:00:00Z -!set sqlQueryId dummy -!set outputformat mysql -!use druidtest:/// -SELECT dim1, dim2, t1.v, t1.v -FROM foo -INNER JOIN - (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 - ON foo.dim2 = t1.k; -+------+------+------+------+ -| dim1 | dim2 | v | v | -+------+------+------+------+ -| | a | xabc | xabc | -| 1 | a | xabc | xabc | -+------+------+------+------+ -(2 rows) - -!ok -LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) - LogicalJoin(condition=[=($1, $2)], joinType=[inner]) - LogicalProject(dim1=[$1], dim2=[$2]) - LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) - LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) - LogicalTableScan(table=[[lookup, lookyloo]]) - -!logicalPlan -DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) - DruidJoin(condition=[=($1, $2)], joinType=[inner]) - DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) - DruidTableScan(table=[[druid, foo]], druid=[logical]) - DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) - DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) - DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) - -!druidPlan -{ - "queryType" : "scan", - "dataSource" : { - "type" : "join", - "left" : { - "type" : "table", - "name" : "foo" - }, - "right" : { - "type" : "query", - "query" : { - "queryType" : "groupBy", - "dataSource" : { - "type" : "lookup", - "lookup" : "lookyloo" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "granularity" : { - "type" : "all" - }, - "dimensions" : [ { - "type" : "extraction", - "dimension" : "k", - "outputName" : "d0", - "outputType" : "STRING", - "extractionFn" : { - "type" : "substring", - "index" : 0, - "length" : 1 - } - } ], - "aggregations" : [ { - "type" : "stringAny", - "name" : "a0:a", - "fieldName" : "v", - "maxStringBytes" : 10, - "aggregateMultipleValues" : true - } ], - "postAggregations" : [ { - "type" : "finalizingFieldAccess", - "name" : "a0", - "fieldName" : "a0:a" - } ], - "limitSpec" : { - "type" : "NoopLimitSpec" - } - } - }, - "rightPrefix" : "j0.", - "condition" : "(\"dim2\" == \"j0.d0\")", - "joinType" : "INNER" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "resultFormat" : "compactedList", - "columns" : [ "dim1", "dim2", "j0.a0" ], - "columnTypes" : [ "STRING", "STRING", "STRING" ], - "granularity" : { - "type" : "all" - }, - "legacy" : false -} -!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq deleted file mode 100644 index ad63ef44426..00000000000 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq +++ /dev/null @@ -1,113 +0,0 @@ -# testInnerJoinQueryOfLookup@filter-on-value-column_disabled case-crc:dbd4147e -# quidem testcase reason: FINALIZING_FIELD_ACCESS -!set debug true -!set defaultTimeout 300000 -!set enableJoinFilterRewrite true -!set enableJoinFilterRewriteValueColumnFilters false -!set enableRewriteJoinToFilter true -!set maxScatterGatherBytes 9223372036854775807 -!set plannerStrategy DECOUPLED -!set sqlCurrentTimestamp 2000-01-01T00:00:00Z -!set sqlQueryId dummy -!set outputformat mysql -!use druidtest:/// -SELECT dim1, dim2, t1.v, t1.v -FROM foo -INNER JOIN - (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 - ON foo.dim2 = t1.k; -+------+------+------+------+ -| dim1 | dim2 | v | v | -+------+------+------+------+ -| | a | xabc | xabc | -| 1 | a | xabc | xabc | -+------+------+------+------+ -(2 rows) - -!ok -LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) - LogicalJoin(condition=[=($1, $2)], joinType=[inner]) - LogicalProject(dim1=[$1], dim2=[$2]) - LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) - LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) - LogicalTableScan(table=[[lookup, lookyloo]]) - -!logicalPlan -DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) - DruidJoin(condition=[=($1, $2)], joinType=[inner]) - DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) - DruidTableScan(table=[[druid, foo]], druid=[logical]) - DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) - DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) - DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) - -!druidPlan -{ - "queryType" : "scan", - "dataSource" : { - "type" : "join", - "left" : { - "type" : "table", - "name" : "foo" - }, - "right" : { - "type" : "query", - "query" : { - "queryType" : "groupBy", - "dataSource" : { - "type" : "lookup", - "lookup" : "lookyloo" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "granularity" : { - "type" : "all" - }, - "dimensions" : [ { - "type" : "extraction", - "dimension" : "k", - "outputName" : "d0", - "outputType" : "STRING", - "extractionFn" : { - "type" : "substring", - "index" : 0, - "length" : 1 - } - } ], - "aggregations" : [ { - "type" : "stringAny", - "name" : "a0:a", - "fieldName" : "v", - "maxStringBytes" : 10, - "aggregateMultipleValues" : true - } ], - "postAggregations" : [ { - "type" : "finalizingFieldAccess", - "name" : "a0", - "fieldName" : "a0:a" - } ], - "limitSpec" : { - "type" : "NoopLimitSpec" - } - } - }, - "rightPrefix" : "j0.", - "condition" : "(\"dim2\" == \"j0.d0\")", - "joinType" : "INNER" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "resultFormat" : "compactedList", - "columns" : [ "dim1", "dim2", "j0.a0" ], - "columnTypes" : [ "STRING", "STRING", "STRING" ], - "granularity" : { - "type" : "all" - }, - "legacy" : false -} -!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq deleted file mode 100644 index 48890391a6b..00000000000 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq +++ /dev/null @@ -1,113 +0,0 @@ -# testInnerJoinQueryOfLookup@filter-rewrites-disabled case-crc:57dd8dfa -# quidem testcase reason: FINALIZING_FIELD_ACCESS -!set debug true -!set defaultTimeout 300000 -!set enableJoinFilterRewrite false -!set enableJoinFilterRewriteValueColumnFilters true -!set enableRewriteJoinToFilter true -!set maxScatterGatherBytes 9223372036854775807 -!set plannerStrategy DECOUPLED -!set sqlCurrentTimestamp 2000-01-01T00:00:00Z -!set sqlQueryId dummy -!set outputformat mysql -!use druidtest:/// -SELECT dim1, dim2, t1.v, t1.v -FROM foo -INNER JOIN - (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 - ON foo.dim2 = t1.k; -+------+------+------+------+ -| dim1 | dim2 | v | v | -+------+------+------+------+ -| | a | xabc | xabc | -| 1 | a | xabc | xabc | -+------+------+------+------+ -(2 rows) - -!ok -LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) - LogicalJoin(condition=[=($1, $2)], joinType=[inner]) - LogicalProject(dim1=[$1], dim2=[$2]) - LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) - LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) - LogicalTableScan(table=[[lookup, lookyloo]]) - -!logicalPlan -DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) - DruidJoin(condition=[=($1, $2)], joinType=[inner]) - DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) - DruidTableScan(table=[[druid, foo]], druid=[logical]) - DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) - DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) - DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) - -!druidPlan -{ - "queryType" : "scan", - "dataSource" : { - "type" : "join", - "left" : { - "type" : "table", - "name" : "foo" - }, - "right" : { - "type" : "query", - "query" : { - "queryType" : "groupBy", - "dataSource" : { - "type" : "lookup", - "lookup" : "lookyloo" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "granularity" : { - "type" : "all" - }, - "dimensions" : [ { - "type" : "extraction", - "dimension" : "k", - "outputName" : "d0", - "outputType" : "STRING", - "extractionFn" : { - "type" : "substring", - "index" : 0, - "length" : 1 - } - } ], - "aggregations" : [ { - "type" : "stringAny", - "name" : "a0:a", - "fieldName" : "v", - "maxStringBytes" : 10, - "aggregateMultipleValues" : true - } ], - "postAggregations" : [ { - "type" : "finalizingFieldAccess", - "name" : "a0", - "fieldName" : "a0:a" - } ], - "limitSpec" : { - "type" : "NoopLimitSpec" - } - } - }, - "rightPrefix" : "j0.", - "condition" : "(\"dim2\" == \"j0.d0\")", - "joinType" : "INNER" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "resultFormat" : "compactedList", - "columns" : [ "dim1", "dim2", "j0.a0" ], - "columnTypes" : [ "STRING", "STRING", "STRING" ], - "granularity" : { - "type" : "all" - }, - "legacy" : false -} -!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq deleted file mode 100644 index 1fe9c554541..00000000000 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq +++ /dev/null @@ -1,113 +0,0 @@ -# testInnerJoinQueryOfLookup@filter-rewrites case-crc:10d0367d -# quidem testcase reason: FINALIZING_FIELD_ACCESS -!set debug true -!set defaultTimeout 300000 -!set enableJoinFilterRewrite true -!set enableJoinFilterRewriteValueColumnFilters true -!set enableRewriteJoinToFilter false -!set maxScatterGatherBytes 9223372036854775807 -!set plannerStrategy DECOUPLED -!set sqlCurrentTimestamp 2000-01-01T00:00:00Z -!set sqlQueryId dummy -!set outputformat mysql -!use druidtest:/// -SELECT dim1, dim2, t1.v, t1.v -FROM foo -INNER JOIN - (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 - ON foo.dim2 = t1.k; -+------+------+------+------+ -| dim1 | dim2 | v | v | -+------+------+------+------+ -| | a | xabc | xabc | -| 1 | a | xabc | xabc | -+------+------+------+------+ -(2 rows) - -!ok -LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) - LogicalJoin(condition=[=($1, $2)], joinType=[inner]) - LogicalProject(dim1=[$1], dim2=[$2]) - LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) - LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) - LogicalTableScan(table=[[lookup, lookyloo]]) - -!logicalPlan -DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) - DruidJoin(condition=[=($1, $2)], joinType=[inner]) - DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) - DruidTableScan(table=[[druid, foo]], druid=[logical]) - DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) - DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) - DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) - -!druidPlan -{ - "queryType" : "scan", - "dataSource" : { - "type" : "join", - "left" : { - "type" : "table", - "name" : "foo" - }, - "right" : { - "type" : "query", - "query" : { - "queryType" : "groupBy", - "dataSource" : { - "type" : "lookup", - "lookup" : "lookyloo" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "granularity" : { - "type" : "all" - }, - "dimensions" : [ { - "type" : "extraction", - "dimension" : "k", - "outputName" : "d0", - "outputType" : "STRING", - "extractionFn" : { - "type" : "substring", - "index" : 0, - "length" : 1 - } - } ], - "aggregations" : [ { - "type" : "stringAny", - "name" : "a0:a", - "fieldName" : "v", - "maxStringBytes" : 10, - "aggregateMultipleValues" : true - } ], - "postAggregations" : [ { - "type" : "finalizingFieldAccess", - "name" : "a0", - "fieldName" : "a0:a" - } ], - "limitSpec" : { - "type" : "NoopLimitSpec" - } - } - }, - "rightPrefix" : "j0.", - "condition" : "(\"dim2\" == \"j0.d0\")", - "joinType" : "INNER" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "resultFormat" : "compactedList", - "columns" : [ "dim1", "dim2", "j0.a0" ], - "columnTypes" : [ "STRING", "STRING", "STRING" ], - "granularity" : { - "type" : "all" - }, - "legacy" : false -} -!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq deleted file mode 100644 index 0ad9125b151..00000000000 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq +++ /dev/null @@ -1,113 +0,0 @@ -# testInnerJoinQueryOfLookup@join-to-filter case-crc:967213e2 -# quidem testcase reason: FINALIZING_FIELD_ACCESS -!set debug true -!set defaultTimeout 300000 -!set enableJoinFilterRewrite false -!set enableJoinFilterRewriteValueColumnFilters false -!set enableRewriteJoinToFilter true -!set maxScatterGatherBytes 9223372036854775807 -!set plannerStrategy DECOUPLED -!set sqlCurrentTimestamp 2000-01-01T00:00:00Z -!set sqlQueryId dummy -!set outputformat mysql -!use druidtest:/// -SELECT dim1, dim2, t1.v, t1.v -FROM foo -INNER JOIN - (SELECT SUBSTRING(k, 1, 1) k, ANY_VALUE(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1 - ON foo.dim2 = t1.k; -+------+------+------+------+ -| dim1 | dim2 | v | v | -+------+------+------+------+ -| | a | xabc | xabc | -| 1 | a | xabc | xabc | -+------+------+------+------+ -(2 rows) - -!ok -LogicalProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3]) - LogicalJoin(condition=[=($1, $2)], joinType=[inner]) - LogicalProject(dim1=[$1], dim2=[$2]) - LogicalTableScan(table=[[druid, foo]]) - LogicalAggregate(group=[{0}], v=[ANY_VALUE($1, $2)]) - LogicalProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10]) - LogicalTableScan(table=[[lookup, lookyloo]]) - -!logicalPlan -DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) - DruidJoin(condition=[=($1, $2)], joinType=[inner]) - DruidProject(dim1=[$1], dim2=[$2], druid=[logical]) - DruidTableScan(table=[[druid, foo]], druid=[logical]) - DruidAggregate(group=[{0}], v=[ANY_VALUE($1, $2)], druid=[logical]) - DruidProject(k=[SUBSTRING($0, 1, 1)], v=[$1], $f2=[10], druid=[logical]) - DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) - -!druidPlan -{ - "queryType" : "scan", - "dataSource" : { - "type" : "join", - "left" : { - "type" : "table", - "name" : "foo" - }, - "right" : { - "type" : "query", - "query" : { - "queryType" : "groupBy", - "dataSource" : { - "type" : "lookup", - "lookup" : "lookyloo" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "granularity" : { - "type" : "all" - }, - "dimensions" : [ { - "type" : "extraction", - "dimension" : "k", - "outputName" : "d0", - "outputType" : "STRING", - "extractionFn" : { - "type" : "substring", - "index" : 0, - "length" : 1 - } - } ], - "aggregations" : [ { - "type" : "stringAny", - "name" : "a0:a", - "fieldName" : "v", - "maxStringBytes" : 10, - "aggregateMultipleValues" : true - } ], - "postAggregations" : [ { - "type" : "finalizingFieldAccess", - "name" : "a0", - "fieldName" : "a0:a" - } ], - "limitSpec" : { - "type" : "NoopLimitSpec" - } - } - }, - "rightPrefix" : "j0.", - "condition" : "(\"dim2\" == \"j0.d0\")", - "joinType" : "INNER" - }, - "intervals" : { - "type" : "intervals", - "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] - }, - "resultFormat" : "compactedList", - "columns" : [ "dim1", "dim2", "j0.a0" ], - "columnTypes" : [ "STRING", "STRING", "STRING" ], - "granularity" : { - "type" : "all" - }, - "legacy" : false -} -!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@all_disabled.iq new file mode 100644 index 00000000000..8776842e1b3 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@all_disabled.iq @@ -0,0 +1,128 @@ +# testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@all_disabled case-crc:544a51fb +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM lookup.lookyloo l1 +INNER JOIN lookup.lookyloo l2 ON l1.k = l2.k +INNER JOIN foo on l2.k = foo.m1; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k00=[CAST($1):FLOAT]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k00=[CAST($1):FLOAT], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'DOUBLE')", + "outputType" : "FLOAT" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@all_enabled.iq new file mode 100644 index 00000000000..1d3b37da420 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@all_enabled.iq @@ -0,0 +1,128 @@ +# testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@all_enabled case-crc:21d41c09 +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM lookup.lookyloo l1 +INNER JOIN lookup.lookyloo l2 ON l1.k = l2.k +INNER JOIN foo on l2.k = foo.m1; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k00=[CAST($1):FLOAT]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k00=[CAST($1):FLOAT], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'DOUBLE')", + "outputType" : "FLOAT" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@default.iq new file mode 100644 index 00000000000..5d5d5dc56d9 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@default.iq @@ -0,0 +1,125 @@ +# testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@default case-crc:fe49c163 +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM lookup.lookyloo l1 +INNER JOIN lookup.lookyloo l2 ON l1.k = l2.k +INNER JOIN foo on l2.k = foo.m1; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k00=[CAST($1):FLOAT]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k00=[CAST($1):FLOAT], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'DOUBLE')", + "outputType" : "FLOAT" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@filter-on-value-column_disabled.iq new file mode 100644 index 00000000000..55340f7c4f7 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@filter-on-value-column_disabled.iq @@ -0,0 +1,128 @@ +# testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@filter-on-value-column_disabled case-crc:8bfc8f64 +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM lookup.lookyloo l1 +INNER JOIN lookup.lookyloo l2 ON l1.k = l2.k +INNER JOIN foo on l2.k = foo.m1; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k00=[CAST($1):FLOAT]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k00=[CAST($1):FLOAT], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'DOUBLE')", + "outputType" : "FLOAT" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@filter-rewrites-disabled.iq new file mode 100644 index 00000000000..501ef6650e5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@filter-rewrites-disabled.iq @@ -0,0 +1,128 @@ +# testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@filter-rewrites-disabled case-crc:58dcfbc6 +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM lookup.lookyloo l1 +INNER JOIN lookup.lookyloo l2 ON l1.k = l2.k +INNER JOIN foo on l2.k = foo.m1; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k00=[CAST($1):FLOAT]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k00=[CAST($1):FLOAT], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'DOUBLE')", + "outputType" : "FLOAT" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@filter-rewrites.iq new file mode 100644 index 00000000000..0779be6e531 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@filter-rewrites.iq @@ -0,0 +1,128 @@ +# testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@filter-rewrites case-crc:e016193d +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM lookup.lookyloo l1 +INNER JOIN lookup.lookyloo l2 ON l1.k = l2.k +INNER JOIN foo on l2.k = foo.m1; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k00=[CAST($1):FLOAT]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k00=[CAST($1):FLOAT], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'DOUBLE')", + "outputType" : "FLOAT" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@join-to-filter.iq new file mode 100644 index 00000000000..0b277693ae0 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@join-to-filter.iq @@ -0,0 +1,128 @@ +# testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse@join-to-filter case-crc:1bd2994d +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM lookup.lookyloo l1 +INNER JOIN lookup.lookyloo l2 ON l1.k = l2.k +INNER JOIN foo on l2.k = foo.m1; ++--------+ +| EXPR$0 | ++--------+ +| 1 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k00=[CAST($1):FLOAT]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(k=[$0]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k00=[CAST($1):FLOAT], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(k=[$0], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "right" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "rightPrefix" : "j0.", + "condition" : "(\"k\" == \"j0.k\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'DOUBLE')", + "outputType" : "FLOAT" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOuterGroupByAndSubqueryHasLimit@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOuterGroupByAndSubqueryHasLimit@NullHandling=default.iq new file mode 100644 index 00000000000..b96238832d5 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOuterGroupByAndSubqueryHasLimit@NullHandling=default.iq @@ -0,0 +1,128 @@ +# testJoinOuterGroupByAndSubqueryHasLimit@NullHandling=default case-crc:2e733a5b +# quidem testcase reason: EQUIV_PLAN_EXTRA_COLUMNS +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim2, AVG(m2) FROM (SELECT * FROM foo AS t1 INNER JOIN foo AS t2 ON t1.m1 = t2.m1 LIMIT 10) AS t3 GROUP BY dim2; ++------+--------------------+ +| dim2 | EXPR$1 | ++------+--------------------+ +| | 3.6666666666666665 | +| a | 2.5 | +| abc | 5.0 | ++------+--------------------+ +(3 rows) + +!ok +LogicalAggregate(group=[{0}], EXPR$1=[AVG($2)]) + LogicalSort(fetch=[10]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim2=[$2], m1=[$5], m2=[$6]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], EXPR$1=[AVG($2)], druid=[logical]) + DruidSort(fetch=[10], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim2=[$2], m1=[$5], m2=[$6], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "limit" : 10, + "columns" : [ "dim2", "j0.m1", "m1", "m2" ], + "columnTypes" : [ "STRING", "FLOAT", "FLOAT", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "doubleSum", + "name" : "a0:sum", + "fieldName" : "m2" + }, { + "type" : "count", + "name" : "a0:count" + } ], + "postAggregations" : [ { + "type" : "arithmetic", + "name" : "a0", + "fn" : "quotient", + "fields" : [ { + "type" : "fieldAccess", + "fieldName" : "a0:sum" + }, { + "type" : "fieldAccess", + "fieldName" : "a0:count" + } ] + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOuterGroupByAndSubqueryHasLimit@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOuterGroupByAndSubqueryHasLimit@NullHandling=sql.iq new file mode 100644 index 00000000000..ec8a580d34c --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOuterGroupByAndSubqueryHasLimit@NullHandling=sql.iq @@ -0,0 +1,140 @@ +# testJoinOuterGroupByAndSubqueryHasLimit@NullHandling=sql case-crc:2e733a5b +# quidem testcase reason: EQUIV_PLAN_EXTRA_COLUMNS +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT dim2, AVG(m2) FROM (SELECT * FROM foo AS t1 INNER JOIN foo AS t2 ON t1.m1 = t2.m1 LIMIT 10) AS t3 GROUP BY dim2; ++------+--------+ +| dim2 | EXPR$1 | ++------+--------+ +| | 3.0 | +| a | 2.5 | +| abc | 5.0 | +| | 4.0 | ++------+--------+ +(4 rows) + +!ok +LogicalAggregate(group=[{0}], EXPR$1=[AVG($2)]) + LogicalSort(fetch=[10]) + LogicalJoin(condition=[=($1, $3)], joinType=[inner]) + LogicalProject(dim2=[$2], m1=[$5], m2=[$6]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(m1=[$5]) + LogicalTableScan(table=[[druid, foo]]) + +!logicalPlan +DruidAggregate(group=[{0}], EXPR$1=[AVG($2)], druid=[logical]) + DruidSort(fetch=[10], druid=[logical]) + DruidJoin(condition=[=($1, $3)], joinType=[inner]) + DruidProject(dim2=[$2], m1=[$5], m2=[$6], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(m1=[$5], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "groupBy", + "dataSource" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "foo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "m1" ], + "columnTypes" : [ "FLOAT" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"m1\" == \"j0.m1\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "limit" : 10, + "columns" : [ "dim2", "j0.m1", "m1", "m2" ], + "columnTypes" : [ "STRING", "FLOAT", "FLOAT", "DOUBLE" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "dimensions" : [ { + "type" : "default", + "dimension" : "dim2", + "outputName" : "d0", + "outputType" : "STRING" + } ], + "aggregations" : [ { + "type" : "doubleSum", + "name" : "a0:sum", + "fieldName" : "m2" + }, { + "type" : "filtered", + "aggregator" : { + "type" : "count", + "name" : "a0:count" + }, + "filter" : { + "type" : "not", + "field" : { + "type" : "null", + "column" : "m2" + } + }, + "name" : "a0:count" + } ], + "postAggregations" : [ { + "type" : "arithmetic", + "name" : "a0", + "fn" : "quotient", + "fields" : [ { + "type" : "fieldAccess", + "fieldName" : "a0:sum" + }, { + "type" : "fieldAccess", + "fieldName" : "a0:count" + } ] + } ], + "limitSpec" : { + "type" : "NoopLimitSpec" + } +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@all_disabled.iq new file mode 100644 index 00000000000..66130f6329e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@all_disabled.iq @@ -0,0 +1,152 @@ +# testJoinTableLookupTableMismatchedTypesWithoutComma@all_disabled case-crc:63a29f32 +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo +INNER JOIN lookup.lookyloo l ON foo.cnt = l.k +INNER JOIN numfoo ON l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], k0=[CAST($0):BIGINT]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], k0=[CAST($0):BIGINT], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "columnTypes" : [ "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"cnt\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@all_enabled.iq new file mode 100644 index 00000000000..d376316dd74 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@all_enabled.iq @@ -0,0 +1,152 @@ +# testJoinTableLookupTableMismatchedTypesWithoutComma@all_enabled case-crc:906d660f +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo +INNER JOIN lookup.lookyloo l ON foo.cnt = l.k +INNER JOIN numfoo ON l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], k0=[CAST($0):BIGINT]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], k0=[CAST($0):BIGINT], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "columnTypes" : [ "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"cnt\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@default.iq new file mode 100644 index 00000000000..eb68a671056 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@default.iq @@ -0,0 +1,149 @@ +# testJoinTableLookupTableMismatchedTypesWithoutComma@default case-crc:7765d2be +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo +INNER JOIN lookup.lookyloo l ON foo.cnt = l.k +INNER JOIN numfoo ON l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], k0=[CAST($0):BIGINT]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], k0=[CAST($0):BIGINT], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "columnTypes" : [ "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"cnt\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@filter-on-value-column_disabled.iq new file mode 100644 index 00000000000..6b52402d4e2 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@filter-on-value-column_disabled.iq @@ -0,0 +1,152 @@ +# testJoinTableLookupTableMismatchedTypesWithoutComma@filter-on-value-column_disabled case-crc:af07ed7a +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo +INNER JOIN lookup.lookyloo l ON foo.cnt = l.k +INNER JOIN numfoo ON l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], k0=[CAST($0):BIGINT]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], k0=[CAST($0):BIGINT], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "columnTypes" : [ "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"cnt\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@filter-rewrites-disabled.iq new file mode 100644 index 00000000000..0c6e376533e --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@filter-rewrites-disabled.iq @@ -0,0 +1,152 @@ +# testJoinTableLookupTableMismatchedTypesWithoutComma@filter-rewrites-disabled case-crc:0637e32a +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo +INNER JOIN lookup.lookyloo l ON foo.cnt = l.k +INNER JOIN numfoo ON l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], k0=[CAST($0):BIGINT]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], k0=[CAST($0):BIGINT], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "columnTypes" : [ "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"cnt\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@filter-rewrites.iq new file mode 100644 index 00000000000..0d767ed5e7d --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@filter-rewrites.iq @@ -0,0 +1,152 @@ +# testJoinTableLookupTableMismatchedTypesWithoutComma@filter-rewrites case-crc:c7b42928 +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite true +!set enableJoinFilterRewriteValueColumnFilters true +!set enableRewriteJoinToFilter false +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo +INNER JOIN lookup.lookyloo l ON foo.cnt = l.k +INNER JOIN numfoo ON l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], k0=[CAST($0):BIGINT]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], k0=[CAST($0):BIGINT], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "columnTypes" : [ "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"cnt\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@join-to-filter.iq new file mode 100644 index 00000000000..074179dcb53 --- /dev/null +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinTableLookupTableMismatchedTypesWithoutComma@join-to-filter.iq @@ -0,0 +1,152 @@ +# testJoinTableLookupTableMismatchedTypesWithoutComma@join-to-filter case-crc:84b4a463 +# quidem testcase reason: EQUIV_PLAN_CAST_MATERIALIZED_EARLIER +!set debug true +!set defaultTimeout 300000 +!set enableJoinFilterRewrite false +!set enableJoinFilterRewriteValueColumnFilters false +!set enableRewriteJoinToFilter true +!set maxScatterGatherBytes 9223372036854775807 +!set plannerStrategy DECOUPLED +!set sqlCurrentTimestamp 2000-01-01T00:00:00Z +!set sqlQueryId dummy +!set outputformat mysql +!use druidtest:/// +SELECT COUNT(*) +FROM foo +INNER JOIN lookup.lookyloo l ON foo.cnt = l.k +INNER JOIN numfoo ON l.k = numfoo.cnt +; ++--------+ +| EXPR$0 | ++--------+ +| 0 | ++--------+ +(1 row) + +!ok +LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) + LogicalJoin(condition=[=($0, $1)], joinType=[inner]) + LogicalProject(k0=[CAST($1):BIGINT]) + LogicalJoin(condition=[=($0, $2)], joinType=[inner]) + LogicalProject(cnt=[$4]) + LogicalTableScan(table=[[druid, foo]]) + LogicalProject(k=[$0], k0=[CAST($0):BIGINT]) + LogicalTableScan(table=[[lookup, lookyloo]]) + LogicalProject(cnt=[$13]) + LogicalTableScan(table=[[druid, numfoo]]) + +!logicalPlan +DruidAggregate(group=[{}], EXPR$0=[COUNT()], druid=[logical]) + DruidJoin(condition=[=($0, $1)], joinType=[inner]) + DruidProject(k0=[CAST($1):BIGINT], druid=[logical]) + DruidJoin(condition=[=($0, $2)], joinType=[inner]) + DruidProject(cnt=[$4], druid=[logical]) + DruidTableScan(table=[[druid, foo]], druid=[logical]) + DruidProject(k=[$0], k0=[CAST($0):BIGINT], druid=[logical]) + DruidTableScan(table=[[lookup, lookyloo]], druid=[logical]) + DruidProject(cnt=[$13], druid=[logical]) + DruidTableScan(table=[[druid, numfoo]], druid=[logical]) + +!druidPlan +{ + "queryType" : "timeseries", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "join", + "left" : { + "type" : "table", + "name" : "foo" + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "lookup", + "lookup" : "lookyloo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "k", "v0" ], + "columnTypes" : [ "STRING", "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "j0.", + "condition" : "(\"cnt\" == \"j0.v0\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "virtualColumns" : [ { + "type" : "expression", + "name" : "v0", + "expression" : "CAST(\"j0.k\", 'LONG')", + "outputType" : "LONG" + } ], + "resultFormat" : "compactedList", + "columns" : [ "v0" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "right" : { + "type" : "query", + "query" : { + "queryType" : "scan", + "dataSource" : { + "type" : "table", + "name" : "numfoo" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "resultFormat" : "compactedList", + "columns" : [ "cnt" ], + "columnTypes" : [ "LONG" ], + "granularity" : { + "type" : "all" + }, + "legacy" : false + } + }, + "rightPrefix" : "_j0.", + "condition" : "(\"v0\" == \"_j0.cnt\")", + "joinType" : "INNER" + }, + "intervals" : { + "type" : "intervals", + "intervals" : [ "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" ] + }, + "granularity" : { + "type" : "all" + }, + "aggregations" : [ { + "type" : "count", + "name" : "a0" + } ] +} +!nativePlan diff --git a/sql/src/test/resources/calcite/expected/ingest/httpExtern-logicalPlan.txt b/sql/src/test/resources/calcite/expected/ingest/httpExtern-logicalPlan.txt index 18dcef56af4..f9e4c4a5de2 100644 --- a/sql/src/test/resources/calcite/expected/ingest/httpExtern-logicalPlan.txt +++ b/sql/src/test/resources/calcite/expected/ingest/httpExtern-logicalPlan.txt @@ -1,3 +1,3 @@ LogicalInsert(target=[dst], partitionedBy=[ALL TIME], clusteredBy=[]) LogicalProject(inputs=[0..2]) - ExternalTableScan(dataSource=[{"type":"external","inputSource":{"type":"http","uris":["http://foo.com/bar.csv"],"httpAuthenticationUsername":"bob","httpAuthenticationPassword":{"type":"default","password":"secret"}},"inputFormat":{"type":"csv","columns":["x","y","z"]},"signature":[{"name":"x","type":"STRING"},{"name":"y","type":"STRING"},{"name":"z","type":"LONG"}]}]) + ExternalTableScan(dataSource=[{"type":"external","inputSource":{"type":"http","uris":["http://foo.com/bar.csv"],"httpAuthenticationUsername":"bob","httpAuthenticationPassword":{"type":"default","password":"secret"},"requestHeaders":{}},"inputFormat":{"type":"csv","columns":["x","y","z"]},"signature":[{"name":"x","type":"STRING"},{"name":"y","type":"STRING"},{"name":"z","type":"LONG"}]}]) diff --git a/sql/src/test/resources/calcite/tests/window/arrayAggWithOrderBy.sqlTest b/sql/src/test/resources/calcite/tests/window/arrayAggWithOrderBy.sqlTest index bee3baeac0c..3cbdf42af6f 100644 --- a/sql/src/test/resources/calcite/tests/window/arrayAggWithOrderBy.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/arrayAggWithOrderBy.sqlTest @@ -10,5 +10,5 @@ sql: | ORDER BY d1, f1, m1 expectedResults: - - [2,"[1.0]","[1.0]","[1.0]"] - - [2,"[1.7]","[0.1]","[2.0]"] + - [2,[1.0],[1.0],[1.0]] + - [2,[1.7],[0.1],[2.0]] diff --git a/sql/src/test/resources/calcite/tests/window/arrayConcatAgg.sqlTest b/sql/src/test/resources/calcite/tests/window/arrayConcatAgg.sqlTest index 9ec451a94d9..2a648abc17b 100644 --- a/sql/src/test/resources/calcite/tests/window/arrayConcatAgg.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/arrayConcatAgg.sqlTest @@ -9,5 +9,5 @@ sql: | GROUP BY cityName expectedResults: - - ["Horsching","[\"Horsching\"]"] - - ["Vienna","[\"Vienna\"]"] + - ["Horsching",["Horsching"]] + - ["Vienna",["Vienna"]] diff --git a/sql/src/test/resources/calcite/tests/window/rank_handling.sqlTest b/sql/src/test/resources/calcite/tests/window/rank_handling.sqlTest index 1e4de22dfca..0e66ed87460 100644 --- a/sql/src/test/resources/calcite/tests/window/rank_handling.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/rank_handling.sqlTest @@ -1,8 +1,5 @@ type: "operatorValidation" -queryContext: - maxSubqueryBytes: 100000 - sql: | SELECT __time diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest index 87873d44c48..104cb0d2422 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest @@ -2,7 +2,7 @@ type: "operatorValidation" sql: | SELECT - countryIsoCode, + countryIsoCode, CAST (FLOOR(__time TO HOUR) AS BIGINT) t, SUM(delta) delta, SUM(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY CAST (FLOOR(__time TO HOUR) AS BIGINT) ROWS BETWEEN 3 PRECEDING AND 2 FOLLOWING) windowedDelta diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/array_concat_agg/empty_over_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/array_concat_agg/empty_over_1.e new file mode 100644 index 00000000000..486bf8740fe --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/array_concat_agg/empty_over_1.e @@ -0,0 +1 @@ +Guatemala ["Guatemala"] diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/array_concat_agg/empty_over_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/array_concat_agg/empty_over_1.q new file mode 100644 index 00000000000..5448fb8d9be --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/array_concat_agg/empty_over_1.q @@ -0,0 +1,4 @@ +select +countryName, array_concat_agg(ARRAY[countryName], 10000) over () as c1 +from wikipedia where countryName='Guatemala' +group by countryName diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/array_concat_agg/only_sorting_column_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/array_concat_agg/only_sorting_column_1.e new file mode 100644 index 00000000000..8757e543a4f --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/array_concat_agg/only_sorting_column_1.e @@ -0,0 +1,10 @@ +Austria null #de.wikipedia ["N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria"] +Austria Horsching #de.wikipedia ["N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria"] +Austria Vienna #de.wikipedia ["N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria"] +Austria Vienna #es.wikipedia ["N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria"] +Austria Vienna #tr.wikipedia ["N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria"] +Republic of Korea null #ko.wikipedia ["N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea"] +Republic of Korea Jeonju #ko.wikipedia ["N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea"] +Republic of Korea Seoul #ko.wikipedia ["N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea"] +Republic of Korea null #en.wikipedia ["N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea"] +Republic of Korea null #ja.wikipedia ["N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Austria","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea","N/A","Republic of Korea"] diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/array_concat_agg/only_sorting_column_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/array_concat_agg/only_sorting_column_1.q new file mode 100644 index 00000000000..54bb76b1bd2 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/array_concat_agg/only_sorting_column_1.q @@ -0,0 +1,4 @@ +select countryName, cityName, channel, array_concat_agg(ARRAY['N/A', countryName], 10000) over (order by countryName) as c +from wikipedia +where countryName in ('Austria', 'Republic of Korea') and (cityName in ('Horsching', 'Vienna', 'Seoul', 'Jeonju') or cityName is null) +group by countryName, cityName, channel diff --git a/web-console/package-lock.json b/web-console/package-lock.json index 44d3cbae2a4..961630ad011 100644 --- a/web-console/package-lock.json +++ b/web-console/package-lock.json @@ -1,12 +1,12 @@ { "name": "web-console", - "version": "31.0.0", + "version": "32.0.0", "lockfileVersion": 2, "requires": true, "packages": { "": { "name": "web-console", - "version": "31.0.0", + "version": "32.0.0", "license": "Apache-2.0", "dependencies": { "@blueprintjs/core": "^5.10.5", diff --git a/web-console/package.json b/web-console/package.json index 9c9f38d24cb..d0e864edb8f 100644 --- a/web-console/package.json +++ b/web-console/package.json @@ -1,6 +1,6 @@ { "name": "web-console", - "version": "31.0.0", + "version": "32.0.0", "description": "A web console for Apache Druid", "author": "Apache Druid Developers ", "license": "Apache-2.0", diff --git a/web-console/pom.xml b/web-console/pom.xml index 42bf05c442f..8571e07f510 100644 --- a/web-console/pom.xml +++ b/web-console/pom.xml @@ -28,7 +28,7 @@ org.apache.druid druid - 31.0.0-SNAPSHOT + 32.0.0-SNAPSHOT diff --git a/web-console/src/components/auto-form/auto-form.tsx b/web-console/src/components/auto-form/auto-form.tsx index 91282d971b6..4c51d2cbf5a 100644 --- a/web-console/src/components/auto-form/auto-form.tsx +++ b/web-console/src/components/auto-form/auto-form.tsx @@ -57,7 +57,7 @@ export interface Field { | 'json' | 'interval' | 'custom'; - defaultValue?: any; + defaultValue?: Functor; emptyValue?: any; suggestions?: Functor; placeholder?: Functor; @@ -131,7 +131,9 @@ export class AutoForm> extends React.PureComponent const required = AutoForm.evaluateFunctor(field.required, model, false); return { required, - defaultValue: required ? undefined : field.defaultValue, + defaultValue: required + ? undefined + : AutoForm.evaluateFunctor(field.defaultValue, model as any, undefined), modelValue: deepGet(model as any, field.name), }; } diff --git a/web-console/src/console-application.tsx b/web-console/src/console-application.tsx index 36a0b8aa392..8a166b932e8 100644 --- a/web-console/src/console-application.tsx +++ b/web-console/src/console-application.tsx @@ -327,7 +327,6 @@ export class ConsoleApplication extends React.PureComponent< baseQueryContext={baseQueryContext} serverQueryContext={serverQueryContext} queryEngines={queryEngines} - allowExplain goToTask={this.goToTasksWithTaskId} getClusterCapacity={maybeGetClusterCapacity} />, diff --git a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx index 03898732bb0..65a7ed3b6c4 100644 --- a/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx +++ b/web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx @@ -1060,6 +1060,7 @@ export function getIoConfigFormFields(ingestionComboType: IngestionComboType): F name: 'inputSource.filter', label: 'Delta filter', type: 'json', + placeholder: '{"type": "=", "column": "name", "value": "foo"}', defaultValue: {}, info: ( <> @@ -1072,6 +1073,19 @@ export function getIoConfigFormFields(ingestionComboType: IngestionComboType): F ), }, + { + name: 'inputSource.snapshotVersion', + label: 'Delta snapshot version', + type: 'number', + placeholder: '(latest)', + defaultValue: {}, + info: ( + <> + The snapshot version to read from the Delta table. By default, the latest snapshot is + read. + + ), + }, ]; case 'index_parallel:hdfs': @@ -2031,7 +2045,7 @@ const TUNING_FORM_FIELDS: Field[] = [ { name: 'spec.tuningConfig.maxRowsInMemory', type: 'number', - defaultValue: 1000000, + defaultValue: (spec: IngestionSpec) => (isStreamingSpec(spec) ? 150000 : 1000000), info: <>Used in determining when intermediate persists to disk should occur., }, { diff --git a/web-console/src/druid-models/input-source/input-source.tsx b/web-console/src/druid-models/input-source/input-source.tsx index 4479150227d..eb19e4575db 100644 --- a/web-console/src/druid-models/input-source/input-source.tsx +++ b/web-console/src/druid-models/input-source/input-source.tsx @@ -652,7 +652,7 @@ export const INPUT_SOURCE_FIELDS: Field[] = [ label: 'Delta filter', type: 'json', placeholder: '{"type": "=", "column": "name", "value": "foo"}', - defined: inputSource => inputSource.type === 'delta' && deepGet(inputSource, 'filter'), + defined: typeIsKnown(KNOWN_TYPES, 'delta'), required: false, info: ( <> @@ -663,6 +663,19 @@ export const INPUT_SOURCE_FIELDS: Field[] = [ ), }, + { + name: 'snapshotVersion', + label: 'Delta snapshot version', + type: 'number', + placeholder: '(latest)', + defined: typeIsKnown(KNOWN_TYPES, 'delta'), + required: false, + info: ( + <> + The snapshot version to read from the Delta table. By default, the latest snapshot is read. + + ), + }, // sql { diff --git a/web-console/src/entry.scss b/web-console/src/entry.scss index 1304f2aa350..c980decba1e 100644 --- a/web-console/src/entry.scss +++ b/web-console/src/entry.scss @@ -64,6 +64,11 @@ body { } } +// Make segmented control buttons appear blue when they are primary +.#{$bp-ns}-segmented-control > .#{$bp-ns}-button.#{$bp-ns}-small.#{$bp-ns}-intent-primary { + color: $blue5; +} + // Prevent popover menus from being longer than 45% of available height, let them scroll instead .#{$bp-ns}-popover-content > .#{$bp-ns}-menu { max-height: 47vh; diff --git a/web-console/src/views/workbench-view/max-tasks-button/__snapshots__/max-tasks-button.spec.tsx.snap b/web-console/src/views/workbench-view/max-tasks-button/__snapshots__/max-tasks-button.spec.tsx.snap index d4326fdf6dc..89f2fdf4143 100644 --- a/web-console/src/views/workbench-view/max-tasks-button/__snapshots__/max-tasks-button.spec.tsx.snap +++ b/web-console/src/views/workbench-view/max-tasks-button/__snapshots__/max-tasks-button.spec.tsx.snap @@ -100,14 +100,13 @@ exports[`MaxTasksButton matches snapshot 1`] = ` multiline={true} onClick={[Function]} popoverProps={{}} - shouldDismissPopover={false} + shouldDismissPopover={true} text={ Max - : - uses the maximum possible tasks up to the specified limit. + : uses the maximum possible tasks up to the specified limit. } /> @@ -115,24 +114,28 @@ exports[`MaxTasksButton matches snapshot 1`] = ` active={false} disabled={false} icon="blank" - labelElement={ - - } multiline={true} onClick={[Function]} popoverProps={{}} - shouldDismissPopover={false} + shouldDismissPopover={true} text={ Auto - : - maximizes the number of tasks while staying within 512 MiB or 10,000 files per task, unless more tasks are needed to stay under the max task limit. + : uses the minimum number of tasks while + + + staying within constraints. + } /> diff --git a/web-console/src/views/workbench-view/max-tasks-button/max-tasks-button.tsx b/web-console/src/views/workbench-view/max-tasks-button/max-tasks-button.tsx index 21ba60f6b8d..509e943d8b7 100644 --- a/web-console/src/views/workbench-view/max-tasks-button/max-tasks-button.tsx +++ b/web-console/src/views/workbench-view/max-tasks-button/max-tasks-button.tsx @@ -19,34 +19,17 @@ import type { ButtonProps } from '@blueprintjs/core'; import { Button, Menu, MenuDivider, MenuItem, Popover, Position } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import type { JSX, ReactNode } from 'react'; +import type { JSX } from 'react'; import React, { useState } from 'react'; import { NumericInputDialog } from '../../../dialogs'; -import type { QueryContext, TaskAssignment } from '../../../druid-models'; +import type { QueryContext } from '../../../druid-models'; import { getQueryContextKey } from '../../../druid-models'; import { getLink } from '../../../links'; import { capitalizeFirst, deleteKeys, formatInteger, tickIcon } from '../../../utils'; -const MAX_NUM_TASK_OPTIONS = [2, 3, 4, 5, 7, 9, 11, 17, 33, 65, 129]; -const TASK_ASSIGNMENT_OPTIONS: TaskAssignment[] = ['max', 'auto']; - -const TASK_ASSIGNMENT_DESCRIPTION: Record = { - max: 'uses the maximum possible tasks up to the specified limit.', - auto: 'maximizes the number of tasks while staying within 512 MiB or 10,000 files per task, unless more tasks are needed to stay under the max task limit.', -}; - -const TASK_ASSIGNMENT_LABEL_ELEMENT: Record = { - auto: ( -